You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2017/06/13 17:04:54 UTC

[5/9] carbondata git commit: add EncodingStrategy

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 35825ef..0e7f365 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -333,21 +333,21 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
               .convertToMeasureColDataChunk(pageIndex);
       switch (msrType) {
         case SHORT:
-          msrValue = (short) measureColumnDataChunk.getMeasureDataHolder()
-              .getReadableLongValueByIndex(index);
+          msrValue = (short) measureColumnDataChunk.getColumnPage()
+              .getLong(index);
           break;
         case INT:
           msrValue =
-              (int)measureColumnDataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+              (int)measureColumnDataChunk.getColumnPage().getLong(index);
           break;
         case LONG:
           msrValue =
-              measureColumnDataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+              measureColumnDataChunk.getColumnPage().getLong(index);
           break;
         case DECIMAL:
           BigDecimal bigDecimalValue =
-              measureColumnDataChunk.getMeasureDataHolder()
-                  .getReadableBigDecimalValueByIndex(index);
+              measureColumnDataChunk.getColumnPage()
+                  .getDecimal(index);
           if (null != bigDecimalValue
               && msrColumnEvalutorInfo.getCarbonColumn().getColumnSchema().getScale()
               > bigDecimalValue.scale()) {
@@ -359,7 +359,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           break;
         default:
           msrValue =
-              measureColumnDataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+              measureColumnDataChunk.getColumnPage().getDouble(index);
       }
       record[msrColumnEvalutorInfo.getRowIndex()] =
           measureColumnDataChunk.getNullValueIndexHolder().getBitSet().get(index) ? null : msrValue;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
index c24b73c..783c29e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
@@ -579,8 +579,8 @@ public abstract class AbstractScannedResult {
    * @return measure value of long type
    */
   protected long getLongMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal][pageCounter].getMeasureDataHolder()
-        .getReadableLongValueByIndex(rowIndex);
+    return measureDataChunks[ordinal][pageCounter].getColumnPage()
+        .getLong(rowIndex);
   }
 
   /**
@@ -591,8 +591,8 @@ public abstract class AbstractScannedResult {
    * @return measure value of double type
    */
   protected double getDoubleMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal][pageCounter].getMeasureDataHolder()
-        .getReadableDoubleValueByIndex(rowIndex);
+    return measureDataChunks[ordinal][pageCounter].getColumnPage()
+        .getDouble(rowIndex);
   }
 
   /**
@@ -603,8 +603,8 @@ public abstract class AbstractScannedResult {
    * @return measure of big decimal type
    */
   protected BigDecimal getBigDecimalMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal][pageCounter].getMeasureDataHolder()
-        .getReadableBigDecimalValueByIndex(rowIndex);
+    return measureDataChunks[ordinal][pageCounter].getColumnPage()
+        .getDecimal(rowIndex);
   }
 
   public int getRowCounter() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
index c100ee2..1b0c6ca 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
@@ -46,7 +46,7 @@ public class MeasureDataVectorProcessor {
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
           vector.putInt(vectorOffset,
-              (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+              (int)dataChunk.getColumnPage().getLong(i));
           vectorOffset++;
         }
       } else {
@@ -55,7 +55,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putInt(vectorOffset,
-                (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+                (int)dataChunk.getColumnPage().getLong(i));
           }
           vectorOffset++;
         }
@@ -74,7 +74,7 @@ public class MeasureDataVectorProcessor {
         for (int i = offset; i < len; i++) {
           int currentRow = rowMapping[i];
           vector.putInt(vectorOffset,
-              (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+              (int)dataChunk.getColumnPage().getLong(currentRow));
           vectorOffset++;
         }
       } else {
@@ -84,7 +84,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putInt(vectorOffset,
-                (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+                (int)dataChunk.getColumnPage().getLong(currentRow));
           }
           vectorOffset++;
         }
@@ -104,7 +104,7 @@ public class MeasureDataVectorProcessor {
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
           vector.putShort(vectorOffset,
-              (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+              (short) dataChunk.getColumnPage().getLong(i));
           vectorOffset++;
         }
       } else {
@@ -113,7 +113,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putShort(vectorOffset,
-                (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+                (short) dataChunk.getColumnPage().getLong(i));
           }
           vectorOffset++;
         }
@@ -132,7 +132,7 @@ public class MeasureDataVectorProcessor {
         for (int i = offset; i < len; i++) {
           int currentRow = rowMapping[i];
           vector.putShort(vectorOffset,
-              (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+              (short) dataChunk.getColumnPage().getLong(currentRow));
           vectorOffset++;
         }
       } else {
@@ -142,7 +142,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putShort(vectorOffset,
-                (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+                (short) dataChunk.getColumnPage().getLong(currentRow));
           }
           vectorOffset++;
         }
@@ -162,7 +162,7 @@ public class MeasureDataVectorProcessor {
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
           vector.putLong(vectorOffset,
-              dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+              dataChunk.getColumnPage().getLong(i));
           vectorOffset++;
         }
       } else {
@@ -171,7 +171,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putLong(vectorOffset,
-                dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+                dataChunk.getColumnPage().getLong(i));
           }
           vectorOffset++;
         }
@@ -190,7 +190,7 @@ public class MeasureDataVectorProcessor {
         for (int i = offset; i < len; i++) {
           int currentRow = rowMapping[i];
           vector.putLong(vectorOffset,
-              dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+              dataChunk.getColumnPage().getLong(currentRow));
           vectorOffset++;
         }
       } else {
@@ -200,7 +200,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putLong(vectorOffset,
-                dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+                dataChunk.getColumnPage().getLong(currentRow));
           }
           vectorOffset++;
         }
@@ -224,7 +224,7 @@ public class MeasureDataVectorProcessor {
           vector.putNull(vectorOffset);
         } else {
           BigDecimal decimal =
-              dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(i);
+              dataChunk.getColumnPage().getDecimal(i);
           if (decimal.scale() < newMeasureScale) {
             decimal = decimal.setScale(newMeasureScale);
           }
@@ -250,7 +250,7 @@ public class MeasureDataVectorProcessor {
           vector.putNull(vectorOffset);
         } else {
           BigDecimal decimal =
-              dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(currentRow);
+              dataChunk.getColumnPage().getDecimal(currentRow);
           if (info.measure.getMeasure().getScale() > decimal.scale()) {
             decimal = decimal.setScale(info.measure.getMeasure().getScale());
           }
@@ -274,7 +274,7 @@ public class MeasureDataVectorProcessor {
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
           vector.putDouble(vectorOffset,
-              dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(i));
+              dataChunk.getColumnPage().getDouble(i));
           vectorOffset++;
         }
       } else {
@@ -283,7 +283,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putDouble(vectorOffset,
-                dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(i));
+                dataChunk.getColumnPage().getDouble(i));
           }
           vectorOffset++;
         }
