You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2016/03/27 03:32:46 UTC

[43/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/a48d1004
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/a48d1004
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/a48d1004

Branch: refs/heads/1.3.x
Commit: a48d1004119bd95c65482cc8419e3e8dee628b7e
Parents: d517530
Author: wangxiaoyu <ro...@gmail.com>
Authored: Tue Mar 8 23:42:14 2016 +0800
Committer: Xiaoyu Wang <wa...@apache.org>
Committed: Mon Mar 21 20:00:58 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/ByteArray.java | 106 ++++++-
 .../kylin/cube/CubeCapabilityChecker.java       |   6 +-
 .../test_kylin_cube_without_slr_desc.json       |  60 +++-
 ...t_kylin_cube_without_slr_left_join_desc.json |  58 +++-
 .../kylin/measure/MeasureTypeFactory.java       |   2 +
 .../apache/kylin/measure/raw/RawAggregator.java |  65 +++++
 .../kylin/measure/raw/RawMeasureType.java       | 280 +++++++++++++++++++
 .../apache/kylin/measure/raw/RawSerializer.java | 110 ++++++++
 .../metadata/datatype/DataTypeSerializer.java   |   1 +
 .../kylin/metadata/realization/SQLDigest.java   |   1 +
 .../org/apache/kylin/metadata/tuple/Tuple.java  |   9 +-
 .../kylin/measure/raw/RawAggregatorTest.java    |  52 ++++
 .../kylin/measure/raw/RawSerializerTest.java    | 126 +++++++++
 .../kylin/query/enumerator/CubeEnumerator.java  |   1 +
 .../apache/kylin/query/test/KylinQueryTest.java |   7 +-
 .../apache/kylin/query/test/KylinTestBase.java  |   2 +-
 .../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/cubeSchema.js         |   3 +
 webapp/app/js/model/cubeConfig.js               |   2 +-
 24 files changed, 985 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/common/src/main/java/org/apache/kylin/common/util/ByteArray.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/ByteArray.java b/common/src/main/java/org/apache/kylin/common/util/ByteArray.java
index 92e0da2..afbe968 100644
--- a/common/src/main/java/org/apache/kylin/common/util/ByteArray.java
+++ b/common/src/main/java/org/apache/kylin/common/util/ByteArray.java
@@ -14,26 +14,97 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.Arrays;
+import java.io.Serializable;
 
 /**
  * @author yangli9
  */
-public class ByteArray implements Comparable<ByteArray> {
+public class ByteArray implements Comparable<ByteArray>, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    public static ByteArray allocate(int length) {
+        return new ByteArray(new byte[length]);
+    }
 
     public byte[] data;
+    private int offset;
+    private int length;
+
+    public ByteArray() {
+        this(null, 0, 0);
+    }
+
+    public ByteArray(int capacity) {
+        this(new byte[capacity], 0, capacity);
+    }
 
     public ByteArray(byte[] data) {
+        this(data, 0, data == null ? 0 : data.length);
+    }
+
+    public ByteArray(byte[] data, int offset, int length) {
         this.data = data;
+        this.offset = offset;
+        this.length = length;
+    }
+
+    public byte[] array() {
+        return data;
+    }
+
+    public int offset() {
+        return offset;
+    }
+
+    public int length() {
+        return length;
+    }
+
+    public void set(byte[] array) {
+        set(array, 0, array.length);
+    }
+
+    public void set(byte[] array, int offset, int length) {
+        this.data = array;
+        this.offset = offset;
+        this.length = length;
+    }
+
+    public void set(ByteArray o) {
+        set(o.data, o.offset, o.length);
+    }
+
+    public void set(int offset, int length) {
+        this.offset = offset;
+        this.length = length;
+    }
+
+    public void setLength(int length) {
+        this.length = length;
+    }
+
+    public ByteArray copy() {
+        ByteArray copy = new ByteArray(length);
+        copy.copyFrom(this);
+        return copy;
+    }
+
+    public void copyFrom(ByteArray other) {
+        System.arraycopy(other.array(), other.offset, data, offset, other.length);
+        this.length = other.length;
     }
 
     @Override
     public int hashCode() {
-        return Bytes.hashCode(data);
+        if (data == null)
+            return 0;
+        else
+            return Bytes.hashCode(data, offset, length);
     }
 
     @Override
@@ -44,14 +115,33 @@ public class ByteArray implements Comparable<ByteArray> {
             return false;
         if (getClass() != obj.getClass())
             return false;
-        ByteArray other = (ByteArray) obj;
-        if (!Arrays.equals(data, other.data))
+        ByteArray o = (ByteArray) obj;
+        if (this.data == null && o.data == null)
+            return true;
+        else if (this.data == null || o.data == null)
             return false;
-        return true;
+        else
+            return Bytes.equals(this.data, this.offset, this.length, o.data, o.offset, o.length);
     }
 
     @Override
     public int compareTo(ByteArray o) {
-        return Bytes.compareTo(this.data, o.data);
+        if (this.data == null && o.data == null)
+            return 0;
+        else if (this.data == null)
+            return -1;
+        else if (o.data == null)
+            return 1;
+        else
+            return Bytes.compareTo(this.data, this.offset, this.length, o.data, o.offset, o.length);
     }
+
+    @Override
+    public String toString() {
+        if (data == null)
+            return null;
+        else
+            return Bytes.toStringBinary(data, offset, length);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java b/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index 5668030..fce11c7 100644
--- a/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -64,9 +64,7 @@ public class CubeCapabilityChecker {
         Collection<FunctionDesc> unmatchedAggregations = unmatchedAggregations(aggrFunctions, cube);
 
         // try custom measure types
-        if (!unmatchedDimensions.isEmpty() || !unmatchedAggregations.isEmpty()) {
-            tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, digest, cube, result);
-        }
+        tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, digest, cube, result);
 
         // try dimension-as-measure
         if (!unmatchedAggregations.isEmpty()) {
@@ -180,8 +178,6 @@ 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;
 
             MeasureType<?> measureType = measure.getFunction().getMeasureType();
             if (measureType instanceof BasicMeasureType)

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/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 f31e358..2dcd963 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
@@ -179,6 +179,58 @@
         "returntype": "bitmap"
       },
       "dependent_measure_ref": null
+    },
+    {
+      "id": 9,
+      "name": "CAL_DT_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "CAL_DT"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
+    },
+    {
+      "id": 10,
+      "name": "LSTG_FORMAT_NAME_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "LSTG_FORMAT_NAME"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
+    },
+    {
+      "id": 11,
+      "name": "LEAF_CATEG_ID_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "LEAF_CATEG_ID"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
+    },
+    {
+      "id": 12,
+      "name": "PRICE_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "PRICE"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
     }
   ],
   "rowkey": {
@@ -263,7 +315,11 @@
               "gmv_min",
               "gmv_max",
               "trans_cnt",
-              "item_count_sum"
+              "item_count_sum",
+              "CAL_DT_RAW",
+              "LSTG_FORMAT_NAME_RAW",
+              "LEAF_CATEG_ID_RAW",
+              "PRICE_RAW"
             ]
           }
         ]
