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 2016/03/23 07:59:12 UTC

[23/50] [abbrv] kylin git commit: KYLIN-1122 Kylin support detail data query from fact table

KYLIN-1122 Kylin support detail data query from fact table


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

Branch: refs/heads/master
Commit: 139b80ac16bd08ead4a24142f6bc37a745b253bb
Parents: dba135f
Author: wangxiaoyu <ro...@gmail.com>
Authored: Sun Mar 20 20:31:58 2016 +0800
Committer: wangxiaoyu <ro...@gmail.com>
Committed: Sun Mar 20 20:31:58 2016 +0800

----------------------------------------------------------------------
 .../kylin/cube/CubeCapabilityChecker.java       |   9 +-
 .../kylin/measure/MeasureTypeFactory.java       |   2 +
 .../apache/kylin/measure/raw/RawAggregator.java |  65 +++++
 .../kylin/measure/raw/RawMeasureType.java       | 275 +++++++++++++++++++
 .../apache/kylin/measure/raw/RawSerializer.java | 110 ++++++++
 .../metadata/datatype/DataTypeSerializer.java   |   1 +
 .../kylin/metadata/realization/SQLDigest.java   |   6 +
 .../org/apache/kylin/metadata/tuple/Tuple.java  |   9 +-
 .../kylin/measure/raw/RawAggregatorTest.java    |  35 +++
 .../kylin/measure/raw/RawSerializerTest.java    | 111 ++++++++
 .../test_kylin_cube_without_slr_desc.json       |  57 +++-
 ...t_kylin_cube_without_slr_left_join_desc.json |  52 +++-
 .../apache/kylin/query/ITKylinQueryTest.java    |   7 +-
 .../org/apache/kylin/query/KylinTestBase.java   |   3 +-
 .../test/resources/query/sql_raw/query01.sql    |  19 ++
 .../test/resources/query/sql_raw/query02.sql    |  19 ++
 .../test/resources/query/sql_raw/query03.sql    |  19 ++
 .../test/resources/query/sql_raw/query04.sql    |  19 ++
 .../test/resources/query/sql_raw/query05.sql    |  19 ++
 .../test/resources/query/sql_raw/query06.sql    |  19 ++
 webapp/app/js/controllers/cubeMeasures.js       |   3 +
 webapp/app/js/model/cubeConfig.js               |   2 +-
 22 files changed, 848 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index 48b3f90..418b522 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -64,9 +64,10 @@ public class CubeCapabilityChecker {
         Collection<FunctionDesc> unmatchedAggregations = unmatchedAggregations(aggrFunctions, cube);
         
         // try custom measure types
-        if (!unmatchedDimensions.isEmpty() || !unmatchedAggregations.isEmpty()) {
+        // in RAW query, unmatchedDimensions and unmatchedAggregations will null, so can't chose RAW cube well!
+//        if (!unmatchedDimensions.isEmpty() || !unmatchedAggregations.isEmpty()) {
             tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, digest, cube, result);
-        }
+//        }
         
         // try dimension-as-measure
         if (!unmatchedAggregations.isEmpty()) {
@@ -180,8 +181,8 @@ public class CubeCapabilityChecker {
     private static void tryCustomMeasureTypes(Collection<TblColRef> unmatchedDimensions, Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult result) {
         CubeDesc cubeDesc = cube.getDescriptor();
         for (MeasureDesc measure : cubeDesc.getMeasures()) {
-            if (unmatchedDimensions.isEmpty() && unmatchedAggregations.isEmpty())
-                break;
+//            if (unmatchedDimensions.isEmpty() && unmatchedAggregations.isEmpty())
+//                break;
             
             MeasureType<?> measureType = measure.getFunction().getMeasureType();
             if (measureType instanceof BasicMeasureType)

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
index e58d82a..bd235d6 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
@@ -25,6 +25,7 @@ import org.apache.kylin.measure.basic.BasicMeasureType;
 import org.apache.kylin.measure.bitmap.BitmapMeasureType;
 import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
 import org.apache.kylin.measure.hllc.HLLCMeasureType;
+import org.apache.kylin.measure.raw.RawMeasureType;
 import org.apache.kylin.measure.topn.TopNMeasureType;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.datatype.DataTypeSerializer;
@@ -95,6 +96,7 @@ abstract public class MeasureTypeFactory<T> {
         factoryInsts.add(new HLLCMeasureType.Factory());
         factoryInsts.add(new BitmapMeasureType.Factory());
         factoryInsts.add(new TopNMeasureType.Factory());
+        factoryInsts.add(new RawMeasureType.Factory());
         factoryInsts.add(new ExtendedColumnMeasureType.Factory());
 
         /*

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
new file mode 100644
index 0000000..42b41ed
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.raw;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.measure.MeasureAggregator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * RAW data Aggregator
+ */
+public class RawAggregator extends MeasureAggregator<List<ByteArray>> {
+
+    List<ByteArray> list = null;
+
+    @Override
+    public void reset() {
+        list = null;
+    }
+
+    @Override
+    public void aggregate(List<ByteArray> value) {
+        if(value != null) {
+            if (list == null) {
+                list = new ArrayList<>(value.size());
+            }
+            list.addAll(value);
+        }
+    }
+
+    @Override
+    public List<ByteArray> getState() {
+        return list;
+    }
+
+    @Override
+    public int getMemBytesEstimate() {
+        int bytes = 0;
+        if(list != null) {
+            for (ByteArray array : list) {
+                bytes += array.length() + 1;
+            }
+        }
+        return bytes;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
new file mode 100644
index 0000000..8846f00
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.raw;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.dimension.Dictionary;
+import org.apache.kylin.measure.MeasureAggregator;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.MeasureTypeFactory;
+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.ParameterDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.realization.CapabilityResult;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.TupleInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+public class RawMeasureType extends MeasureType<List<ByteArray>> {
+
+    private static final Logger logger = LoggerFactory.getLogger(RawMeasureType.class);
+
+    public static final String FUNC_RAW = "RAW";
+    public static final String DATATYPE_RAW = "raw";
+
+    public static class Factory extends MeasureTypeFactory<List<ByteArray>> {
+
+        @Override
+        public MeasureType<List<ByteArray>> createMeasureType(String funcName, DataType dataType) {
+            return new RawMeasureType(funcName, dataType);
+        }
+
+        @Override
+        public String getAggrFunctionName() {
+            return FUNC_RAW;
+        }
+
+        @Override
+        public String getAggrDataTypeName() {
+            return DATATYPE_RAW;
+        }
+
+        @Override
+        public Class<? extends DataTypeSerializer<List<ByteArray>>> getAggrDataTypeSerializer() {
+            return RawSerializer.class;
+        }
+    }
+
+    private final DataType dataType;
+
+    public RawMeasureType(String funcName, DataType dataType) {
+        this.dataType = dataType;
+    }
+
+    public void validate(FunctionDesc functionDesc) throws IllegalArgumentException {
+        validate(functionDesc.getExpression(), functionDesc.getReturnDataType(), true);
+    }
+
+    private void validate(String funcName, DataType dataType, boolean checkDataType) {
+        if (FUNC_RAW.equals(funcName) == false)
+            throw new IllegalArgumentException();
+
+        if (DATATYPE_RAW.equals(dataType.getName()) == false)
+            throw new IllegalArgumentException();
+
+    }
+
+    @Override
+    public boolean isMemoryHungry() {
+        return true;
+    }
+
+    @Override
+    public MeasureIngester<List<ByteArray>> newIngester() {
+        return new MeasureIngester<List<ByteArray>>() {
+            //encode measure value to dictionary
+            @Override
+            public List<ByteArray> valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+                if (values.length != 1)
+                    throw new IllegalArgumentException();
+
+                //source input column value
+                String literal = values[0];
+                // encode literal using dictionary
+                TblColRef literalCol = getRawColumn(measureDesc.getFunction());
+                Dictionary<String> dictionary = dictionaryMap.get(literalCol);
+                int keyEncodedValue = dictionary.getIdFromValue(literal);
+
+                ByteArray key = new ByteArray(dictionary.getSizeOfId());
+                BytesUtil.writeUnsigned(keyEncodedValue, key.array(), key.offset(), dictionary.getSizeOfId());
+
+                List<ByteArray> valueList = new ArrayList<ByteArray>(1);
+                valueList.add(key);
+                return valueList;
+            }
+
+            //merge measure dictionary
+            @Override
+            public List<ByteArray> reEncodeDictionary(List<ByteArray> value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDicts, Map<TblColRef, Dictionary<String>> newDicts) {
+                TblColRef colRef = getRawColumn(measureDesc.getFunction());
+                Dictionary<String> sourceDict = oldDicts.get(colRef);
+                Dictionary<String> mergedDict = newDicts.get(colRef);
+
+                int valueSize = value.size();
+                byte[] newIdBuf = new byte[valueSize * mergedDict.getSizeOfId()];
+                byte[] literal = new byte[sourceDict.getSizeOfValue()];
+
+                int bufOffset = 0;
+                for (ByteArray c : value) {
+                    int oldId = BytesUtil.readUnsigned(c.array(), c.offset(), c.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.set(newIdBuf, bufOffset, mergedDict.getSizeOfId());
+                    bufOffset += mergedDict.getSizeOfId();
+                }
+                return value;
+            }
+        };
+    }
+
+    @Override
+    public MeasureAggregator<List<ByteArray>> newAggregator() {
+        return new RawAggregator();
+    }
+
+    @Override
+    public List<TblColRef> getColumnsNeedDictionary(FunctionDesc functionDesc) {
+        TblColRef literalCol = functionDesc.getParameter().getColRefs().get(0);
+        return Collections.singletonList(literalCol);
+    }
+
+    public CapabilityResult.CapabilityInfluence influenceCapabilityCheck(Collection<TblColRef> unmatchedDimensions, Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, MeasureDesc measureDesc) {
+        //is raw query
+        if (!digest.isRawQuery())
+            return null;
+
+        TblColRef rawColumn = getRawColumn(measureDesc.getFunction());
+        if (!digest.allColumns.isEmpty() && !digest.allColumns.contains(rawColumn)) {
+            return null;
+        }
+
+        unmatchedAggregations.remove(measureDesc.getFunction());
+
+        //contain one raw measure : cost * 0.9
+        return new CapabilityResult.CapabilityInfluence(){
+            @Override
+            public double suggestCostMultiplier() {
+                return 0.9;
+            }
+        };
+    }
+
+    @Override
+    public boolean needRewrite() {
+        return false;
+    }
+
+    @Override
+    public Class<?> getRewriteCalciteAggrFunctionClass() {
+        return null;
+    }
+
+    @Override
+    public void adjustSqlDigest(MeasureDesc measureDesc, SQLDigest sqlDigest) {
+        if (sqlDigest.isRawQuery()) {
+            TblColRef col = this.getRawColumn(measureDesc.getFunction());
+            ParameterDesc colParameter = new ParameterDesc();
+            colParameter.setType("column");
+            colParameter.setValue(col.getName());
+            FunctionDesc rawFunc = new FunctionDesc();
+            rawFunc.setExpression("RAW");
+            rawFunc.setParameter(colParameter);
+
+            if (sqlDigest.allColumns.contains(col)) {
+                if (measureDesc.getFunction().equals(rawFunc)) {
+                    FunctionDesc sumFunc = new FunctionDesc();
+                    sumFunc.setExpression("SUM");
+                    sumFunc.setParameter(colParameter);
+                    sqlDigest.aggregations.remove(sumFunc);
+                    sqlDigest.aggregations.add(rawFunc);
+                    logger.info("Add RAW measure on column " + col);
+                }
+                if (!sqlDigest.metricColumns.contains(col)) {
+                    sqlDigest.metricColumns.add(col);
+                }
+            }
+        }
+    }
+
+    @Override
+    public boolean needAdvancedTupleFilling() {
+        return true;
+    }
+
+    @Override
+    public void fillTupleSimply(Tuple tuple, int indexInTuple, Object measureValue) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IAdvMeasureFiller getAdvancedTupleFiller(FunctionDesc function, TupleInfo tupleInfo, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+        final TblColRef literalCol = getRawColumn(function);
+        final Dictionary<String> rawColDict = dictionaryMap.get(literalCol);
+        final int literalTupleIdx = tupleInfo.hasColumn(literalCol) ? tupleInfo.getColumnIndex(literalCol) : -1;
+
+        return new IAdvMeasureFiller() {
+            private List<ByteArray> rawList;
+            private Iterator<ByteArray> rawIterator;
+            private int expectRow = 0;
+
+            @SuppressWarnings("unchecked")
+            @Override
+            public void reload(Object measureValue) {
+                this.rawList = (List<ByteArray>) measureValue;
+                this.rawIterator = rawList.iterator();
+                this.expectRow = 0;
+            }
+
+            @Override
+            public int getNumOfRows() {
+                return rawList.size();
+            }
+
+            @Override
+            public void fillTuple(Tuple tuple, int row) {
+                if (expectRow++ != row)
+                    throw new IllegalStateException();
+
+                ByteArray raw = rawIterator.next();
+                int key = BytesUtil.readUnsigned(raw.array(), raw.offset(), raw.length());
+                String colValue = rawColDict.getValueFromId(key);
+                tuple.setDimensionValue(literalTupleIdx, colValue);
+            }
+        };
+    }
+
+    private TblColRef getRawColumn(FunctionDesc functionDesc) {
+        return functionDesc.getParameter().getColRefs().get(0);
+    }
+
+    @Override
+    public boolean onlyAggrInBaseCuboid() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
new file mode 100644
index 0000000..c23d649
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.measure.raw;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+@SuppressWarnings("unused")
+public class RawSerializer extends DataTypeSerializer<List<ByteArray>> {
+
+    //one dictionary id value need 1~4 bytes,length need 1~4 bytes, this buffer can contain 1024/(2 to 8) * 1024 values
+    //FIXME to config this and RowConstants.ROWVALUE_BUFFER_SIZE in properties file
+    public static final int RAW_BUFFER_SIZE = 1024 * 1024;//1M
+
+    private ThreadLocal<List<ByteArray>> current = new ThreadLocal<>();
+
+    public RawSerializer(DataType dataType) {
+    }
+
+    private List<ByteArray> current() {
+        List<ByteArray> l = current.get();
+        if (l == null) {
+            l = new ArrayList<ByteArray>();
+            current.set(l);
+        }
+        return l;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+        int len = 0;
+        if (in.hasRemaining()) {
+            int size = BytesUtil.readVInt(in);
+            len = in.position() - mark;
+            for (int i = 0; i < size; i++) {
+                int length = BytesUtil.peekByteArrayLength(in);
+                in.position(in.position() + length);
+                len += length;
+            }
+        }
+        in.position(mark);
+        return len;
+    }
+
+    @Override
+    public int maxLength() {
+        return RAW_BUFFER_SIZE;
+    }
+
+    @Override
+    public int getStorageBytesEstimate() {
+        return 8;
+    }
+
+    @Override
+    public void serialize(List<ByteArray> values, ByteBuffer out) {
+        if (values == null) {
+            BytesUtil.writeVInt(0, out);
+        } else {
+            BytesUtil.writeVInt(values.size(), out);
+            for (ByteArray array : values) {
+                if (!out.hasRemaining() || out.remaining() < array.length()) {
+                    throw new RuntimeException("BufferOverflow! Please use one higher cardinality column for dimension column when build RAW cube!");
+                }
+                BytesUtil.writeByteArray(BytesUtil.subarray(array.array(), array.offset(), array.offset() + array.length()), out);
+            }
+        }
+    }
+
+    @Override
+    public List<ByteArray> deserialize(ByteBuffer in) {
+        List<ByteArray> values = current();
+        values.clear();
+        int size = BytesUtil.readVInt(in);
+        if (size >= 0) {
+            for (int i = 0; i < size; i++) {
+                ByteArray ba = new ByteArray(BytesUtil.readByteArray(in));
+                if (ba.length() != 0){
+                    values.add(ba);
+                }
+            }
+        } else {
+            throw new RuntimeException("Read error data size:" + size);
+        }
+        return values;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/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
index 1b18a34..038d01b 100644
--- 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
@@ -32,6 +32,7 @@ abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
 
     final static Map<String, Class<?>> implementations = Maps.newHashMap();
     static {
+        implementations.put("char", StringSerializer.class);
         implementations.put("varchar", StringSerializer.class);
         implementations.put("decimal", BigDecimalSerializer.class);
         implementations.put("double", DoubleSerializer.class);

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
index e48cebe..08bfc8c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
@@ -45,6 +45,7 @@ public class SQLDigest {
     public Collection<FunctionDesc> aggregations;
     public Collection<MeasureDesc> sortMeasures;
     public Collection<OrderEnum> sortOrders;
+    private boolean isRawQuery = false;
 
     public SQLDigest(String factTable, TupleFilter filter, Collection<JoinDesc> joinDescs, Collection<TblColRef> allColumns, //
             Collection<TblColRef> groupbyColumns, Collection<TblColRef> filterColumns, Collection<TblColRef> aggregatedColumns, Collection<FunctionDesc> aggregateFunnc, Collection<MeasureDesc> sortMeasures, Collection<OrderEnum> sortOrders) {
@@ -58,6 +59,11 @@ public class SQLDigest {
         this.aggregations = aggregateFunnc;
         this.sortMeasures = sortMeasures;
         this.sortOrders = sortOrders;
+        this.isRawQuery = this.groupbyColumns.isEmpty() && this.metricColumns.isEmpty();
+    }
+
+    public boolean isRawQuery() {
+        return isRawQuery;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index d38aafd..c5ed0dd 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -111,10 +111,17 @@ public class Tuple implements ITuple {
         // 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)) {
+        } else if ("integer".equals(dataType) && fieldValue instanceof Number) {
             fieldValue = ((Number) fieldValue).intValue();
         } else if ("float".equals(dataType) && fieldValue instanceof BigDecimal) {
             fieldValue = ((BigDecimal) fieldValue).floatValue();
+        } else if ("date".equals(dataType) && fieldValue instanceof Long) {
+            long millis = ((Long)fieldValue).longValue();
+            fieldValue = (int) (millis / (1000 * 3600 * 24));
+        } else if ("smallint".equals(dataType) && fieldValue instanceof Long) {
+            fieldValue = ((Long)fieldValue).shortValue();
+        } else if ((!"varchar".equals(dataType) || !"char".equals(dataType)) && fieldValue instanceof String) {
+            fieldValue = convertOptiqCellValue((String)fieldValue, dataType);
         }
         values[idx] = fieldValue;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java
new file mode 100644
index 0000000..f70490d
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java
@@ -0,0 +1,35 @@
+package org.apache.kylin.measure.raw;
+
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class RawAggregatorTest {
+    private RawAggregator agg = new RawAggregator();
+
+    @Test
+    public void testNormal(){
+        int size = 100;
+        List<ByteArray> valueList = new ArrayList<ByteArray>(size);
+        for (Integer i = 0; i < size; i++) {
+            ByteArray key = new ByteArray(1);
+            BytesUtil.writeUnsigned(i, key.array(), 0, key.length());
+            valueList.add(key);
+        }
+        agg.aggregate(valueList);
+        agg.aggregate(valueList);
+        assertEquals(valueList.size() * 2, agg.getState().size());
+    }
+
+    @Test
+    public void testNull(){
+        agg.aggregate(null);
+        assertEquals(agg.getState(), null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java
new file mode 100644
index 0000000..390f48e
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java
@@ -0,0 +1,111 @@
+package org.apache.kylin.measure.raw;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.assertEquals;
+
+public class RawSerializerTest {
+    static {
+        DataType.register("raw");
+    }
+
+    private RawSerializer rawSerializer = new RawSerializer(DataType.getType("raw"));
+
+    @Test
+    public void testPeekLength() {
+        ByteBuffer out = ByteBuffer.allocate(1024 * 1024 * 128);
+        int size = 127;
+        List<ByteArray> input = getValueList(size);
+        rawSerializer.serialize(input, out);
+        out.rewind();
+        assertEquals(size * 2 + 1, rawSerializer.peekLength(out));
+
+        size = 128;
+        out.clear();
+        input = getValueList(size);
+        rawSerializer.serialize(input, out);
+        out.rewind();
+        assertEquals(size * 2 + 2, rawSerializer.peekLength(out));
+
+        size = 255;
+        out.clear();
+        input = getValueList(size);
+        rawSerializer.serialize(input, out);
+        out.rewind();
+        assertEquals(size * 2 + 2, rawSerializer.peekLength(out));
+
+        size = 256;
+        out.clear();
+        input = getValueList(size);
+        rawSerializer.serialize(input, out);
+        out.rewind();
+        assertEquals(size * 2 + 3, rawSerializer.peekLength(out));
+
+        size = 1024 * 63;
+        out.clear();
+        input = getValueList(size);
+        rawSerializer.serialize(input, out);
+        out.rewind();
+        assertEquals(size * 2 + 3, rawSerializer.peekLength(out));
+
+        size = 1024 * 64;
+        out.clear();
+        input = getValueList(size);
+        rawSerializer.serialize(input, out);
+        out.rewind();
+        assertEquals(size * 2 + 4, rawSerializer.peekLength(out));
+    }
+
+    @Test
+    public void testNormal() {
+        List<ByteArray> input = getValueList(1024);
+        List<ByteArray> output = doSAndD(input);
+        assertEquals(input, output);
+    }
+
+    @Test
+    public void testNull() {
+        List<ByteArray> output = doSAndD(null);
+        assertEquals(output.size(), 0);
+        List<ByteArray> input = new ArrayList<ByteArray>();
+        output = doSAndD(input);
+        assertEquals(input, output);
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void testOverflow() {
+        List<ByteArray> input = getValueList(512 * 1024);
+        doSAndD(input);
+    }
+
+    private List<ByteArray> doSAndD(List<ByteArray> input) {
+        ByteBuffer out = ByteBuffer.allocate(rawSerializer.maxLength());
+        out.mark();
+        rawSerializer.serialize(input, out);
+        out.reset();
+        return rawSerializer.deserialize(out);
+    }
+
+    private List<ByteArray> getValueList(int size) {
+        if (size == -1) {
+            return null;
+        }
+        List<ByteArray> valueList = new ArrayList<ByteArray>(size);
+        for (Integer i = 0; i < size; i++) {
+            ByteArray key = new ByteArray(1);
+            BytesUtil.writeUnsigned(i, key.array(), 0, key.length());
+            valueList.add(key);
+        }
+        return valueList;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
index d2bcbd2..e1d740a 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
@@ -1,4 +1,3 @@
-
 {
  
   "uuid": "9ac9b7a8-3929-4dff-b59d-2100aadc8dbf",
@@ -161,6 +160,54 @@
         "returntype": "extendedcolumn(100)"
       },
       "dependent_measure_ref": null
+    }, {
+      "name" : "CAL_DT_RAW",
+      "function" : {
+        "expression" : "RAW",
+        "parameter" : {
+          "type" : "column",
+          "value" : "CAL_DT",
+          "next_parameter" : null
+        },
+        "returntype" : "raw"
+      },
+      "dependent_measure_ref" : null
+    }, {
+      "name" : "LSTG_FORMAT_NAME_RAW",
+      "function" : {
+        "expression" : "RAW",
+        "parameter" : {
+          "type" : "column",
+          "value" : "LSTG_FORMAT_NAME",
+          "next_parameter" : null
+        },
+        "returntype" : "raw"
+      },
+      "dependent_measure_ref" : null
+    }, {
+      "name" : "LEAF_CATEG_ID_RAW",
+      "function" : {
+        "expression" : "RAW",
+        "parameter" : {
+          "type" : "column",
+          "value" : "LEAF_CATEG_ID",
+          "next_parameter" : null
+        },
+        "returntype" : "raw"
+      },
+      "dependent_measure_ref" : null
+    }, {
+      "name" : "PRICE_RAW",
+      "function" : {
+        "expression" : "RAW",
+        "parameter" : {
+          "type" : "column",
+          "value" : "PRICE",
+          "next_parameter" : null
+        },
+        "returntype" : "raw"
+      },
+      "dependent_measure_ref" : null
     }
   ],
   "rowkey": {
@@ -217,7 +264,11 @@
               "trans_cnt",
               "item_count_sum",
               "SITE_EXTENDED_1",
-              "SITE_EXTENDED_2"
+              "SITE_EXTENDED_2",
+              "CAL_DT_RAW",
+              "LSTG_FORMAT_NAME_RAW",
+              "LEAF_CATEG_ID_RAW",
+              "PRICE_RAW"
             ]
           }
         ]
@@ -281,4 +332,4 @@
   "engine_type": 2,
   "storage_type": 0,
   "partition_date_start": 0
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
index 42a242b..5e25a5b 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
@@ -159,6 +159,54 @@
       "returntype" : "topn(100)"
     },
     "dependent_measure_ref" : null
+  }, {
+    "name" : "CAL_DT_RAW",
+    "function" : {
+      "expression" : "RAW",
+      "parameter" : {
+        "type" : "column",
+        "value" : "CAL_DT",
+        "next_parameter" : null
+      },
+      "returntype" : "raw"
+    },
+    "dependent_measure_ref" : null
+  }, {
+    "name" : "LSTG_FORMAT_NAME_RAW",
+    "function" : {
+      "expression" : "RAW",
+      "parameter" : {
+        "type" : "column",
+        "value" : "LSTG_FORMAT_NAME",
+        "next_parameter" : null
+      },
+      "returntype" : "raw"
+    },
+    "dependent_measure_ref" : null
+  }, {
+    "name" : "LEAF_CATEG_ID_RAW",
+    "function" : {
+      "expression" : "RAW",
+      "parameter" : {
+        "type" : "column",
+        "value" : "LEAF_CATEG_ID",
+        "next_parameter" : null
+      },
+      "returntype" : "raw"
+    },
+    "dependent_measure_ref" : null
+  }, {
+    "name" : "PRICE_RAW",
+    "function" : {
+      "expression" : "RAW",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE",
+        "next_parameter" : null
+      },
+      "returntype" : "raw"
+    },
+    "dependent_measure_ref" : null
   } ],
   "rowkey" : {
     "rowkey_columns" : [ {
@@ -196,7 +244,7 @@
       "name" : "f1",
       "columns" : [ {
         "qualifier" : "m",
-        "measure_refs" : [ "gmv_sum", "gmv_min", "gmv_max", "trans_cnt", "item_count_sum" ]
+        "measure_refs" : [ "gmv_sum", "gmv_min", "gmv_max", "trans_cnt", "item_count_sum", "CAL_DT_RAW", "LSTG_FORMAT_NAME_RAW", "LEAF_CATEG_ID_RAW", "PRICE_RAW" ]
       } ]
     }, {
       "name" : "f2",
@@ -234,4 +282,4 @@
   "engine_type" : 2,
   "storage_type" : 2,
   "partition_date_start": 0
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index 54abd4d..3fb93d7 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -68,7 +68,7 @@ public class ITKylinQueryTest extends KylinTestBase {
         //setup cube conn
         File olapTmp = OLAPSchemaFactory.createTempOLAPJson(ProjectInstance.DEFAULT_PROJECT_NAME, config);
         Properties props = new Properties();
-        props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, "10000");
+        props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, "10001");
         cubeConnection = DriverManager.getConnection("jdbc:calcite:model=" + olapTmp.getAbsolutePath(), props);
 
         //setup h2
@@ -252,6 +252,11 @@ public class ITKylinQueryTest extends KylinTestBase {
         }
     }
 
+    @Test
+    public void testRawQuery() throws Exception {
+        this.execAndCompQuery("src/test/resources/query/sql_raw", null, true);
+    }
+
     private void assertLimitWasEnabled() {
         OLAPContext context = getFirstOLAPContext();
         assertTrue(context.storageContext.isLimitEnabled());

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java b/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
index c4a94ed..ec3e60f 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
@@ -321,7 +321,7 @@ public class KylinTestBase {
             ITable kylinTable = executeQuery(kylinConn, queryName, sql, false);
 
             // compare the result
-            Assert.assertEquals(expectRowCount, kylinTable.getRowCount());
+            Assert.assertEquals(queryName, expectRowCount, kylinTable.getRowCount());
             // Assertion.assertEquals(expectRowCount, kylinTable.getRowCount());
         }
     }
@@ -427,6 +427,7 @@ public class KylinTestBase {
             h2Conn.getConfig().setProperty(DatabaseConfig.PROPERTY_DATATYPE_FACTORY, new TestH2DataTypeFactory());
             ITable h2Table = executeDynamicQuery(h2Conn, queryName, sql, parameters, needSort);
 
+
             // compare the result
             Assertion.assertEquals(h2Table, kylinTable);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/resources/query/sql_raw/query01.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_raw/query01.sql b/kylin-it/src/test/resources/query/sql_raw/query01.sql
new file mode 100644
index 0000000..54816ea
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_raw/query01.sql
@@ -0,0 +1,19 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select PRICE from test_kylin_fact

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/resources/query/sql_raw/query02.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_raw/query02.sql b/kylin-it/src/test/resources/query/sql_raw/query02.sql
new file mode 100644
index 0000000..bf29a83
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_raw/query02.sql
@@ -0,0 +1,19 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select PRICE from test_kylin_fact where LSTG_FORMAT_NAME = 'ABIN'

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/resources/query/sql_raw/query03.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_raw/query03.sql b/kylin-it/src/test/resources/query/sql_raw/query03.sql
new file mode 100644
index 0000000..f6a8fe7
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_raw/query03.sql
@@ -0,0 +1,19 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select CAL_DT,LSTG_FORMAT_NAME,PRICE from test_kylin_fact where LSTG_FORMAT_NAME = 'ABIN'

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/resources/query/sql_raw/query04.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_raw/query04.sql b/kylin-it/src/test/resources/query/sql_raw/query04.sql
new file mode 100644
index 0000000..2dc7d25
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_raw/query04.sql
@@ -0,0 +1,19 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select CAL_DT,LSTG_FORMAT_NAME from test_kylin_fact where LSTG_FORMAT_NAME = 'ABIN'

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/resources/query/sql_raw/query05.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_raw/query05.sql b/kylin-it/src/test/resources/query/sql_raw/query05.sql
new file mode 100644
index 0000000..d5502a2
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_raw/query05.sql
@@ -0,0 +1,19 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select CAL_DT from test_kylin_fact

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/kylin-it/src/test/resources/query/sql_raw/query06.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_raw/query06.sql b/kylin-it/src/test/resources/query/sql_raw/query06.sql
new file mode 100644
index 0000000..0701c1f
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_raw/query06.sql
@@ -0,0 +1,19 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select LSTG_FORMAT_NAME,LSTG_SITE_ID,SLR_SEGMENT_CD,CAL_DT,LEAF_CATEG_ID,PRICE from test_kylin_fact
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/webapp/app/js/controllers/cubeMeasures.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeMeasures.js b/webapp/app/js/controllers/cubeMeasures.js
index 1e81e72..11cb1eb 100644
--- a/webapp/app/js/controllers/cubeMeasures.js
+++ b/webapp/app/js/controllers/cubeMeasures.js
@@ -153,6 +153,9 @@ KylinApp.controller('CubeMeasuresCtrl', function ($scope, $modal,MetaModel,cubes
         case "MAX":
           $scope.newMeasure.function.returntype = colType;
           break;
+        case "RAW":
+          $scope.newMeasure.function.returntype = "raw";
+          break;
         case "COUNT":
           $scope.newMeasure.function.returntype = "bigint";
           break;

http://git-wip-us.apache.org/repos/asf/kylin/blob/139b80ac/webapp/app/js/model/cubeConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeConfig.js b/webapp/app/js/model/cubeConfig.js
index 7747ecb..99c2ba1 100644
--- a/webapp/app/js/model/cubeConfig.js
+++ b/webapp/app/js/model/cubeConfig.js
@@ -20,7 +20,7 @@ KylinApp.constant('cubeConfig', {
 
   //~ Define metadata & class
   measureParamType: ['column', 'constant'],
-  measureExpressions: ['SUM', 'MIN', 'MAX', 'COUNT', 'COUNT_DISTINCT',"TOP_N"],
+  measureExpressions: ['SUM', 'MIN', 'MAX', 'COUNT', 'COUNT_DISTINCT',"TOP_N", 'RAW'],
   dimensionDataTypes: ["string", "tinyint", "int", "bigint", "date"],
   cubeCapacities: ["SMALL", "MEDIUM", "LARGE"],
   cubePartitionTypes: ['APPEND'],