@@ -302,7 +302,7 @@ public class MeasureDataVectorProcessor {
         for (int i = offset; i < len; i++) {
           int currentRow = rowMapping[i];
           vector.putDouble(vectorOffset,
-              dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(currentRow));
+              dataChunk.getColumnPage().getDouble(currentRow));
           vectorOffset++;
         }
       } else {
@@ -312,7 +312,7 @@ public class MeasureDataVectorProcessor {
             vector.putNull(vectorOffset);
           } else {
             vector.putDouble(vectorOffset,
-                dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(currentRow));
+                dataChunk.getColumnPage().getDouble(currentRow));
           }
           vectorOffset++;
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 1a00fd5..200d5ca 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -1389,7 +1389,7 @@ public final class CarbonUtil {
   }
 
   /**
-   * Below method will be used to convert the encode metadata to
+   * Below method will be used to convert the apply metadata to
    * ValueEncoderMeta object
    *
    * @param encoderMeta

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/main/java/org/apache/carbondata/core/util/CompressionFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CompressionFinder.java b/core/src/main/java/org/apache/carbondata/core/util/CompressionFinder.java
deleted file mode 100644
index 732d053..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/CompressionFinder.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.util;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.ValueCompressionUtil.COMPRESSION_TYPE;
-
-
-/**
- * through the size of data type,priority and compression type, select the
- * best compression type
- */
-public class CompressionFinder implements Comparable<CompressionFinder> {
-  private COMPRESSION_TYPE compType;
-
-  private DataType actualDataType;
-
-  private DataType convertedDataType;
-  /**
-   * the size of changed data
-   */
-  private int size;
-
-  private PRIORITY priority;
-
-  private DataType measureStoreType;
-
-  /**
-   * CompressionFinder constructor.
-   *
-   * @param compType
-   * @param actualDataType
-   * @param convertedDataType
-   */
-  CompressionFinder(COMPRESSION_TYPE compType, DataType actualDataType,
-      DataType convertedDataType, DataType measureStoreType) {
-    super();
-    this.compType = compType;
-    this.actualDataType = actualDataType;
-    this.convertedDataType = convertedDataType;
-    this.measureStoreType = measureStoreType;
-  }
-
-  /**
-   * CompressionFinder overloaded constructor.
-   *
-   * @param compType
-   * @param actualDataType
-   * @param convertedDataType
-   * @param priority
-   */
-
-  CompressionFinder(COMPRESSION_TYPE compType, DataType actualDataType, DataType convertedDataType,
-      PRIORITY priority, DataType measureStoreType) {
-    super();
-    this.actualDataType = actualDataType;
-    this.convertedDataType = convertedDataType;
-    this.size = ValueCompressionUtil.getSize(convertedDataType);
-    this.priority = priority;
-    this.compType = compType;
-    this.measureStoreType = measureStoreType;
-  }
-
-  @Override public boolean equals(Object obj) {
-    boolean equals = false;
-    if (obj instanceof CompressionFinder) {
-      CompressionFinder cf = (CompressionFinder) obj;
-
-      if (this.size == cf.size && this.priority == cf.priority) {
-        equals = true;
-      }
-
-    }
-    return equals;
-  }
-
-  @Override public int hashCode() {
-    final int code = 31;
-    int result = 1;
-
-    result = code * result + this.size;
-    result = code * result + ((priority == null) ? 0 : priority.hashCode());
-    return result;
-  }
-
-  @Override public int compareTo(CompressionFinder o) {
-    int returnVal = 0;
-    // the big size have high priority
-    if (this.equals(o)) {
-      returnVal = 0;
-    } else if (this.size == o.size) {
-      // the compression type priority
-      if (priority.priority > o.priority.priority) {
-        returnVal = 1;
-      } else if (priority.priority < o.priority.priority) {
-        returnVal = -1;
-      }
-
-    } else if (this.size > o.size) {
-      returnVal = 1;
-    } else {
-      returnVal = -1;
-    }
-    return returnVal;
-  }
-
-  /**
-   * Compression type priority.
-   * ACTUAL is the highest priority and DIFFNONDECIMAL is the lowest
-   * priority
-   */
-  enum PRIORITY {
-    ACTUAL(0),
-    DIFFSIZE(1),
-    MAXNONDECIMAL(2),
-    DIFFNONDECIMAL(3);
-    private int priority;
-
-    PRIORITY(int priority) {
-      this.priority = priority;
-    }
-  }
-
-  public COMPRESSION_TYPE getCompType() {
-    return compType;
-  }
-
-  public DataType getActualDataType() {
-    return actualDataType;
-  }
-
-  public DataType getConvertedDataType() {
-    return convertedDataType;
-  }
-
-  public int getSize() {
-    return size;
-  }
-
-  public PRIORITY getPriority() {
-    return priority;
-  }
-
-  public DataType getMeasureStoreType() {
-    return measureStoreType;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java b/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
deleted file mode 100644
index f5c7640..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
+++ /dev/null
@@ -1,741 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.util;
-
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.carbondata.core.compression.BigIntCompressor;
-import org.apache.carbondata.core.compression.DoubleCompressor;
-import org.apache.carbondata.core.compression.ValueCompressor;
-import org.apache.carbondata.core.datastore.compression.ReaderCompressModel;
-import org.apache.carbondata.core.datastore.compression.ValueCompressionHolder;
-import org.apache.carbondata.core.datastore.compression.decimal.CompressByteArray;
-import org.apache.carbondata.core.datastore.compression.decimal.CompressionMaxMinByte;
-import org.apache.carbondata.core.datastore.compression.decimal.CompressionMaxMinDefault;
-import org.apache.carbondata.core.datastore.compression.decimal.CompressionMaxMinInt;
-import org.apache.carbondata.core.datastore.compression.decimal.CompressionMaxMinLong;
-import org.apache.carbondata.core.datastore.compression.decimal.CompressionMaxMinShort;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalByte;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalDefault;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalInt;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalLong;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalMaxMinByte;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalMaxMinDefault;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalMaxMinInt;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalMaxMinLong;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalMaxMinShort;
-import org.apache.carbondata.core.datastore.compression.nondecimal.CompressionNonDecimalShort;
-import org.apache.carbondata.core.datastore.compression.none.CompressionNoneByte;
-import org.apache.carbondata.core.datastore.compression.none.CompressionNoneDefault;
-import org.apache.carbondata.core.datastore.compression.none.CompressionNoneInt;
-import org.apache.carbondata.core.datastore.compression.none.CompressionNoneLong;
-import org.apache.carbondata.core.datastore.compression.none.CompressionNoneShort;
-import org.apache.carbondata.core.metadata.ValueEncoderMeta;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-
-public final class ValueCompressionUtil {
-
-  private ValueCompressionUtil() {
-  }
-
-  /**
-   * decide actual type of value
-   *
-   * @param value   :the measure value
-   * @param mantissa :
-   * @return: actual type of value
-   * @see
-   */
-  private static DataType getDataType(double value, int mantissa, byte dataTypeSelected) {
-    DataType dataType = DataType.DOUBLE;
-    if (mantissa == 0) {
-      if (value <= Byte.MAX_VALUE && value >= Byte.MIN_VALUE) {
-        dataType = DataType.BYTE;
-      } else if (value <= Short.MAX_VALUE && value >= Short.MIN_VALUE) {
-        dataType = DataType.SHORT;
-      } else if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
-        dataType = DataType.INT;
-      } else if (value <= Long.MAX_VALUE && value >= Long.MIN_VALUE) {
-        dataType = DataType.LONG;
-      }
-    } else {
-      if (dataTypeSelected == 1) {
-        if (value <= Float.MAX_VALUE && value >= Float.MIN_VALUE) {
-          float floatValue = (float) value;
-          if (floatValue - value != 0) {
-            dataType = DataType.DOUBLE;
-          } else {
-            dataType = DataType.FLOAT;
-          }
-        } else if (value <= Double.MAX_VALUE && value >= Double.MIN_VALUE) {
-          dataType = DataType.DOUBLE;
-        }
-      }
-    }
-    return dataType;
-  }
-
-  /**
-   * Gives the size of datatype
-   *
-   * @param dataType : measure value type
-   * @return: the size of DataType
-   * @see
-   */
-  public static int getSize(DataType dataType) {
-    switch (dataType) {
-      case BOOLEAN:
-      case BYTE:
-        return 1;
-      case SHORT:
-        return 2;
-      case INT:
-      case FLOAT:
-        return 4;
-      default:
-        return 8;
-    }
-  }
-
-  /**
-   * get the best compression type. priority list,from high to low:
-   * COMPRESSION_TYPE.ADAPTIVE COMPRESSION_TYPE.DELTA_DOUBLE
-   * COMPRESSION_TYPE.BIGINT COMPRESSION_TYPE.DELTA_NON_DECIMAL
-   *
-   * @param maxValue : max value of one measure
-   * @param minValue : min value of one measure
-   * @param mantissa  : decimal num of one measure
-   * @return : the best compression type
-   * @see
-   */
-  public static CompressionFinder getCompressionFinder(Object maxValue, Object minValue,
-      int mantissa, DataType measureStoreType, byte dataTypeSelected) {
-    switch (measureStoreType) {
-      case DECIMAL:
-        return new CompressionFinder(COMPRESSION_TYPE.BIGDECIMAL, DataType.BYTE,
-            DataType.BYTE, measureStoreType);
-      case SHORT:
-      case INT:
-      case LONG:
-        return getLongCompressorFinder(maxValue, minValue, mantissa, dataTypeSelected,
-            measureStoreType);
-      case DOUBLE:
-        return getDoubleCompressorFinder(maxValue, minValue, mantissa, dataTypeSelected,
-            measureStoreType);
-      default:
-        throw new IllegalArgumentException("unsupported measure type");
-    }
-  }
-
-  private static CompressionFinder getDoubleCompressorFinder(Object maxValue, Object minValue,
-      int mantissa, byte dataTypeSelected, DataType measureStoreType) {
-    //Here we should use the Max abs as max to getDatatype, let's say -1 and -10000000, -1 is max,
-    //but we can't use -1 to getDatatype, we should use -10000000.
-    double absMaxValue = Math.abs((double) maxValue) >= Math.abs((double) minValue) ?
-        (double) maxValue : (double) minValue;
-    DataType adaptiveDataType = getDataType(absMaxValue, mantissa, dataTypeSelected);
-    DataType deltaDataType = getDataType((double) maxValue - (double) minValue, mantissa,
-        dataTypeSelected);
-
-    if (mantissa == 0) {
-      // short, int, long
-      int adaptiveSize = getSize(adaptiveDataType);
-      int deltaSize = getSize(deltaDataType);
-      if (adaptiveSize > deltaSize) {
-        return new CompressionFinder(COMPRESSION_TYPE.DELTA_DOUBLE, DataType.DOUBLE,
-            deltaDataType, measureStoreType);
-      } else if (adaptiveSize < deltaSize) {
-        return new CompressionFinder(COMPRESSION_TYPE.ADAPTIVE, DataType.DOUBLE,
-            deltaDataType, measureStoreType);
-      } else {
-        return new CompressionFinder(COMPRESSION_TYPE.ADAPTIVE, DataType.DOUBLE,
-            adaptiveDataType, measureStoreType);
-      }
-    } else {
-      // double
-      DataType maxNonDecDataType =
-          getDataType(Math.pow(10, mantissa) * absMaxValue, 0, dataTypeSelected);
-      DataType diffNonDecDataType =
-          getDataType(Math.pow(10, mantissa) * ((double) maxValue - (double) minValue), 0,
-              dataTypeSelected);
-
-      CompressionFinder[] finders = {
-          new CompressionFinder(COMPRESSION_TYPE.ADAPTIVE, adaptiveDataType, adaptiveDataType,
-              CompressionFinder.PRIORITY.ACTUAL, measureStoreType),
-          new CompressionFinder(COMPRESSION_TYPE.DELTA_DOUBLE, adaptiveDataType, deltaDataType,
-              CompressionFinder.PRIORITY.DIFFSIZE, measureStoreType),
-          new CompressionFinder(COMPRESSION_TYPE.BIGINT, adaptiveDataType, maxNonDecDataType,
-              CompressionFinder.PRIORITY.MAXNONDECIMAL, measureStoreType),
-          new CompressionFinder(COMPRESSION_TYPE.DELTA_NON_DECIMAL, adaptiveDataType,
-              diffNonDecDataType, CompressionFinder.PRIORITY.DIFFNONDECIMAL, measureStoreType) };
-      // sort the compressionFinder.The top have the highest priority
-      Arrays.sort(finders);
-      return finders[0];
-    }
-  }
-
-  private static CompressionFinder getLongCompressorFinder(Object maxValue, Object minValue,
-      int mantissa, byte dataTypeSelected, DataType measureStoreType) {
-    long value = Math.max(Math.abs((long)maxValue), Math.abs((long)minValue));
-    DataType adaptiveDataType = getDataType(value, mantissa, dataTypeSelected);
-    int adaptiveSize = getSize(adaptiveDataType);
-    DataType deltaDataType = null;
-    // we cannot apply compression in case actual data type of the column is long
-    // consider the scenario when max and min value are equal to is long max and min value OR
-    // when the max and min value are resulting in a value greater than long max value, then
-    // it is not possible to determine the compression type.
-    if (adaptiveDataType == DataType.LONG) {
-      deltaDataType = DataType.LONG;
-    } else {
-      deltaDataType = getDataType((long) maxValue - (long) minValue, mantissa, dataTypeSelected);
-    }
-    int deltaSize = getSize(deltaDataType);
-    if (adaptiveSize > deltaSize) {
-      return new CompressionFinder(COMPRESSION_TYPE.DELTA_DOUBLE, DataType.LONG,
-          deltaDataType, measureStoreType);
-    } else if (adaptiveSize < deltaSize) {
-      return new CompressionFinder(COMPRESSION_TYPE.ADAPTIVE, DataType.LONG,
-          deltaDataType, measureStoreType);
-    } else {
-      return new CompressionFinder(COMPRESSION_TYPE.ADAPTIVE, DataType.LONG,
-          adaptiveDataType, measureStoreType);
-    }
-  }
-
-  /**
-   * @param compType        : compression type
-   * @param values          : the data of one measure
-   * @param changedDataType : changed data type
-   * @param maxValue        : the max value of one measure
-   * @param mantissa         : the decimal length of one measure
-   * @return: the compress data array
-   * @see
-   */
-  public static Object getCompressedValues(COMPRESSION_TYPE compType, double[] values,
-      DataType changedDataType, double maxValue, int mantissa) {
-    switch (compType) {
-      case ADAPTIVE:
-        return compressNone(changedDataType, values);
-      case DELTA_DOUBLE:
-        return compressMaxMin(changedDataType, values, maxValue);
-      case BIGINT:
-        return compressNonDecimal(changedDataType, values, mantissa);
-      default:
-        return compressNonDecimalMaxMin(changedDataType, values, mantissa, maxValue);
-    }
-  }
-
-  /**
-   * It returns Compressor for given datatype
-   * @param compressorFinder
-   * @return compressor based on actualdatatype
-   */
-  public static ValueCompressor getValueCompressor(CompressionFinder compressorFinder) {
-    switch (compressorFinder.getMeasureStoreType()) {
-      case SHORT:
-      case INT:
-      case LONG:
-        return new BigIntCompressor();
-      default:
-        return new DoubleCompressor();
-    }
-  }
-
-  /**
-   * get uncompressed object
-   * @param compressionFinders : Compression types for measures
-   * @return
-   */
-  public static ValueCompressionHolder[] getValueCompressionHolder(
-      CompressionFinder[] compressionFinders) {
-    ValueCompressionHolder[] valueCompressionHolders =
-        new ValueCompressionHolder[compressionFinders.length];
-    for (int i = 0; i < compressionFinders.length; i++) {
-      valueCompressionHolders[i] = getValueCompressionHolder(compressionFinders[i]);
-    }
-    return valueCompressionHolders;
-  }
-
-
-  /**
-   *
-   * @param compressionFinder for measure other then bigdecimal
-   * @return
-   */
-  private static ValueCompressionHolder getValueCompressionHolder(
-      CompressionFinder compressionFinder) {
-    switch (compressionFinder.getMeasureStoreType()) {
-      default:
-        return getValueCompressionHolder(compressionFinder.getCompType(),
-            compressionFinder.getActualDataType(), compressionFinder.getConvertedDataType());
-    }
-  }
-
-  private static ValueCompressionHolder getValueCompressionHolder(COMPRESSION_TYPE compType,
-      DataType actualDataType, DataType changedDataType) {
-    switch (compType) {
-      case ADAPTIVE:
-        return getCompressionNone(changedDataType, actualDataType);
-      case DELTA_DOUBLE:
-        return getCompressionDecimalMaxMin(changedDataType, actualDataType);
-      case DELTA_NON_DECIMAL:
-        return getCompressionNonDecimalMaxMin(changedDataType);
-      case BIGINT:
-        return getCompressionNonDecimal(changedDataType);
-      case BIGDECIMAL:
-        return new CompressByteArray();
-      default:
-        throw new IllegalArgumentException("unsupported compType: " + compType);
-    }
-  }
-
-  /**
-   * compress data to other type for example: double -> int
-   */
-  private static Object compressNone(DataType changedDataType, double[] value) {
-    int i = 0;
-    switch (changedDataType) {
-      case BYTE:
-        byte[] result = new byte[value.length];
-
-        for (double a : value) {
-          result[i] = (byte) a;
-          i++;
-        }
-        return result;
-
-      case SHORT:
-        short[] shortResult = new short[value.length];
-
-        for (double a : value) {
-          shortResult[i] = (short) a;
-          i++;
-        }
-        return shortResult;
-
-      case INT:
-        int[] intResult = new int[value.length];
-
-        for (double a : value) {
-          intResult[i] = (int) a;
-          i++;
-        }
-        return intResult;
-
-      case LONG:
-        long[] longResult = new long[value.length];
-
-        for (double a : value) {
-          longResult[i] = (long) a;
-          i++;
-        }
-        return longResult;
-
-      case FLOAT:
-        float[] floatResult = new float[value.length];
-
-        for (double a : value) {
-          floatResult[i] = (float) a;
-          i++;
-        }
-        return floatResult;
-
-      default:
-        return value;
-    }
-  }
-
-  /**
-   * compress data to other type through sub value for example: 1. subValue =
-   * maxValue - value 2. subValue: double->int
-   */
-  private static Object compressMaxMin(DataType changedDataType, double[] value, double maxValue) {
-    int i = 0;
-    switch (changedDataType) {
-      case BYTE:
-
-        byte[] result = new byte[value.length];
-        for (double a : value) {
-          result[i] = (byte) (maxValue - a);
-          i++;
-        }
-        return result;
-
-      case SHORT:
-
-        short[] shortResult = new short[value.length];
-
-        for (double a : value) {
-          shortResult[i] = (short) (maxValue - a);
-          i++;
-        }
-        return shortResult;
-
-      case INT:
-
-        int[] intResult = new int[value.length];
-
-        for (double a : value) {
-          intResult[i] = (int) (maxValue - a);
-          i++;
-        }
-        return intResult;
-
-      case LONG:
-
-        long[] longResult = new long[value.length];
-
-        for (double a : value) {
-          longResult[i] = (long) (maxValue - a);
-          i++;
-        }
-        return longResult;
-
-      case FLOAT:
-
-        float[] floatResult = new float[value.length];
-
-        for (double a : value) {
-          floatResult[i] = (float) (maxValue - a);
-          i++;
-        }
-        return floatResult;
-
-      default:
-
-        double[] defaultResult = new double[value.length];
-
-        for (double a : value) {
-          defaultResult[i] = maxValue - a;
-          i++;
-        }
-        return defaultResult;
-
-    }
-  }
-
-  /**
-   * compress data to other type through sub value for example: 1. subValue =
-   * value * Math.pow(10, mantissa) 2. subValue: double->int
-   */
-  private static Object compressNonDecimal(DataType changedDataType, double[] value, int mantissa) {
-    int i = 0;
-    switch (changedDataType) {
-      case BYTE:
-        byte[] result = new byte[value.length];
-
-        for (double a : value) {
-          result[i] = (byte) (Math.round(Math.pow(10, mantissa) * a));
-          i++;
-        }
-        return result;
-      case SHORT:
-        short[] shortResult = new short[value.length];
-
-        for (double a : value) {
-          shortResult[i] = (short) (Math.round(Math.pow(10, mantissa) * a));
-          i++;
-        }
-        return shortResult;
-      case INT:
-
-        int[] intResult = new int[value.length];
-
-        for (double a : value) {
-          intResult[i] = (int) (Math.round(Math.pow(10, mantissa) * a));
-          i++;
-        }
-        return intResult;
-
-      case LONG:
-
-        long[] longResult = new long[value.length];
-
-        for (double a : value) {
-          longResult[i] = Math.round(Math.pow(10, mantissa) * a);
-          i++;
-        }
-        return longResult;
-
-      case FLOAT:
-
-        float[] floatResult = new float[value.length];
-
-        for (double a : value) {
-          floatResult[i] = (float) (Math.round(Math.pow(10, mantissa) * a));
-          i++;
-        }
-        return floatResult;
-
-      default:
-        double[] defaultResult = new double[value.length];
-
-        for (double a : value) {
-          defaultResult[i] = (double) (Math.round(Math.pow(10, mantissa) * a));
-          i++;
-        }
-        return defaultResult;
-    }
-  }
-
-  /**
-   * compress data to other type through sub value for example: 1. subValue =
-   * maxValue - value 2. subValue = subValue * Math.pow(10, mantissa) 3.
-   * subValue: double->int
-   */
-  private static Object compressNonDecimalMaxMin(DataType changedDataType, double[] value,
-      int mantissa, double maxValue) {
-    int i = 0;
-    BigDecimal max = BigDecimal.valueOf(maxValue);
-    switch (changedDataType) {
-      case BYTE:
-
-        byte[] result = new byte[value.length];
-
-        for (double a : value) {
-          BigDecimal val = BigDecimal.valueOf(a);
-          double diff = max.subtract(val).doubleValue();
-          result[i] = (byte) (Math.round(diff * Math.pow(10, mantissa)));
-          i++;
-        }
-        return result;
-
-      case SHORT:
-
-        short[] shortResult = new short[value.length];
-
-        for (double a : value) {
-          BigDecimal val = BigDecimal.valueOf(a);
-          double diff = max.subtract(val).doubleValue();
-          shortResult[i] = (short) (Math.round(diff * Math.pow(10, mantissa)));
-          i++;
-        }
-        return shortResult;
-
-      case INT:
-
-        int[] intResult = new int[value.length];
-
-        for (double a : value) {
-          BigDecimal val = BigDecimal.valueOf(a);
-          double diff = max.subtract(val).doubleValue();
-          intResult[i] = (int) (Math.round(diff * Math.pow(10, mantissa)));
-          i++;
-        }
-        return intResult;
-
-      case LONG:
-
-        long[] longResult = new long[value.length];
-
-        for (double a : value) {
-          BigDecimal val = BigDecimal.valueOf(a);
-          double diff = max.subtract(val).doubleValue();
-          longResult[i] = Math.round(diff * Math.pow(10, mantissa));
-          i++;
-        }
-        return longResult;
-
-      case FLOAT:
-
-        float[] floatResult = new float[value.length];
-
-        for (double a : value) {
-          BigDecimal val = BigDecimal.valueOf(a);
-          double diff = max.subtract(val).doubleValue();
-          floatResult[i] = (float) (Math.round(diff * Math.pow(10, mantissa)));
-          i++;
-        }
-        return floatResult;
-
-      default:
-
-        double[] defaultResult = new double[value.length];
-
-        for (double a : value) {
-          BigDecimal val = BigDecimal.valueOf(a);
-          double diff = max.subtract(val).doubleValue();
-          defaultResult[i] =  (Math.round(diff * Math.pow(10, mantissa)));
-          i++;
-        }
-        return defaultResult;
-
-    }
-  }
-
-  /**
-   * uncompress data for example: int -> double
-   */
-  public static ValueCompressionHolder getCompressionNone(DataType compDataType,
-      DataType actualDataType) {
-    switch (compDataType) {
-      case BYTE:
-        return new CompressionNoneByte(actualDataType);
-      case SHORT:
-        return new CompressionNoneShort(actualDataType);
-      case INT:
-        return new CompressionNoneInt(actualDataType);
-      case LONG:
-        return new CompressionNoneLong(actualDataType);
-      default:
-        return new CompressionNoneDefault(actualDataType);
-    }
-  }
-
-  /**
-   * uncompress data 1. value = maxValue - subValue 2. value: int->double
-   */
-  public static ValueCompressionHolder getCompressionDecimalMaxMin(
-      DataType compDataType, DataType actualDataType) {
-    switch (compDataType) {
-      case BYTE:
-        return new CompressionMaxMinByte(actualDataType);
-      case SHORT:
-        return new CompressionMaxMinShort(actualDataType);
-      case INT:
-        return new CompressionMaxMinInt(actualDataType);
-      case LONG:
-        return new CompressionMaxMinLong(actualDataType);
-      default:
-        return new CompressionMaxMinDefault(actualDataType);
-    }
-  }
-
-  /**
-   * uncompress data value = value/Math.pow(10, mantissa)
-   */
-  public static ValueCompressionHolder getCompressionNonDecimal(
-      DataType compDataType) {
-    switch (compDataType) {
-      case BYTE:
-        return new CompressionNonDecimalByte();
-      case SHORT:
-        return new CompressionNonDecimalShort();
-      case INT:
-        return new CompressionNonDecimalInt();
-      case LONG:
-        return new CompressionNonDecimalLong();
-      default:
-        return new CompressionNonDecimalDefault();
-    }
-  }
-
-  /**
-   * uncompress data value = (maxValue - subValue)/Math.pow(10, mantissa)
-   */
-  public static ValueCompressionHolder getCompressionNonDecimalMaxMin(
-      DataType compDataType) {
-    switch (compDataType) {
-      case BYTE:
-        return new CompressionNonDecimalMaxMinByte();
-      case SHORT:
-        return new CompressionNonDecimalMaxMinShort();
-      case INT:
-        return new CompressionNonDecimalMaxMinInt();
-      case LONG:
-        return new CompressionNonDecimalMaxMinLong();
-      default:
-        return new CompressionNonDecimalMaxMinDefault();
-    }
-  }
-
-  /**
-   * Create Value compression model for read path
-   */
-  public static ReaderCompressModel getReaderCompressModel(ValueEncoderMeta meta) {
-    ReaderCompressModel compressModel = new ReaderCompressModel();
-    CompressionFinder compressFinder =
-        getCompressionFinder(meta.getMaxValue(), meta.getMinValue(), meta.getDecimal(),
-            meta.getType(), meta.getDataTypeSelected());
-    compressModel.setValueCompressionHolder(
-          ValueCompressionUtil.getValueCompressionHolder(compressFinder));
-    compressModel.setConvertedDataType(compressFinder.getConvertedDataType());
-    compressModel.setValueEncoderMeta(meta);
-    return compressModel;
-  }
-
-  public static short[] convertToShortArray(ByteBuffer buffer, int length) {
-    buffer.rewind();
-    short[] values = new short[length / 2];
-
-    for (int i = 0; i < values.length; i++) {
-      values[i] = buffer.getShort();
-    }
-    return values;
-  }
-
-  public static int[] convertToIntArray(ByteBuffer buffer, int length) {
-    buffer.rewind();
-    int[] values = new int[length / 4];
-
-    for (int i = 0; i < values.length; i++) {
-      values[i] = buffer.getInt();
-    }
-    return values;
-  }
-
-  public static long[] convertToLongArray(ByteBuffer buffer, int length) {
-    buffer.rewind();
-    long[] values = new long[length / 8];
-    for (int i = 0; i < values.length; i++) {
-      values[i] = buffer.getLong();
-    }
-    return values;
-  }
-
-  public static double[] convertToDoubleArray(ByteBuffer buffer, int length) {
-    buffer.rewind();
-    double[] values = new double[length / 8];
-    for (int i = 0; i < values.length; i++) {
-      values[i] = buffer.getDouble();
-    }
-    return values;
-  }
-
-  /**
-   * use to identify compression type.
-   */
-  public enum COMPRESSION_TYPE {
-    /**
-     * adaptive compression based on data type
-     */
-    ADAPTIVE,
-
-    /**
-     * min max delta compression for double
-     */
-    DELTA_DOUBLE,
-
-    /**
-     * min max delta compression for short, int, long
-     */
-    DELTA_NON_DECIMAL,
-
-    /**
-     * for bigint
-     */
-    BIGINT,
-
-    /**
-     * for big decimal (PR388)
-     */
-    BIGDECIMAL
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/CompressedDataMeasureWrapperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/CompressedDataMeasureWrapperTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/CompressedDataMeasureWrapperTest.java
deleted file mode 100644
index fa5a7ee..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/CompressedDataMeasureWrapperTest.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
-
-import org.apache.carbondata.core.datastore.dataholder.CarbonReadDataHolder;
-import org.apache.carbondata.core.datastore.impl.CompressedDataMeasureDataWrapper;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class CompressedDataMeasureWrapperTest {
-  private static CompressedDataMeasureDataWrapper compressedDataMeasureDataWrapper;
-  private static CarbonReadDataHolder carbonReadDataHolder;
-  private static CarbonReadDataHolder[] carbonReadDataHolders;
-
-  @BeforeClass public static void setUp() {
-    carbonReadDataHolder = new CarbonReadDataHolder(null);
-    carbonReadDataHolders = new CarbonReadDataHolder[] { carbonReadDataHolder };
-
-  }
-
-  @Test public void testGetValues() {
-    compressedDataMeasureDataWrapper = new CompressedDataMeasureDataWrapper(carbonReadDataHolders);
-    int expected = 1;
-    int actual = compressedDataMeasureDataWrapper.getValues().length;
-    assertEquals(expected, actual);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollectorTest.java
deleted file mode 100644
index 8f902b1..0000000
--- a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollectorTest.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.collector.impl;
-
-
-public class DictionaryBasedResultCollectorTest {
-
-//  private static DictionaryBasedResultCollector dictionaryBasedResultCollector;
-//  private static BlockExecutionInfo blockExecutionInfo;
-//
-//  @BeforeClass public static void setUp() {
-//    blockExecutionInfo = new BlockExecutionInfo();
-//    KeyStructureInfo keyStructureInfo = new KeyStructureInfo();
-//    blockExecutionInfo.setKeyStructureInfo(keyStructureInfo);
-//    MeasureInfo aggregatorInfo = new MeasureInfo();
-//    aggregatorInfo.setMeasureOrdinals(new int[] { 10, 20, 30, 40 });
-//    aggregatorInfo.setMeasureExists(new boolean[] { true, false, false, false });
-//    aggregatorInfo.setDefaultValues(new Object[] { 1, 2, 3, 4 });
-//    aggregatorInfo.setMeasureDataTypes(
-//        new DataType[] { DataType.INT, DataType.TIMESTAMP, DataType.INT, DataType.INT });
-//    blockExecutionInfo.setMeasureInfo(aggregatorInfo);
-//    QueryDimension queryDimension1 = new QueryDimension("QDCol1");
-//    queryDimension1.setQueryOrder(1);
-//    ColumnSchema columnSchema = new ColumnSchema();
-//    queryDimension1.setDimension(new CarbonDimension(columnSchema, 0, 0, 0, 0));
-//    QueryDimension queryDimension2 = new QueryDimension("QDCol2");
-//    queryDimension2.setQueryOrder(2);
-//    queryDimension2.setDimension(new CarbonDimension(columnSchema, 1, 1, 1, 1));
-//    QueryDimension queryDimension3 = new QueryDimension("QDCol3");
-//    queryDimension3.setQueryOrder(3);
-//    queryDimension3.setDimension(new CarbonDimension(columnSchema, 2, 0, 0, 0));
-//    QueryDimension queryDimension4 = new QueryDimension("QDCol4");
-//    queryDimension4.setQueryOrder(4);
-//    queryDimension4.setDimension(new CarbonDimension(columnSchema, 3, 0, 0, 0));
-//    blockExecutionInfo.setQueryDimensions(
-//        new QueryDimension[] { queryDimension1, queryDimension2, queryDimension3,
-//            queryDimension4 });
-//    QueryMeasure queryMeasure1 = new QueryMeasure("QMCol1");
-//    queryMeasure1.setQueryOrder(1);
-//    QueryMeasure queryMeasure2 = new QueryMeasure("QMCol2");
-//    queryMeasure1.setQueryOrder(2);
-//    QueryMeasure queryMeasure3 = new QueryMeasure("QMCol3");
-//    queryMeasure1.setQueryOrder(3);
-//    QueryMeasure queryMeasure4 = new QueryMeasure("QMCol4");
-//    queryMeasure1.setQueryOrder(4);
-//    blockExecutionInfo.setQueryMeasures(
-//        new QueryMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
-//    Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
-//    complexDimensionInfoMap.put(1, new ArrayQueryType("name1", "parent1", 1));
-//    complexDimensionInfoMap.put(2, new ArrayQueryType("name2", "parent2", 2));
-//    complexDimensionInfoMap.put(3, new ArrayQueryType("name3", "parent3", 3));
-//    complexDimensionInfoMap.put(4, new ArrayQueryType("name4", "parent4", 4));
-//    blockExecutionInfo.setComplexDimensionInfoMap(complexDimensionInfoMap);
-//    dictionaryBasedResultCollector = new DictionaryBasedResultCollector(blockExecutionInfo);
-//  }
-//
-//  @Test public void testToCollectData() {
-//    new MockUp<CarbonUtil>() {
-//      @SuppressWarnings("unused") @Mock boolean[] getDictionaryEncodingArray(
-//          QueryDimension[] queryDimensions) {
-//        return new boolean[] { true, false, true, true };
-//      }
-//
-//      @SuppressWarnings("unused") @Mock boolean[] getDirectDictionaryEncodingArray(
-//          QueryDimension[] queryDimensions) {
-//        return new boolean[] { true, true, false, false };
-//      }
-//
-//      @SuppressWarnings("unused") @Mock boolean[] getComplexDataTypeArray(
-//          QueryDimension[] queryDimensions) {
-//        return new boolean[] { false, false, true, false };
-//      }
-//    };
-//    new MockUp<DataTypeUtil>() {
-//      @SuppressWarnings("unused") @Mock Object getDataBasedOnDataType(String data,
-//          DataType actualDataType) {
-//        return 1;
-//      }
-//    };
-//
-//    new MockUp<NonFilterQueryScannedResult>() {
-//      @SuppressWarnings("unused") @Mock int[] getDictionaryKeyIntegerArray() {
-//        this.getMockInstance().incrementCounter();
-//        System.out.println("Mocked");
-//        return new int[] { 1, 2 };
-//      }
-//
-//      @SuppressWarnings("unused") @Mock String[] getNoDictionaryKeyStringArray() {
-//        return new String[] { "1", "2" };
-//      }
-//
-//      @SuppressWarnings("unused") @Mock byte[][] getComplexTypeKeyArray() {
-//        return new byte[][] { { 1, 2 }, { 1, 2 } };
-//      }
-//
-//      @SuppressWarnings("unused") @Mock public MeasureColumnDataChunk getMeasureChunk(int ordinal) {
-//        MeasureColumnDataChunk measureColumnDataChunk = new MeasureColumnDataChunk();
-//        PresenceMeta presenceMeta = new PresenceMeta();
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        presenceMeta.setBitSet(bitSet);
-//        measureColumnDataChunk.setNullValueIndexHolder(presenceMeta);
-//        CarbonReadDataHolder carbonReadDataHolder = new CarbonReadDataHolder();
-//        carbonReadDataHolder.setReadableLongValues(new long[] { 1 });
-//        measureColumnDataChunk.setMeasureDataHolder(carbonReadDataHolder);
-//        return measureColumnDataChunk;
-//      }
-//    };
-//
-//    new MockUp<DirectDictionaryKeyGeneratorFactory>() {
-//      @SuppressWarnings("unused") @Mock DirectDictionaryGenerator getDirectDictionaryGenerator(
-//          DataType dataType) {
-//        if (dataType == DataType.TIMESTAMP) return new TimeStampDirectDictionaryGenerator(
-//            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-//        else return null;
-//      }
-//    };
-//    new MockUp<TimeStampDirectDictionaryGenerator>() {
-//      @SuppressWarnings("unused") @Mock Object getValueFromSurrogate(int key) {
-//        return 100L;
-//      }
-//    };
-//
-//    new MockUp<ArrayQueryType>() {
-//      @SuppressWarnings("unused") @Mock Object getDataBasedOnDataTypeFromSurrogates(
-//          ByteBuffer surrogateData) {
-//        return ByteBuffer.wrap("1".getBytes());
-//      }
-//    };
-//
-//    AbstractScannedResult abstractScannedResult =
-//        new NonFilterQueryScannedResult(blockExecutionInfo);
-//    abstractScannedResult.setNumberOfRows(2);
-//    List<Object[]> result = dictionaryBasedResultCollector.collectData(abstractScannedResult, 2);
-//    int expectedResult = 2;
-//    assertThat(result.size(), is(equalTo(expectedResult)));
-//  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
index 2a8f78e..9a9a773 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
@@ -108,7 +108,7 @@ public class RawBasedResultCollectorTest {
 //        measureColumnDataChunk.setNullValueIndexHolder(presenceMeta);
 //        CarbonReadDataHolder carbonReadDataHolder = new CarbonReadDataHolder();
 //        carbonReadDataHolder.setReadableLongValues(new long[] { 1 });
-//        measureColumnDataChunk.setMeasureDataHolder(carbonReadDataHolder);
+//        measureColumnDataChunk.setColumnPage(carbonReadDataHolder);
 //        return measureColumnDataChunk;
 //      }
 //    };

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/test/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResultTest.java
deleted file mode 100644
index 6753afc..0000000
--- a/core/src/test/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResultTest.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.result.impl;
-
-
-public class FilterQueryScannedResultTest {
-
-//  private static FilterQueryScannedResult filterQueryScannedResult;
-//
-//  @BeforeClass public static void setUp() {
-//    BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
-//    blockExecutionInfo.setFixedLengthKeySize(2);
-//    blockExecutionInfo.setNoDictionaryBlockIndexes(new int[] { 0, 1 });
-//    blockExecutionInfo.setDictionaryColumnBlockIndex(new int[] { 0, 1 });
-//    Map<Integer, KeyStructureInfo> columnGourpToKeyInfo = new HashMap<>();
-//    columnGourpToKeyInfo.put(1, new KeyStructureInfo());
-//    blockExecutionInfo.setColumnGroupToKeyStructureInfo(columnGourpToKeyInfo);
-//    Map<Integer, GenericQueryType> genericQueryType = new HashMap<>();
-//    genericQueryType.put(1, new ArrayQueryType("Query1", "Parent", 1));
-//    blockExecutionInfo.setComplexDimensionInfoMap(genericQueryType);
-//    blockExecutionInfo.setComplexColumnParentBlockIndexes(new int[] { 1 });
-//    QueryDimension[] queryDimensions = { new QueryDimension("Col1"), new QueryDimension("Col2") };
-//    blockExecutionInfo.setQueryDimensions(queryDimensions);
-//    filterQueryScannedResult = new FilterQueryScannedResult(blockExecutionInfo);
-//    filterQueryScannedResult.setIndexes(new int[] { 1, 2, 3, 4 });
-//    DimensionChunkAttributes dimensionChunkAttributes = new DimensionChunkAttributes();
-//    dimensionChunkAttributes.setEachRowSize(0);
-//    ColumnGroupDimensionDataChunk[] columnGroupDimensionDataChunks =
-//        { new ColumnGroupDimensionDataChunk(new byte[] { 1, 2 }, dimensionChunkAttributes),
-//            new ColumnGroupDimensionDataChunk(new byte[] { 2, 3 }, dimensionChunkAttributes) };
-//    filterQueryScannedResult.setDimensionChunks(columnGroupDimensionDataChunks);
-//    MeasureColumnDataChunk measureColumnDataChunk = new MeasureColumnDataChunk();
-//    filterQueryScannedResult
-//        .setMeasureChunks(new MeasureColumnDataChunk[] { measureColumnDataChunk });
-//  }
-//
-//  @Test public void testToGetDictionaryKeyArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public int fillChunkData(byte[] data, int offset, int rowId,
-//          KeyStructureInfo restructuringInfo) {
-//        return 1;
-//      }
-//    };
-//    byte[] keyArray = filterQueryScannedResult.getDictionaryKeyArray();
-//    byte[] expectedResult = { 0, 0 };
-//    assertThat(expectedResult, is(equalTo(keyArray)));
-//  }
-//
-//  @Test public void testToGetDictionaryKeyIntegerArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused")
-//      public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-//          KeyStructureInfo info) {
-//        return 1;
-//      }
-//    };
-//    int[] keyArray = filterQueryScannedResult.getDictionaryKeyIntegerArray();
-//    int[] expectedResult = { 0, 0 };
-//    assertThat(expectedResult, is(equalTo(keyArray)));
-//  }
-//
-//  @Test public void testToGetComplexTypeKeyArray() {
-//    new MockUp<ByteArrayOutputStream>() {
-//      @Mock @SuppressWarnings("unused") public synchronized byte toByteArray()[] {
-//        return new byte[] { 1, 2, 3 };
-//      }
-//    };
-//    new MockUp<ArrayQueryType>() {
-//      @Mock @SuppressWarnings("unused") public void parseBlocksAndReturnComplexColumnByteArray(
-//          DimensionColumnDataChunk[] dimensionColumnDataChunks, int rowNumber,
-//          DataOutputStream dataOutputStream) throws IOException {
-//      }
-//    };
-//    filterQueryScannedResult.incrementCounter();
-//    byte[][] keyArray = filterQueryScannedResult.getComplexTypeKeyArray();
-//    byte[][] expectedResult = { { 1, 2, 3 } };
-//    assertThat(expectedResult, is(equalTo(keyArray)));
-//  }
-//
-//  @Test public void testToGetNoDictionaryKeyArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public byte[] getChunkData(int rowId) {
-//        return new byte[] { 1, 2, 3 };
-//      }
-//    };
-//    byte[][] dictionaryKeyArray = filterQueryScannedResult.getNoDictionaryKeyArray();
-//    byte[][] expectedResult = { { 1, 2, 3 }, { 1, 2, 3 } };
-//    assertThat(expectedResult, is(equalTo(dictionaryKeyArray)));
-//  }
-//
-//  @Test public void testToGetNoDictionaryKeyStringArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public byte[] getChunkData(int rowId) {
-//        return "1".getBytes();
-//      }
-//    };
-//    filterQueryScannedResult.incrementCounter();
-//    String[] dictionaryKeyStringArray = filterQueryScannedResult.getNoDictionaryKeyStringArray();
-//    String[] expectedResult = { "1", "1" };
-//    assertThat(expectedResult, is(equalTo(dictionaryKeyStringArray)));
-//  }
-//
-//  @Test public void testToGetCurrentRowId() {
-//    int rowId = filterQueryScannedResult.getCurrentRowId();
-//    int expectedResult = 3;
-//    assertThat(expectedResult, is(equalTo(rowId)));
-//  }
-//
-//  @Test public void testToGetDimensionKey() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public byte[] getChunkData(int rowId) {
-//        return "1".getBytes();
-//      }
-//    };
-//    byte[] dictionaryKeyStringArray = filterQueryScannedResult.getDimensionKey(0);
-//    byte[] expectedResult = "1".getBytes();
-//    assertThat(expectedResult, is(equalTo(dictionaryKeyStringArray)));
-//  }
-//
-//  @Test public void testToGetIsNullMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public PresenceMeta getNullValueIndexHolder() {
-//        return new PresenceMeta();
-//
-//      }
-//    };
-//    new MockUp<PresenceMeta>() {
-//      @Mock @SuppressWarnings("unused") public BitSet getBitSet() {
-//        return new BitSet();
-//      }
-//    };
-//    new MockUp<BitSet>() {
-//      @Mock @SuppressWarnings("unused") public boolean get(int bitIndex) {
-//        return false;
-//      }
-//    };
-//
-//    boolean nullMeasureValue = filterQueryScannedResult.isNullMeasureValue(0);
-//    assertThat(false, is(equalTo(nullMeasureValue)));
-//  }
-//
-//  @Test public void testToGetLongMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public CarbonReadDataHolder getMeasureDataHolder() {
-//        return new CarbonReadDataHolder();
-//
-//      }
-//    };
-//    new MockUp<CarbonReadDataHolder>() {
-//      @Mock @SuppressWarnings("unused") public long getReadableLongValueByIndex(int index) {
-//        return 2L;
-//      }
-//    };
-//    long longMeasureValue = filterQueryScannedResult.getLongMeasureValue(0);
-//    long expectedResult = 2L;
-//    assertThat(expectedResult, is(equalTo(longMeasureValue)));
-//  }
-//
-//  @Test public void testToGetDoubleMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public CarbonReadDataHolder getMeasureDataHolder() {
-//        return new CarbonReadDataHolder();
-//
-//      }
-//    };
-//    new MockUp<CarbonReadDataHolder>() {
-//      @Mock @SuppressWarnings("unused") public double getReadableDoubleValueByIndex(int index) {
-//        return 2.0;
-//      }
-//    };
-//    double longMeasureValue = filterQueryScannedResult.getDoubleMeasureValue(0);
-//    double expectedResult = 2.0;
-//    assertThat(expectedResult, is(equalTo(longMeasureValue)));
-//  }
-//
-//  @Test public void testToGetBigDecimalMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public CarbonReadDataHolder getMeasureDataHolder() {
-//        return new CarbonReadDataHolder();
-//
-//      }
-//    };
-//    new MockUp<CarbonReadDataHolder>() {
-//      @Mock @SuppressWarnings("unused")
-//      public BigDecimal getReadableBigDecimalValueByIndex(int index) {
-//        return new BigDecimal(2);
-//      }
-//    };
-//    BigDecimal longMeasureValue = filterQueryScannedResult.getBigDecimalMeasureValue(0);
-//    BigDecimal expectedResult = new BigDecimal(2);
-//    assertThat(expectedResult, is(equalTo(longMeasureValue)));
-//  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/edda2483/core/src/test/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResultTest.java
deleted file mode 100644
index cbac11b..0000000
--- a/core/src/test/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResultTest.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.result.impl;
-
-
-public class NonFilterQueryScannedResultTest {
-
-//  private static NonFilterQueryScannedResult filterQueryScannedResult;
-//
-//  @BeforeClass public static void setUp() {
-//    BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
-//    blockExecutionInfo.setFixedLengthKeySize(2);
-//    blockExecutionInfo.setNoDictionaryBlockIndexes(new int[] { 0, 1 });
-//    blockExecutionInfo.setDictionaryColumnBlockIndex(new int[] { 0, 1 });
-//    Map<Integer, KeyStructureInfo> columnGourpToKeyInfo = new HashMap<>();
-//    columnGourpToKeyInfo.put(1, new KeyStructureInfo());
-//    blockExecutionInfo.setColumnGroupToKeyStructureInfo(columnGourpToKeyInfo);
-//    Map<Integer, GenericQueryType> genericQueryType = new HashMap<>();
-//    genericQueryType.put(1, new ArrayQueryType("Query1", "Parent", 1));
-//    blockExecutionInfo.setComplexDimensionInfoMap(genericQueryType);
-//    blockExecutionInfo.setComplexColumnParentBlockIndexes(new int[] { 1 });
-//    QueryDimension[] queryDimensions = { new QueryDimension("Col1"), new QueryDimension("Col2") };
-//    blockExecutionInfo.setQueryDimensions(queryDimensions);
-//    filterQueryScannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-//    DimensionChunkAttributes dimensionChunkAttributes = new DimensionChunkAttributes();
-//    dimensionChunkAttributes.setEachRowSize(0);
-//    ColumnGroupDimensionDataChunk[] columnGroupDimensionDataChunks =
-//        { new ColumnGroupDimensionDataChunk(new byte[] { 1, 2 }, dimensionChunkAttributes),
-//            new ColumnGroupDimensionDataChunk(new byte[] { 2, 3 }, dimensionChunkAttributes) };
-//    filterQueryScannedResult.setDimensionChunks(columnGroupDimensionDataChunks);
-//    MeasureColumnDataChunk measureColumnDataChunk = new MeasureColumnDataChunk();
-//    filterQueryScannedResult
-//        .setMeasureChunks(new MeasureColumnDataChunk[] { measureColumnDataChunk });
-//  }
-//
-//  @Test public void testToGetDictionaryKeyArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public int fillChunkData(byte[] data, int offset, int rowId,
-//          KeyStructureInfo restructuringInfo) {
-//        return 1;
-//      }
-//    };
-//    byte[] keyArray = filterQueryScannedResult.getDictionaryKeyArray();
-//    byte[] expectedResult = { 0, 0 };
-//    assertThat(expectedResult, is(equalTo(keyArray)));
-//  }
-//
-//  @Test public void testToGetDictionaryKeyIntegerArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused")
-//      public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-//          KeyStructureInfo info) {
-//        return 1;
-//      }
-//    };
-//    int[] keyArray = filterQueryScannedResult.getDictionaryKeyIntegerArray();
-//    int[] expectedResult = { 0, 0 };
-//    assertThat(expectedResult, is(equalTo(keyArray)));
-//  }
-//
-//  @Test public void testToGetComplexTypeKeyArray() {
-//    new MockUp<ByteArrayOutputStream>() {
-//      @Mock @SuppressWarnings("unused") public synchronized byte toByteArray()[] {
-//        return new byte[] { 1, 2, 3 };
-//      }
-//    };
-//    new MockUp<ArrayQueryType>() {
-//      @Mock @SuppressWarnings("unused") public void parseBlocksAndReturnComplexColumnByteArray(
-//          DimensionColumnDataChunk[] dimensionColumnDataChunks, int rowNumber,
-//          DataOutputStream dataOutputStream) throws IOException {
-//      }
-//    };
-//    filterQueryScannedResult.incrementCounter();
-//    byte[][] keyArray = filterQueryScannedResult.getComplexTypeKeyArray();
-//    byte[][] expectedResult = { { 1, 2, 3 } };
-//    assertThat(expectedResult, is(equalTo(keyArray)));
-//  }
-//
-//  @Test public void testToGetNoDictionaryKeyArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public byte[] getChunkData(int rowId) {
-//        return new byte[] { 1, 2, 3 };
-//      }
-//    };
-//    byte[][] dictionaryKeyArray = filterQueryScannedResult.getNoDictionaryKeyArray();
-//    byte[][] expectedResult = { { 1, 2, 3 }, { 1, 2, 3 } };
-//    assertThat(expectedResult, is(equalTo(dictionaryKeyArray)));
-//  }
-//
-//  @Test public void testToGetNoDictionaryKeyStringArray() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public byte[] getChunkData(int rowId) {
-//        return "1".getBytes();
-//      }
-//    };
-//    filterQueryScannedResult.incrementCounter();
-//    String[] dictionaryKeyStringArray = filterQueryScannedResult.getNoDictionaryKeyStringArray();
-//    String[] expectedResult = { "1", "1" };
-//    assertThat(expectedResult, is(equalTo(dictionaryKeyStringArray)));
-//  }
-//
-//  @Test public void testToGetCurrentRowId() {
-//    int rowId = filterQueryScannedResult.getCurrentRowId();
-//    int expectedResult = 2;
-//    assertThat(expectedResult, is(equalTo(rowId)));
-//  }
-//
-//  @Test public void testToGetDimensionKey() {
-//    new MockUp<ColumnGroupDimensionDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public byte[] getChunkData(int rowId) {
-//        return "1".getBytes();
-//      }
-//    };
-//    byte[] dictionaryKeyStringArray = filterQueryScannedResult.getDimensionKey(0);
-//    byte[] expectedResult = "1".getBytes();
-//    assertThat(expectedResult, is(equalTo(dictionaryKeyStringArray)));
-//  }
-//
-//  @Test public void testToGetIsNullMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public PresenceMeta getNullValueIndexHolder() {
-//        return new PresenceMeta();
-//
-//      }
-//    };
-//    new MockUp<PresenceMeta>() {
-//      @Mock @SuppressWarnings("unused") public BitSet getBitSet() {
-//        return new BitSet();
-//      }
-//    };
-//    new MockUp<BitSet>() {
-//      @Mock @SuppressWarnings("unused") public boolean get(int bitIndex) {
-//        return false;
-//      }
-//    };
-//
-//    boolean nullMeasureValue = filterQueryScannedResult.isNullMeasureValue(0);
-//    assertThat(false, is(equalTo(nullMeasureValue)));
-//  }
-//
-//  @Test public void testToGetLongMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public CarbonReadDataHolder getMeasureDataHolder() {
-//        return new CarbonReadDataHolder();
-//
-//      }
-//    };
-//    new MockUp<CarbonReadDataHolder>() {
-//      @Mock @SuppressWarnings("unused") public long getReadableLongValueByIndex(int index) {
-//        return 2L;
-//      }
-//    };
-//    long longMeasureValue = filterQueryScannedResult.getLongMeasureValue(0);
-//    long expectedResult = 2L;
-//    assertThat(expectedResult, is(equalTo(longMeasureValue)));
-//  }
-//
-//  @Test public void testToGetDoubleMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public CarbonReadDataHolder getMeasureDataHolder() {
-//        return new CarbonReadDataHolder();
-//
-//      }
-//    };
-//    new MockUp<CarbonReadDataHolder>() {
-//      @Mock @SuppressWarnings("unused") public double getReadableDoubleValueByIndex(int index) {
-//        return 2.0;
-//      }
-//    };
-//    double longMeasureValue = filterQueryScannedResult.getDoubleMeasureValue(0);
-//    double expectedResult = 2.0;
-//    assertThat(expectedResult, is(equalTo(longMeasureValue)));
-//  }
-//
-//  @Test public void testToGetBigDecimalMeasureValue() {
-//    new MockUp<MeasureColumnDataChunk>() {
-//      @Mock @SuppressWarnings("unused") public CarbonReadDataHolder getMeasureDataHolder() {
-//        return new CarbonReadDataHolder();
-//
-//      }
-//    };
-//    new MockUp<CarbonReadDataHolder>() {
-//      @Mock @SuppressWarnings("unused")
-//      public BigDecimal getReadableBigDecimalValueByIndex(int index) {
-//        return new BigDecimal(2);
-//      }
-//    };
-//    BigDecimal longMeasureValue = filterQueryScannedResult.getBigDecimalMeasureValue(0);
-//    BigDecimal expectedResult = new BigDecimal(2);
-//    assertThat(expectedResult, is(equalTo(longMeasureValue)));
-//  }
-
-}