@@ -284,4 +340,4 @@
     ]
   },
   "notify_list": null
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/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 ef27b03..a2632ea 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
@@ -179,6 +179,58 @@
         "returntype": "bitmap"
       },
       "dependent_measure_ref": null
+    },
+    {
+      "id": 9,
+      "name": "CAL_DT_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "CAL_DT"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
+    },
+    {
+      "id": 10,
+      "name": "LSTG_FORMAT_NAME_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "LSTG_FORMAT_NAME"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
+    },
+    {
+      "id": 11,
+      "name": "LEAF_CATEG_ID_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "LEAF_CATEG_ID"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
+    },
+    {
+      "id": 12,
+      "name": "PRICE_RAW",
+      "function": {
+        "expression": "RAW",
+        "parameter": {
+          "type": "column",
+          "value": "PRICE"
+        },
+        "returntype": "raw"
+      },
+      "dependent_measure_ref": null
     }
   ],
   "rowkey": {
@@ -263,7 +315,11 @@
               "gmv_min",
               "gmv_max",
               "trans_cnt",
-              "item_count_sum"
+              "item_count_sum",
+              "CAL_DT_RAW",
+              "LSTG_FORMAT_NAME_RAW",
+              "LEAF_CATEG_ID_RAW",
+              "PRICE_RAW"
             ]
           }
         ]

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java b/metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
index 33d94fb..1a77f72 100644
--- a/metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
+++ b/metadata/src/main/java/org/apache/kylin/measure/MeasureTypeFactory.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.kylin.measure.basic.BasicMeasureType;
 import org.apache.kylin.measure.bitmap.BitmapMeasureType;
 import org.apache.kylin.measure.hllc.HLLCMeasureType;
+import org.apache.kylin.measure.raw.RawMeasureType;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 
@@ -48,6 +49,7 @@ abstract public class MeasureTypeFactory<T> {
         // two built-in advanced measure types
         factoryInsts.add(new HLLCMeasureType.Factory());
         factoryInsts.add(new BitmapMeasureType.Factory());
+        factoryInsts.add(new RawMeasureType.Factory());
 
         /*
          * Maybe do classpath search for more custom measure types?

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java b/metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
new file mode 100644
index 0000000..c2fe30d
--- /dev/null
+++ b/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/a48d1004/metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java b/metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
new file mode 100644
index 0000000..c6b1637
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+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.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.aggregations.size() != 0 || digest.metricColumns.size() != 0)
+            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 String rewriteFieldName = function.getRewriteFieldName();
+
+        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 fillTuplle(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);
+                if (!tuple.getAllFields().contains(literalCol.getName())) {
+                    if (tuple.getAllFields().contains(rewriteFieldName)) {
+                        int literalTupleIdx = tuple.getInfo().getFieldIndex(rewriteFieldName);
+                        tuple.getInfo().setField(literalCol.getName(), literalCol, literalCol.getType().getName(), literalTupleIdx);
+                    }
+                }
+                tuple.setDimensionValue(literalCol.getName(), 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/a48d1004/metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java b/metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
new file mode 100644
index 0000000..3d7e0b6
--- /dev/null
+++ b/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/a48d1004/metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java b/metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
index af4f97f..56e3778 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
+++ b/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/a48d1004/metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java b/metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
index a3d9763..3e124ba 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/realization/SQLDigest.java
@@ -37,6 +37,7 @@ public class SQLDigest {
     public Collection<TblColRef> filterColumns;
     public Collection<TblColRef> metricColumns;
     public Collection<FunctionDesc> aggregations;
+    public 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) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java b/metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index 321a768..9c2416e 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -96,10 +96,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);
         }
 
         setFieldObjectValue(idx, fieldValue);

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java b/metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java
new file mode 100644
index 0000000..ec59d36
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kylin/measure/raw/RawAggregatorTest.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kylin.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/a48d1004/metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java b/metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java
new file mode 100644
index 0000000..8bd09ff
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kylin/measure/raw/RawSerializerTest.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class RawSerializerTest {
+    static {
+        DataType.register("raw");
+    }
+
+    private RawSerializer rawSerializer = new RawSerializer(DataType.getInstance("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/a48d1004/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
index 66a4035..a556847 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
@@ -192,6 +192,7 @@ public class CubeEnumerator implements Enumerator<Object[]> {
         if (!sqlDigest.groupbyColumns.isEmpty() || !sqlDigest.metricColumns.isEmpty())
             return;
 
+        sqlDigest.isRawQuery = true;
         // If no group by and metric found, then it's simple query like select ... from ... where ...,
         // But we have no raw data stored, in order to return better results, we hack to output sum of metric column
         logger.info("No group by and aggregation found in this query, will hack some result for better look of output...");

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
index 2c6b155..3359798 100644
--- a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
+++ b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
@@ -80,7 +80,7 @@ public class KylinQueryTest extends KylinTestBase {
         // Cube Connection
         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);
     }
 
@@ -264,6 +264,11 @@ public class KylinQueryTest extends KylinTestBase {
         assertLimitWasEnabled();
     }
 
+    @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/a48d1004/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java b/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
index f9205b7..b3f46ca 100644
--- a/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
+++ b/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
@@ -336,7 +336,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());
         }
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a48d1004/query/src/test/resources/query/sql_raw/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_raw/query01.sql b/query/src/test/resources/query/sql_raw/query01.sql
new file mode 100644
index 0000000..54816ea
--- /dev/null
+++ b/query/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/a48d1004/query/src/test/resources/query/sql_raw/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_raw/query02.sql b/query/src/test/resources/query/sql_raw/query02.sql
new file mode 100644
index 0000000..bf29a83
--- /dev/null
+++ b/query/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/a48d1004/query/src/test/resources/query/sql_raw/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_raw/query03.sql b/query/src/test/resources/query/sql_raw/query03.sql
new file mode 100644
index 0000000..f6a8fe7
--- /dev/null
+++ b/query/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/a48d1004/query/src/test/resources/query/sql_raw/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_raw/query04.sql b/query/src/test/resources/query/sql_raw/query04.sql
new file mode 100644
index 0000000..2dc7d25
--- /dev/null
+++ b/query/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/a48d1004/query/src/test/resources/query/sql_raw/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_raw/query05.sql b/query/src/test/resources/query/sql_raw/query05.sql
new file mode 100644
index 0000000..d5502a2
--- /dev/null
+++ b/query/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/a48d1004/query/src/test/resources/query/sql_raw/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_raw/query06.sql b/query/src/test/resources/query/sql_raw/query06.sql
new file mode 100644
index 0000000..0701c1f
--- /dev/null
+++ b/query/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/a48d1004/webapp/app/js/controllers/cubeSchema.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeSchema.js b/webapp/app/js/controllers/cubeSchema.js
index ba64a1d..daa4de5 100755
--- a/webapp/app/js/controllers/cubeSchema.js
+++ b/webapp/app/js/controllers/cubeSchema.js
@@ -162,6 +162,9 @@ KylinApp.controller('CubeSchemaCtrl', function ($scope, QueryService, UserServic
         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/a48d1004/webapp/app/js/model/cubeConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeConfig.js b/webapp/app/js/model/cubeConfig.js
index 1ceecaa..92088d7 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'],
+  measureExpressions: ['SUM', 'MIN', 'MAX', 'COUNT', 'COUNT_DISTINCT', 'RAW'],
   dimensionDataTypes: ["string", "tinyint", "int", "bigint", "date"],
   cubeCapacities: ["SMALL", "MEDIUM", "LARGE"],
 //    cubePartitionTypes : ['APPEND', 'UPDATE_INSERT'],