http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/result/iterator/VectorDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/result/iterator/VectorDetailQueryResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/scan/result/iterator/VectorDetailQueryResultIterator.java
deleted file mode 100644
index 417f597..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/result/iterator/VectorDetailQueryResultIterator.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.scan.result.iterator;
-
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.carbondata.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.scan.model.QueryModel;
-import org.apache.carbondata.scan.result.vector.CarbonColumnarBatch;
-
-/**
- * It reads the data vector batch format
- */
-public class VectorDetailQueryResultIterator extends 
AbstractDetailQueryResultIterator<Object> {
-
-  private final Object lock = new Object();
-
-  public VectorDetailQueryResultIterator(List<BlockExecutionInfo> infos, 
QueryModel queryModel,
-      ExecutorService execService) {
-    super(infos, queryModel, execService);
-  }
-
-  @Override public Object next() {
-    throw new UnsupportedOperationException("call processNextBatch instaed");
-  }
-
-  public void processNextBatch(CarbonColumnarBatch columnarBatch) {
-    synchronized (lock) {
-      updateDataBlockIterator();
-      if (dataBlockIterator != null) {
-        dataBlockIterator.processNextBatch(columnarBatch);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnVector.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnVector.java
 
b/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnVector.java
deleted file mode 100644
index 0af1857..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnVector.java
+++ /dev/null
@@ -1,47 +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.scan.result.vector;
-
-import org.apache.spark.sql.types.Decimal;
-
-public interface CarbonColumnVector {
-
-  void putShort(int rowId, short value);
-
-  void putInt(int rowId, int value);
-
-  void putLong(int rowId, long value);
-
-  void putDecimal(int rowId, Decimal value, int precision);
-
-  void putDouble(int rowId, double value);
-
-  void putBytes(int rowId, byte[] value);
-
-  void putBytes(int rowId, int offset, int length, byte[] value);
-
-  void putNull(int rowId);
-
-  boolean isNull(int rowId);
-
-  void putObject(int rowId, Object obj);
-
-  Object getData(int rowId);
-
-  void reset();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnarBatch.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnarBatch.java
 
b/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnarBatch.java
deleted file mode 100644
index afc53ca..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/CarbonColumnarBatch.java
+++ /dev/null
@@ -1,62 +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.scan.result.vector;
-
-public class CarbonColumnarBatch {
-
-  public CarbonColumnVector[] columnVectors;
-
-  private int batchSize;
-
-  private int actualSize;
-
-  private int rowCounter;
-
-  public CarbonColumnarBatch(CarbonColumnVector[] columnVectors, int 
batchSize) {
-    this.columnVectors = columnVectors;
-    this.batchSize = batchSize;
-  }
-
-  public int getBatchSize() {
-    return batchSize;
-  }
-
-  public int getActualSize() {
-    return actualSize;
-  }
-
-  public void setActualSize(int actualSize) {
-    this.actualSize = actualSize;
-  }
-
-  public void reset() {
-    actualSize = 0;
-    rowCounter = 0;
-    for (int i = 0; i < columnVectors.length; i++) {
-      columnVectors[i].reset();
-    }
-  }
-
-  public int getRowCounter() {
-    return rowCounter;
-  }
-
-  public void setRowCounter(int rowCounter) {
-    this.rowCounter = rowCounter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/result/vector/ColumnVectorInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/ColumnVectorInfo.java
 
b/core/src/main/java/org/apache/carbondata/scan/result/vector/ColumnVectorInfo.java
deleted file mode 100644
index 4b0d7b3..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/ColumnVectorInfo.java
+++ /dev/null
@@ -1,41 +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.scan.result.vector;
-
-import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.scan.filter.GenericQueryType;
-import org.apache.carbondata.scan.model.QueryDimension;
-import org.apache.carbondata.scan.model.QueryMeasure;
-
-public class ColumnVectorInfo implements Comparable<ColumnVectorInfo> {
-  public int offset;
-  public int size;
-  public CarbonColumnVector vector;
-  public int vectorOffset;
-  public QueryDimension dimension;
-  public QueryMeasure measure;
-  public int ordinal;
-  public DirectDictionaryGenerator directDictionaryGenerator;
-  public MeasureDataVectorProcessor.MeasureVectorFiller measureVectorFiller;
-  public GenericQueryType genericQueryType;
-
-  @Override public int compareTo(ColumnVectorInfo o) {
-    return ordinal - o.ordinal;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/MeasureDataVectorProcessor.java
 
b/core/src/main/java/org/apache/carbondata/scan/result/vector/MeasureDataVectorProcessor.java
deleted file mode 100644
index e713e1b..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/MeasureDataVectorProcessor.java
+++ /dev/null
@@ -1,268 +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.scan.result.vector;
-
-import java.math.BigDecimal;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk;
-import org.apache.carbondata.core.metadata.DataType;
-
-import org.apache.spark.sql.types.Decimal;
-
-public class MeasureDataVectorProcessor {
-
-  public interface MeasureVectorFiller {
-
-    void fillMeasureVector(MeasureColumnDataChunk dataChunk, ColumnVectorInfo 
info);
-
-    void fillMeasureVectorForFilter(int[] rowMapping, MeasureColumnDataChunk 
dataChunk,
-        ColumnVectorInfo info);
-  }
-
-  public static class IntegralMeasureVectorFiller implements 
MeasureVectorFiller {
-
-    @Override
-    public void fillMeasureVector(MeasureColumnDataChunk dataChunk, 
ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putInt(vectorOffset,
-              
(int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
-        }
-        vectorOffset++;
-      }
-    }
-
-    @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, 
MeasureColumnDataChunk dataChunk,
-        ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putInt(vectorOffset,
-              
(int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
-        }
-        vectorOffset++;
-      }
-    }
-  }
-
-  public static class ShortMeasureVectorFiller implements MeasureVectorFiller {
-
-    @Override
-    public void fillMeasureVector(MeasureColumnDataChunk dataChunk, 
ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putShort(vectorOffset,
-              (short) 
dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
-        }
-        vectorOffset++;
-      }
-    }
-
-    @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, 
MeasureColumnDataChunk dataChunk,
-        ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putShort(vectorOffset,
-              (short) 
dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
-        }
-        vectorOffset++;
-      }
-    }
-  }
-
-  public static class LongMeasureVectorFiller implements MeasureVectorFiller {
-
-    @Override
-    public void fillMeasureVector(MeasureColumnDataChunk dataChunk, 
ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putLong(vectorOffset,
-              dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
-        }
-        vectorOffset++;
-      }
-    }
-
-    @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, 
MeasureColumnDataChunk dataChunk,
-        ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putLong(vectorOffset,
-              
dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
-        }
-        vectorOffset++;
-      }
-    }
-  }
-
-  public static class DecimalMeasureVectorFiller implements 
MeasureVectorFiller {
-
-    @Override
-    public void fillMeasureVector(MeasureColumnDataChunk dataChunk, 
ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      int precision = info.measure.getMeasure().getPrecision();
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
-          BigDecimal decimal =
-              
dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(i);
-          Decimal toDecimal = 
org.apache.spark.sql.types.Decimal.apply(decimal);
-          vector.putDecimal(vectorOffset, toDecimal, precision);
-        }
-        vectorOffset++;
-      }
-    }
-
-    @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, 
MeasureColumnDataChunk dataChunk,
-        ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      int precision = info.measure.getMeasure().getPrecision();
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
-          BigDecimal decimal =
-              
dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(currentRow);
-          Decimal toDecimal = 
org.apache.spark.sql.types.Decimal.apply(decimal);
-          vector.putDecimal(vectorOffset, toDecimal, precision);
-        }
-        vectorOffset++;
-      }
-    }
-  }
-
-  public static class DefaultMeasureVectorFiller implements 
MeasureVectorFiller {
-
-    @Override
-    public void fillMeasureVector(MeasureColumnDataChunk dataChunk, 
ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putDouble(vectorOffset,
-              
dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(i));
-        }
-        vectorOffset++;
-      }
-    }
-
-    @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, 
MeasureColumnDataChunk dataChunk,
-        ColumnVectorInfo info) {
-      int offset = info.offset;
-      int len = offset + info.size;
-      int vectorOffset = info.vectorOffset;
-      CarbonColumnVector vector = info.vector;
-      BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
-          vector.putDouble(vectorOffset,
-              
dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(currentRow));
-        }
-        vectorOffset++;
-      }
-    }
-  }
-
-  public static class MeasureVectorFillerFactory {
-
-    public static MeasureVectorFiller getMeasureVectorFiller(DataType 
dataType) {
-      switch (dataType) {
-        case SHORT:
-          return new ShortMeasureVectorFiller();
-        case INT:
-          return new IntegralMeasureVectorFiller();
-        case LONG:
-          return new LongMeasureVectorFiller();
-        case DECIMAL:
-          return new DecimalMeasureVectorFiller();
-        default:
-          return new DefaultMeasureVectorFiller();
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/result/vector/impl/CarbonColumnVectorImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/impl/CarbonColumnVectorImpl.java
 
b/core/src/main/java/org/apache/carbondata/scan/result/vector/impl/CarbonColumnVectorImpl.java
deleted file mode 100644
index 8d0c277..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/result/vector/impl/CarbonColumnVectorImpl.java
+++ /dev/null
@@ -1,154 +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.scan.result.vector.impl;
-
-import java.util.Arrays;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.metadata.DataType;
-import org.apache.carbondata.scan.result.vector.CarbonColumnVector;
-
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.unsafe.types.UTF8String;
-
-public class CarbonColumnVectorImpl implements CarbonColumnVector {
-
-  private Object[] data;
-
-  private int[] ints;
-
-  private long[] longs;
-
-  private Decimal[] decimals;
-
-  private byte[][] bytes;
-
-  private double[] doubles;
-
-  private BitSet nullBytes;
-
-  private DataType dataType;
-
-  public CarbonColumnVectorImpl(int batchSize, DataType dataType) {
-    nullBytes = new BitSet(batchSize);
-    this.dataType = dataType;
-    switch (dataType) {
-      case INT:
-        ints = new int[batchSize];
-        break;
-      case LONG:
-        longs = new long[batchSize];
-        break;
-      case DOUBLE:
-        doubles = new double[batchSize];
-        break;
-      case STRING:
-        bytes = new byte[batchSize][];
-        break;
-      case DECIMAL:
-        decimals = new Decimal[batchSize];
-        break;
-      default:
-        data = new Object[batchSize];
-    }
-  }
-
-  @Override public void putShort(int rowId, short value) {
-
-  }
-
-  @Override public void putInt(int rowId, int value) {
-    ints[rowId] = value;
-  }
-
-  @Override public void putLong(int rowId, long value) {
-    longs[rowId] = value;
-  }
-
-  @Override public void putDecimal(int rowId, Decimal value, int precision) {
-    decimals[rowId] = value;
-  }
-
-  @Override public void putDouble(int rowId, double value) {
-    doubles[rowId] = value;
-  }
-
-  @Override public void putBytes(int rowId, byte[] value) {
-    bytes[rowId] = value;
-  }
-
-  @Override public void putBytes(int rowId, int offset, int length, byte[] 
value) {
-
-  }
-
-  @Override public void putNull(int rowId) {
-    nullBytes.set(rowId);
-  }
-
-  @Override public boolean isNull(int rowId) {
-    return nullBytes.get(rowId);
-  }
-
-  @Override public void putObject(int rowId, Object obj) {
-    data[rowId] = obj;
-  }
-
-  @Override public Object getData(int rowId) {
-    if (nullBytes.get(rowId)) {
-      return null;
-    }
-    switch (dataType) {
-      case INT:
-        return ints[rowId];
-      case LONG:
-        return longs[rowId];
-      case DOUBLE:
-        return doubles[rowId];
-      case STRING:
-        return UTF8String.fromBytes(bytes[rowId]);
-      case DECIMAL:
-        return decimals[rowId];
-      default:
-        return data[rowId];
-    }
-  }
-
-  @Override public void reset() {
-    nullBytes.clear();
-    switch (dataType) {
-      case INT:
-        Arrays.fill(ints, 0);
-        break;
-      case LONG:
-        Arrays.fill(longs, 0);
-        break;
-      case DOUBLE:
-        Arrays.fill(doubles, 0);
-        break;
-      case STRING:
-        Arrays.fill(bytes, null);
-        break;
-      case DECIMAL:
-        Arrays.fill(decimals, null);
-        break;
-      default:
-        Arrays.fill(data, null);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
 
b/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
deleted file mode 100644
index 5343cee..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/scanner/AbstractBlockletScanner.java
+++ /dev/null
@@ -1,93 +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.scan.scanner;
-
-import java.io.IOException;
-
-import 
org.apache.carbondata.common.iudprocessor.iuddata.BlockletDeleteDeltaCacheLoader;
-import 
org.apache.carbondata.common.iudprocessor.iuddata.DeleteDeltaCacheLoaderIntf;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.scan.executor.infos.BlockExecutionInfo;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.scan.result.AbstractScannedResult;
-
-/**
- * Blocklet scanner class to process the block
- */
-public abstract class AbstractBlockletScanner implements BlockletScanner {
-
-  /**
-   * scanner result
-   */
-  protected AbstractScannedResult scannedResult;
-
-  /**
-   * block execution info
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  public QueryStatisticsModel queryStatisticsModel;
-
-  public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) {
-    this.blockExecutionInfo = tableBlockExecutionInfos;
-  }
-
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder 
blocksChunkHolder)
-      throws IOException, FilterUnsupportedException {
-    fillKeyValue(blocksChunkHolder);
-    return scannedResult;
-  }
-
-  protected void fillKeyValue(BlocksChunkHolder blocksChunkHolder) throws 
IOException {
-
-    QueryStatistic totalBlockletStatistic = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
-            .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    
totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-            totalBlockletStatistic.getCount() + 1);
-    
queryStatisticsModel.getRecorder().recordStatistics(totalBlockletStatistic);
-    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel
-            
.getStatisticsTypeAndObjMap().get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
-    validScannedBlockletStatistic
-            
.addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
-                    validScannedBlockletStatistic.getCount() + 1);
-    
queryStatisticsModel.getRecorder().recordStatistics(validScannedBlockletStatistic);
-    scannedResult.reset();
-    scannedResult.setNumberOfRows(blocksChunkHolder.getDataBlock().nodeSize());
-    scannedResult.setBlockletId(
-              blockExecutionInfo.getBlockId() + 
CarbonCommonConstants.FILE_SEPARATOR
-                      + blocksChunkHolder.getDataBlock().nodeNumber());
-    scannedResult.setDimensionChunks(blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedDimensionBlocksIndexes()));
-    scannedResult.setMeasureChunks(blocksChunkHolder.getDataBlock()
-            .getMeasureChunks(blocksChunkHolder.getFileReader(),
-                blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()));
-    // loading delete data cache in blockexecutioninfo instance
-    DeleteDeltaCacheLoaderIntf deleteCacheLoader =
-        new BlockletDeleteDeltaCacheLoader(scannedResult.getBlockletId(),
-            blocksChunkHolder.getDataBlock(), 
blockExecutionInfo.getAbsoluteTableIdentifier());
-    deleteCacheLoader.loadDeleteDeltaFileDataToCache();
-    scannedResult
-        
.setBlockletDeleteDeltaCache(blocksChunkHolder.getDataBlock().getDeleteDeltaDataCache());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/scanner/BlockletScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/scanner/BlockletScanner.java 
b/core/src/main/java/org/apache/carbondata/scan/scanner/BlockletScanner.java
deleted file mode 100644
index 39a20b2..0000000
--- a/core/src/main/java/org/apache/carbondata/scan/scanner/BlockletScanner.java
+++ /dev/null
@@ -1,42 +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.scan.scanner;
-
-import java.io.IOException;
-
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.scan.result.AbstractScannedResult;
-
-/**
- * Interface for processing the block
- * Processing can be filter based processing or non filter based processing
- */
-public interface BlockletScanner {
-
-  /**
-   * Below method will used to process the block data and get the scanned 
result
-   *
-   * @param blocksChunkHolder block chunk which holds the block data
-   * @return scannerResult
-   * result after processing
-   */
-  AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws IOException, FilterUnsupportedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java 
b/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
deleted file mode 100644
index e947e62..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/scanner/impl/FilterScanner.java
+++ /dev/null
@@ -1,200 +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.scan.scanner.impl;
-
-import java.io.IOException;
-import java.util.BitSet;
-
-import 
org.apache.carbondata.common.iudprocessor.iuddata.BlockletDeleteDeltaCacheLoader;
-import 
org.apache.carbondata.common.iudprocessor.iuddata.DeleteDeltaCacheLoaderIntf;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastorage.store.FileHolder;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.scan.executor.infos.BlockExecutionInfo;
-import 
org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.scan.result.AbstractScannedResult;
-import org.apache.carbondata.scan.result.impl.FilterQueryScannedResult;
-import org.apache.carbondata.scan.scanner.AbstractBlockletScanner;
-
-/**
- * Below class will be used for filter query processing
- * this class will be first apply the filter then it will read the block if
- * required and return the scanned result
- */
-public class FilterScanner extends AbstractBlockletScanner {
-
-  /**
-   * filter tree
-   */
-  private FilterExecuter filterExecuter;
-  /**
-   * this will be used to apply min max
-   * this will be useful for dimension column which is on the right side
-   * as node finder will always give tentative blocks, if column data stored 
individually
-   * and data is in sorted order then we can check whether filter is in the 
range of min max or not
-   * if it present then only we can apply filter on complete data.
-   * this will be very useful in case of sparse data when rows are
-   * repeating.
-   */
-  private boolean isMinMaxEnabled;
-
-  private QueryStatisticsModel queryStatisticsModel;
-
-  public FilterScanner(BlockExecutionInfo blockExecutionInfo,
-      QueryStatisticsModel queryStatisticsModel) {
-    super(blockExecutionInfo);
-    scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
-    // to check whether min max is enabled or not
-    String minMaxEnableValue = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
-            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
-    if (null != minMaxEnableValue) {
-      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
-    }
-    // get the filter tree
-    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
-    this.queryStatisticsModel = queryStatisticsModel;
-  }
-
-  /**
-   * Below method will be used to process the block
-   *
-   * @param blocksChunkHolder block chunk holder which holds the data
-   * @throws FilterUnsupportedException
-   */
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder 
blocksChunkHolder)
-      throws IOException, FilterUnsupportedException {
-    fillScannedResult(blocksChunkHolder);
-    return scannedResult;
-  }
-
-  /**
-   * This method will process the data in below order
-   * 1. first apply min max on the filter tree and check whether any of the 
filter
-   * is fall on the range of min max, if not then return empty result
-   * 2. If filter falls on min max range then apply filter on actual
-   * data and get the filtered row index
-   * 3. if row index is empty then return the empty result
-   * 4. if row indexes is not empty then read only those blocks(measure or 
dimension)
-   * which was present in the query but not present in the filter, as while 
applying filter
-   * some of the blocks where already read and present in chunk holder so not 
need to
-   * read those blocks again, this is to avoid reading of same blocks which 
was already read
-   * 5. Set the blocks and filter indexes to result
-   *
-   * @param blocksChunkHolder
-   * @throws FilterUnsupportedException
-   */
-  private void fillScannedResult(BlocksChunkHolder blocksChunkHolder)
-      throws FilterUnsupportedException, IOException {
-    scannedResult.reset();
-    scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR 
+ blocksChunkHolder
-            .getDataBlock().nodeNumber());
-    // apply min max
-    if (isMinMaxEnabled) {
-      BitSet bitSet = this.filterExecuter
-          
.isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-              blocksChunkHolder.getDataBlock().getColumnsMinValue());
-      if (bitSet.isEmpty()) {
-        scannedResult.setNumberOfRows(0);
-        scannedResult.setIndexes(new int[0]);
-        CarbonUtil.freeMemory(blocksChunkHolder.getDimensionDataChunk(),
-            blocksChunkHolder.getMeasureDataChunk());
-        return;
-      }
-    }
-    // apply filter on actual data
-    BitSet bitSet = this.filterExecuter.applyFilter(blocksChunkHolder);
-    // if indexes is empty then return with empty result
-    if (bitSet.isEmpty()) {
-      scannedResult.setNumberOfRows(0);
-      scannedResult.setIndexes(new int[0]);
-      return;
-    }
-    // valid scanned blocklet
-    QueryStatistic validScannedBlockletStatistic = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
-    validScannedBlockletStatistic
-        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
-            validScannedBlockletStatistic.getCount() + 1);
-    
queryStatisticsModel.getRecorder().recordStatistics(validScannedBlockletStatistic);
-    // get the row indexes from bot set
-    int[] indexes = new int[bitSet.cardinality()];
-    int index = 0;
-    for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-      indexes[index++] = i;
-    }
-    // loading delete data cache in blockexecutioninfo instance
-    DeleteDeltaCacheLoaderIntf deleteCacheLoader =
-        new BlockletDeleteDeltaCacheLoader(scannedResult.getBlockletId(),
-            blocksChunkHolder.getDataBlock(), 
blockExecutionInfo.getAbsoluteTableIdentifier());
-    deleteCacheLoader.loadDeleteDeltaFileDataToCache();
-    scannedResult
-        
.setBlockletDeleteDeltaCache(blocksChunkHolder.getDataBlock().getDeleteDeltaDataCache());
-    FileHolder fileReader = blocksChunkHolder.getFileReader();
-    int[][] allSelectedDimensionBlocksIndexes =
-        blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
-    DimensionColumnDataChunk[] projectionListDimensionChunk = 
blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(fileReader, allSelectedDimensionBlocksIndexes);
-
-    DimensionColumnDataChunk[] dimensionColumnDataChunk =
-        new 
DimensionColumnDataChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
-    // read dimension chunk blocks from file which is not present
-    for (int i = 0; i < dimensionColumnDataChunk.length; i++) {
-      if (null != blocksChunkHolder.getDimensionDataChunk()[i]) {
-        dimensionColumnDataChunk[i] = 
blocksChunkHolder.getDimensionDataChunk()[i];
-      }
-    }
-    for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
-      System.arraycopy(projectionListDimensionChunk, 
allSelectedDimensionBlocksIndexes[i][0],
-          dimensionColumnDataChunk, allSelectedDimensionBlocksIndexes[i][0],
-          allSelectedDimensionBlocksIndexes[i][1] + 1 - 
allSelectedDimensionBlocksIndexes[i][0]);
-    }
-    MeasureColumnDataChunk[] measureColumnDataChunk =
-        new 
MeasureColumnDataChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
-    int[][] allSelectedMeasureBlocksIndexes =
-        blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
-    MeasureColumnDataChunk[] projectionListMeasureChunk = 
blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(fileReader, allSelectedMeasureBlocksIndexes);
-    // read the measure chunk blocks which is not present
-    for (int i = 0; i < measureColumnDataChunk.length; i++) {
-      if (null != blocksChunkHolder.getMeasureDataChunk()[i]) {
-        measureColumnDataChunk[i] = blocksChunkHolder.getMeasureDataChunk()[i];
-      }
-    }
-    for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
-      System.arraycopy(projectionListMeasureChunk, 
allSelectedMeasureBlocksIndexes[i][0],
-          measureColumnDataChunk, allSelectedMeasureBlocksIndexes[i][0],
-          allSelectedMeasureBlocksIndexes[i][1] + 1 - 
allSelectedMeasureBlocksIndexes[i][0]);
-    }
-    scannedResult.setDimensionChunks(dimensionColumnDataChunk);
-    scannedResult.setIndexes(indexes);
-    scannedResult.setMeasureChunks(measureColumnDataChunk);
-    scannedResult.setNumberOfRows(indexes.length);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/scanner/impl/NonFilterScanner.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/scanner/impl/NonFilterScanner.java
 
b/core/src/main/java/org/apache/carbondata/scan/scanner/impl/NonFilterScanner.java
deleted file mode 100644
index 0cea193..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/scanner/impl/NonFilterScanner.java
+++ /dev/null
@@ -1,40 +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.scan.scanner.impl;
-
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.scan.result.impl.NonFilterQueryScannedResult;
-import org.apache.carbondata.scan.scanner.AbstractBlockletScanner;
-
-/**
- * Non filter processor which will be used for non filter query
- * In case of non filter query we just need to read all the blocks requested 
in the
- * query and pass it to scanned result
- */
-public class NonFilterScanner extends AbstractBlockletScanner {
-
-  public NonFilterScanner(BlockExecutionInfo blockExecutionInfo,
-                          QueryStatisticsModel queryStatisticsModel) {
-    super(blockExecutionInfo);
-    // as its a non filter query creating a non filter query scanned result 
object
-    scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-    super.queryStatisticsModel = queryStatisticsModel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/main/java/org/apache/carbondata/scan/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/scan/wrappers/ByteArrayWrapper.java 
b/core/src/main/java/org/apache/carbondata/scan/wrappers/ByteArrayWrapper.java
deleted file mode 100644
index 1805636..0000000
--- 
a/core/src/main/java/org/apache/carbondata/scan/wrappers/ByteArrayWrapper.java
+++ /dev/null
@@ -1,221 +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.scan.wrappers;
-
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-
-/**
- * This class will store the dimension column data when query is executed
- * This can be used as a key for aggregation
- */
-public class ByteArrayWrapper implements Comparable<ByteArrayWrapper> {
-
-  /**
-   * to store key which is generated using
-   * key generator
-   */
-  protected byte[] dictionaryKey;
-
-  /**
-   * to store no dictionary column data
-   */
-  protected byte[][] complexTypesKeys;
-
-  /**
-   * to store no dictionary column data
-   */
-  protected byte[][] noDictionaryKeys;
-
-  /**
-   * contains value of implicit columns in byte array format
-   */
-  protected byte[] implicitColumnByteArray;
-
-  public ByteArrayWrapper() {
-  }
-
-  /**
-   * @return the dictionaryKey
-   */
-  public byte[] getDictionaryKey() {
-    return dictionaryKey;
-  }
-
-  /**
-   * @param dictionaryKey the dictionaryKey to set
-   */
-  public void setDictionaryKey(byte[] dictionaryKey) {
-    this.dictionaryKey = dictionaryKey;
-  }
-
-  /**
-   * @param noDictionaryKeys the noDictionaryKeys to set
-   */
-  public void setNoDictionaryKeys(byte[][] noDictionaryKeys) {
-    this.noDictionaryKeys = noDictionaryKeys;
-  }
-
-  /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getNoDictionaryKeyByIndex(int index) {
-    return this.noDictionaryKeys[index];
-  }
-
-  /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getComplexTypeByIndex(int index) {
-    return this.complexTypesKeys[index];
-  }
-
-  /**
-   * to generate the hash code
-   */
-  @Override public int hashCode() {
-    // first generate the has code of the dictionary column
-    int len = dictionaryKey.length;
-    int result = 1;
-    for (int j = 0; j < len; j++) {
-      result = 31 * result + dictionaryKey[j];
-    }
-    // then no dictionary column
-    for (byte[] directSurrogateValue : noDictionaryKeys) {
-      for (int i = 0; i < directSurrogateValue.length; i++) {
-        result = 31 * result + directSurrogateValue[i];
-      }
-    }
-    // then for complex type
-    for (byte[] complexTypeKey : complexTypesKeys) {
-      for (int i = 0; i < complexTypeKey.length; i++) {
-        result = 31 * result + complexTypeKey[i];
-      }
-    }
-    return result;
-  }
-
-  /**
-   * to validate the two
-   *
-   * @param other object
-   */
-  @Override public boolean equals(Object other) {
-    if (null == other || !(other instanceof ByteArrayWrapper)) {
-      return false;
-    }
-    boolean result = false;
-    // Comparison will be as follows
-    // first compare the no dictionary column
-    // if it is not equal then return false
-    // if it is equal then compare the complex column
-    // if it is also equal then compare dictionary column
-    byte[][] noDictionaryKeysOther = ((ByteArrayWrapper) 
other).noDictionaryKeys;
-    if (noDictionaryKeysOther.length != noDictionaryKeys.length) {
-      return false;
-    } else {
-      for (int i = 0; i < noDictionaryKeys.length; i++) {
-        result = UnsafeComparer.INSTANCE.equals(noDictionaryKeys[i], 
noDictionaryKeysOther[i]);
-        if (!result) {
-          return false;
-        }
-      }
-    }
-
-    byte[][] complexTypesKeysOther = ((ByteArrayWrapper) 
other).complexTypesKeys;
-    if (complexTypesKeysOther.length != complexTypesKeys.length) {
-      return false;
-    } else {
-      for (int i = 0; i < complexTypesKeys.length; i++) {
-        result = UnsafeComparer.INSTANCE.equals(complexTypesKeys[i], 
complexTypesKeysOther[i]);
-        if (!result) {
-          return false;
-        }
-      }
-    }
-
-    return UnsafeComparer.INSTANCE.equals(dictionaryKey, ((ByteArrayWrapper) 
other).dictionaryKey);
-  }
-
-  /**
-   * Compare method for ByteArrayWrapper class this will used to compare Two
-   * ByteArrayWrapper data object, basically it will compare two byte array
-   *
-   * @param other ArrayWrapper Object
-   */
-  @Override public int compareTo(ByteArrayWrapper other) {
-    // compare will be as follows
-    //compare dictionary column
-    // then no dictionary column
-    // then complex type column data
-    int compareTo = UnsafeComparer.INSTANCE.compareTo(dictionaryKey, 
other.dictionaryKey);
-    if (compareTo == 0) {
-      for (int i = 0; i < noDictionaryKeys.length; i++) {
-        compareTo =
-            UnsafeComparer.INSTANCE.compareTo(noDictionaryKeys[i], 
other.noDictionaryKeys[i]);
-        if (compareTo != 0) {
-          return compareTo;
-        }
-      }
-    }
-    if (compareTo == 0) {
-      for (int i = 0; i < complexTypesKeys.length; i++) {
-        compareTo =
-            UnsafeComparer.INSTANCE.compareTo(complexTypesKeys[i], 
other.complexTypesKeys[i]);
-        if (compareTo != 0) {
-          return compareTo;
-        }
-      }
-    }
-    return compareTo;
-  }
-
-  /**
-   * @return the complexTypesKeys
-   */
-  public byte[][] getComplexTypesKeys() {
-    return complexTypesKeys;
-  }
-
-  /**
-   * @param complexTypesKeys the complexTypesKeys to set
-   */
-  public void setComplexTypesKeys(byte[][] complexTypesKeys) {
-    this.complexTypesKeys = complexTypesKeys;
-  }
-
-  /**
-   * @return
-   */
-  public byte[] getImplicitColumnByteArray() {
-    return implicitColumnByteArray;
-  }
-
-  /**
-   * @param implicitColumnByteArray
-   */
-  public void setImplicitColumnByteArray(byte[] implicitColumnByteArray) {
-    this.implicitColumnByteArray = implicitColumnByteArray;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/AbsoluteTableIdentifierTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/AbsoluteTableIdentifierTest.java
 
b/core/src/test/java/org/apache/carbondata/core/AbsoluteTableIdentifierTest.java
new file mode 100644
index 0000000..aaf0d97
--- /dev/null
+++ 
b/core/src/test/java/org/apache/carbondata/core/AbsoluteTableIdentifierTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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;
+
+import org.apache.carbondata.core.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.CarbonTableIdentifier;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.TestCase.assertEquals;
+
+public class AbsoluteTableIdentifierTest {
+  static AbsoluteTableIdentifier absoluteTableIdentifier;
+  static AbsoluteTableIdentifier absoluteTableIdentifier1;
+  static AbsoluteTableIdentifier absoluteTableIdentifier2;
+  static AbsoluteTableIdentifier absoluteTableIdentifier3;
+  static AbsoluteTableIdentifier absoluteTableIdentifier4;
+
+  @BeforeClass public static void setup() {
+    absoluteTableIdentifier = new AbsoluteTableIdentifier("storePath",
+        new CarbonTableIdentifier("databaseName", "tableName", "tableId"));
+    absoluteTableIdentifier1 = new AbsoluteTableIdentifier("dummy", null);
+    absoluteTableIdentifier2 = new AbsoluteTableIdentifier("dumgfhmy", null);
+    absoluteTableIdentifier3 =
+        new AbsoluteTableIdentifier("duhgmmy", new 
CarbonTableIdentifier("dummy", "dumy", "dmy"));
+    absoluteTableIdentifier4 = new AbsoluteTableIdentifier("storePath",
+        new CarbonTableIdentifier("databaseName", "tableName", "tableId"));
+  }
+
+  @Test public void equalsTestWithSameInstance() {
+    Boolean res = absoluteTableIdentifier.equals("wrong data");
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithNullObject() {
+    Boolean res = absoluteTableIdentifier.equals(null);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithotherObject() {
+    Boolean res = absoluteTableIdentifier1.equals(absoluteTableIdentifier);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithSameObj() {
+    Boolean res = absoluteTableIdentifier.equals(absoluteTableIdentifier);
+    assert (res);
+  }
+
+  @Test public void equalsTestWithNullColumnIdentifier() {
+    Boolean res = absoluteTableIdentifier1.equals(absoluteTableIdentifier2);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithEqualColumnIdentifier() {
+    Boolean res = absoluteTableIdentifier3.equals(absoluteTableIdentifier4);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithEqualAbsoluteTableIdentifier() {
+    Boolean res = absoluteTableIdentifier.equals(absoluteTableIdentifier4);
+    assert (res);
+  }
+
+  @Test public void hashCodeTest() {
+    int res = absoluteTableIdentifier4.hashCode();
+    int expectedResult = 804398706;
+    assertEquals(res, expectedResult);
+  }
+
+  @Test public void gettablePathTest() {
+    String res = absoluteTableIdentifier4.getTablePath();
+    assert (res.equals("storePath/databaseName/tableName"));
+  }
+
+  @Test public void fromTablePathTest() {
+    AbsoluteTableIdentifier absoluteTableIdentifierTest =
+        
AbsoluteTableIdentifier.fromTablePath("storePath/databaseName/tableName");
+    assert (absoluteTableIdentifierTest.getStorePath()
+        .equals(absoluteTableIdentifier4.getStorePath()));
+  }
+
+  @Test(expected = IllegalArgumentException.class) public void 
fromTablePathWithExceptionTest() {
+    AbsoluteTableIdentifier absoluteTableIdentifierTest =
+        AbsoluteTableIdentifier.fromTablePath("storePath/databaseName");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/CarbonTableIdentifierTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/CarbonTableIdentifierTest.java 
b/core/src/test/java/org/apache/carbondata/core/CarbonTableIdentifierTest.java
new file mode 100644
index 0000000..942291c
--- /dev/null
+++ 
b/core/src/test/java/org/apache/carbondata/core/CarbonTableIdentifierTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core;
+
+import org.apache.carbondata.core.CarbonTableIdentifier;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class CarbonTableIdentifierTest {
+
+  static CarbonTableIdentifier carbonTableIdentifier;
+  static CarbonTableIdentifier carbonTableIdentifier2;
+
+  @BeforeClass public static void setup() {
+    carbonTableIdentifier = new CarbonTableIdentifier("DatabseName", 
"tableName", "tableId");
+
+  }
+
+  @Test public void equalsTestWithSameObject() {
+    Boolean res = carbonTableIdentifier.equals(carbonTableIdentifier);
+    assert (res);
+  }
+
+  @Test public void equalsTestWithSimilarObject() {
+    CarbonTableIdentifier carbonTableIdentifierTest =
+        new CarbonTableIdentifier("DatabseName", "tableName", "tableId");
+    Boolean res = carbonTableIdentifier.equals(carbonTableIdentifierTest);
+    assert (res);
+  }
+
+  @Test public void equalsTestWithNullrObject() {
+    Boolean res = carbonTableIdentifier.equals(carbonTableIdentifier2);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithStringrObject() {
+    Boolean res = carbonTableIdentifier.equals("different class object");
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithoutDatabaseName() {
+    CarbonTableIdentifier carbonTableIdentifierTest =
+        new CarbonTableIdentifier(null, "tableName", "tableId");
+    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithoutTableId() {
+    CarbonTableIdentifier carbonTableIdentifierTest =
+        new CarbonTableIdentifier("DatabseName", "tableName", null);
+    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithDifferentTableId() {
+    CarbonTableIdentifier carbonTableIdentifierTest =
+        new CarbonTableIdentifier("DatabseName", "tableName", "diffTableId");
+    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithNullTableName() {
+    CarbonTableIdentifier carbonTableIdentifierTest =
+        new CarbonTableIdentifier("DatabseName", null, "tableId");
+    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
+    assert (!res);
+  }
+
+  @Test public void equalsTestWithDifferentTableName() {
+    CarbonTableIdentifier carbonTableIdentifierTest =
+        new CarbonTableIdentifier("DatabseName", "diffTableName", "tableId");
+    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
+    assert (!res);
+  }
+
+  @Test public void toStringTest() {
+    String res = carbonTableIdentifier.toString();
+    System.out.printf("sfdsdf " + res);
+    assert (res.equals("DatabseName_tableName"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/ColumnIdentifierTest.java 
b/core/src/test/java/org/apache/carbondata/core/ColumnIdentifierTest.java
new file mode 100644
index 0000000..f4e7094
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/ColumnIdentifierTest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core;
+
+import org.apache.carbondata.core.ColumnIdentifier;
+import org.apache.carbondata.core.metadata.DataType;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static junit.framework.TestCase.assertEquals;
+
+public class ColumnIdentifierTest {
+
+  static ColumnIdentifier columnIdentifier;
+  static Map<String, String> columnProperties;
+
+  @BeforeClass public static void setup() {
+    columnProperties = new HashMap<String, String>();
+    columnProperties.put("key", "value");
+    columnIdentifier = new ColumnIdentifier("columnId", columnProperties, 
DataType.INT);
+  }
+
+  @Test public void hashCodeTest() {
+    int res = columnIdentifier.hashCode();
+    int expectedResult = -623419600;
+    assertEquals(res, expectedResult);
+  }
+
+  @Test public void equalsTestwithSameObject() {
+    Boolean res = columnIdentifier.equals(columnIdentifier);
+    assert (res);
+  }
+
+  @Test public void equalsTestwithSimilarObject() {
+    ColumnIdentifier columnIdentifierTest =
+        new ColumnIdentifier("columnId", columnProperties, DataType.INT);
+    Boolean res = columnIdentifier.equals(columnIdentifierTest);
+    assert (res);
+  }
+
+  @Test public void equalsTestwithNullObject() {
+    Boolean res = columnIdentifier.equals(null);
+    assert (!res);
+  }
+
+  @Test public void equalsTestwithStringObject() {
+    Boolean res = columnIdentifier.equals("String Object");
+    assert (!res);
+  }
+
+  @Test public void equalsTestwithNullColumnId() {
+    ColumnIdentifier columnIdentifierTest =
+        new ColumnIdentifier(null, columnProperties, DataType.INT);
+    Boolean res = columnIdentifierTest.equals(columnIdentifier);
+    assert (!res);
+  }
+
+  @Test public void equalsTestwithDiffColumnId() {
+    ColumnIdentifier columnIdentifierTest =
+        new ColumnIdentifier("diffColumnId", columnProperties, DataType.INT);
+    Boolean res = columnIdentifierTest.equals(columnIdentifier);
+    assert (!res);
+  }
+
+  @Test public void toStringTest() {
+    String res = columnIdentifier.toString();
+    assert (res.equals("ColumnIdentifier [columnId=columnId]"));
+  }
+
+  @Test public void getColumnPropertyTest() {
+    ColumnIdentifier columnIdentifierTest =
+        new ColumnIdentifier("diffColumnId", null, DataType.INT);
+    String res = columnIdentifierTest.getColumnProperty("key");
+    assertEquals(res, null);
+  }
+
+  @Test public void getColumnPropertyTestwithNull() {
+    assert (columnIdentifier.getColumnProperty("key").equals("value"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
 
b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
index 904ddcb..4fd4d37 100644
--- 
a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
+++ 
b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
@@ -37,8 +37,8 @@ import org.apache.carbondata.core.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.DataType;
 import org.apache.carbondata.core.path.CarbonStorePath;
 import org.apache.carbondata.core.path.CarbonTablePath;
-import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
+import org.apache.carbondata.core.datastorage.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastorage.impl.FileFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
 
b/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
deleted file mode 100644
index ef31d06..0000000
--- 
a/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
+++ /dev/null
@@ -1,105 +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;
-
-import org.apache.carbondata.core.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.CarbonTableIdentifier;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class AbsoluteTableIdentifierTest {
-  static AbsoluteTableIdentifier absoluteTableIdentifier;
-  static AbsoluteTableIdentifier absoluteTableIdentifier1;
-  static AbsoluteTableIdentifier absoluteTableIdentifier2;
-  static AbsoluteTableIdentifier absoluteTableIdentifier3;
-  static AbsoluteTableIdentifier absoluteTableIdentifier4;
-
-  @BeforeClass public static void setup() {
-    absoluteTableIdentifier = new AbsoluteTableIdentifier("storePath",
-        new CarbonTableIdentifier("databaseName", "tableName", "tableId"));
-    absoluteTableIdentifier1 = new AbsoluteTableIdentifier("dummy", null);
-    absoluteTableIdentifier2 = new AbsoluteTableIdentifier("dumgfhmy", null);
-    absoluteTableIdentifier3 =
-        new AbsoluteTableIdentifier("duhgmmy", new 
CarbonTableIdentifier("dummy", "dumy", "dmy"));
-    absoluteTableIdentifier4 = new AbsoluteTableIdentifier("storePath",
-        new CarbonTableIdentifier("databaseName", "tableName", "tableId"));
-  }
-
-  @Test public void equalsTestWithSameInstance() {
-    Boolean res = absoluteTableIdentifier.equals("wrong data");
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithNullObject() {
-    Boolean res = absoluteTableIdentifier.equals(null);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithotherObject() {
-    Boolean res = absoluteTableIdentifier1.equals(absoluteTableIdentifier);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithSameObj() {
-    Boolean res = absoluteTableIdentifier.equals(absoluteTableIdentifier);
-    assert (res);
-  }
-
-  @Test public void equalsTestWithNullColumnIdentifier() {
-    Boolean res = absoluteTableIdentifier1.equals(absoluteTableIdentifier2);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithEqualColumnIdentifier() {
-    Boolean res = absoluteTableIdentifier3.equals(absoluteTableIdentifier4);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithEqualAbsoluteTableIdentifier() {
-    Boolean res = absoluteTableIdentifier.equals(absoluteTableIdentifier4);
-    assert (res);
-  }
-
-  @Test public void hashCodeTest() {
-    int res = absoluteTableIdentifier4.hashCode();
-    int expectedResult = 804398706;
-    assertEquals(res, expectedResult);
-  }
-
-  @Test public void gettablePathTest() {
-    String res = absoluteTableIdentifier4.getTablePath();
-    assert (res.equals("storePath/databaseName/tableName"));
-  }
-
-  @Test public void fromTablePathTest() {
-    AbsoluteTableIdentifier absoluteTableIdentifierTest =
-        
AbsoluteTableIdentifier.fromTablePath("storePath/databaseName/tableName");
-    assert (absoluteTableIdentifierTest.getStorePath()
-        .equals(absoluteTableIdentifier4.getStorePath()));
-  }
-
-  @Test(expected = IllegalArgumentException.class) public void 
fromTablePathWithExceptionTest() {
-    AbsoluteTableIdentifier absoluteTableIdentifierTest =
-        AbsoluteTableIdentifier.fromTablePath("storePath/databaseName");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/carbon/CarbonTableIdentifierTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/carbon/CarbonTableIdentifierTest.java
 
b/core/src/test/java/org/apache/carbondata/core/carbon/CarbonTableIdentifierTest.java
deleted file mode 100644
index 1156d35..0000000
--- 
a/core/src/test/java/org/apache/carbondata/core/carbon/CarbonTableIdentifierTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon;
-
-import org.apache.carbondata.core.CarbonTableIdentifier;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class CarbonTableIdentifierTest {
-
-  static CarbonTableIdentifier carbonTableIdentifier;
-  static CarbonTableIdentifier carbonTableIdentifier2;
-
-  @BeforeClass public static void setup() {
-    carbonTableIdentifier = new CarbonTableIdentifier("DatabseName", 
"tableName", "tableId");
-
-  }
-
-  @Test public void equalsTestWithSameObject() {
-    Boolean res = carbonTableIdentifier.equals(carbonTableIdentifier);
-    assert (res);
-  }
-
-  @Test public void equalsTestWithSimilarObject() {
-    CarbonTableIdentifier carbonTableIdentifierTest =
-        new CarbonTableIdentifier("DatabseName", "tableName", "tableId");
-    Boolean res = carbonTableIdentifier.equals(carbonTableIdentifierTest);
-    assert (res);
-  }
-
-  @Test public void equalsTestWithNullrObject() {
-    Boolean res = carbonTableIdentifier.equals(carbonTableIdentifier2);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithStringrObject() {
-    Boolean res = carbonTableIdentifier.equals("different class object");
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithoutDatabaseName() {
-    CarbonTableIdentifier carbonTableIdentifierTest =
-        new CarbonTableIdentifier(null, "tableName", "tableId");
-    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithoutTableId() {
-    CarbonTableIdentifier carbonTableIdentifierTest =
-        new CarbonTableIdentifier("DatabseName", "tableName", null);
-    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithDifferentTableId() {
-    CarbonTableIdentifier carbonTableIdentifierTest =
-        new CarbonTableIdentifier("DatabseName", "tableName", "diffTableId");
-    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithNullTableName() {
-    CarbonTableIdentifier carbonTableIdentifierTest =
-        new CarbonTableIdentifier("DatabseName", null, "tableId");
-    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
-    assert (!res);
-  }
-
-  @Test public void equalsTestWithDifferentTableName() {
-    CarbonTableIdentifier carbonTableIdentifierTest =
-        new CarbonTableIdentifier("DatabseName", "diffTableName", "tableId");
-    Boolean res = carbonTableIdentifierTest.equals(carbonTableIdentifier);
-    assert (!res);
-  }
-
-  @Test public void toStringTest() {
-    String res = carbonTableIdentifier.toString();
-    System.out.printf("sfdsdf " + res);
-    assert (res.equals("DatabseName_tableName"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
 
b/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
deleted file mode 100644
index 4a5fcfd..0000000
--- 
a/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
+++ /dev/null
@@ -1,100 +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;
-
-import org.apache.carbondata.core.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.DataType;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class ColumnIdentifierTest {
-
-  static ColumnIdentifier columnIdentifier;
-  static Map<String, String> columnProperties;
-
-  @BeforeClass public static void setup() {
-    columnProperties = new HashMap<String, String>();
-    columnProperties.put("key", "value");
-    columnIdentifier = new ColumnIdentifier("columnId", columnProperties, 
DataType.INT);
-  }
-
-  @Test public void hashCodeTest() {
-    int res = columnIdentifier.hashCode();
-    int expectedResult = -623419600;
-    assertEquals(res, expectedResult);
-  }
-
-  @Test public void equalsTestwithSameObject() {
-    Boolean res = columnIdentifier.equals(columnIdentifier);
-    assert (res);
-  }
-
-  @Test public void equalsTestwithSimilarObject() {
-    ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier("columnId", columnProperties, DataType.INT);
-    Boolean res = columnIdentifier.equals(columnIdentifierTest);
-    assert (res);
-  }
-
-  @Test public void equalsTestwithNullObject() {
-    Boolean res = columnIdentifier.equals(null);
-    assert (!res);
-  }
-
-  @Test public void equalsTestwithStringObject() {
-    Boolean res = columnIdentifier.equals("String Object");
-    assert (!res);
-  }
-
-  @Test public void equalsTestwithNullColumnId() {
-    ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier(null, columnProperties, DataType.INT);
-    Boolean res = columnIdentifierTest.equals(columnIdentifier);
-    assert (!res);
-  }
-
-  @Test public void equalsTestwithDiffColumnId() {
-    ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier("diffColumnId", columnProperties, DataType.INT);
-    Boolean res = columnIdentifierTest.equals(columnIdentifier);
-    assert (!res);
-  }
-
-  @Test public void toStringTest() {
-    String res = columnIdentifier.toString();
-    assert (res.equals("ColumnIdentifier [columnId=columnId]"));
-  }
-
-  @Test public void getColumnPropertyTest() {
-    ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier("diffColumnId", null, DataType.INT);
-    String res = columnIdentifierTest.getColumnProperty("key");
-    assertEquals(res, null);
-  }
-
-  @Test public void getColumnPropertyTestwithNull() {
-    assert (columnIdentifier.getColumnProperty("key").equals("value"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/952cf517/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 0c3cb84..0000000
--- 
a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/CompressedDataMeasureWrapperTest.java
+++ /dev/null
@@ -1,47 +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.datastorage.store.dataholder.CarbonReadDataHolder;
-import 
org.apache.carbondata.core.datastorage.store.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/incubator-carbondata/blob/952cf517/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
 
b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
deleted file mode 100644
index 3b3ab9f..0000000
--- 
a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
+++ /dev/null
@@ -1,153 +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 java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.carbondata.core.datastorage.store.impl.DFSFileHolderImpl;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.pentaho.di.core.util.Assert.assertNull;
-
-public class DFSFileHolderImplUnitTest {
-
-  private static DFSFileHolderImpl dfsFileHolder;
-  private static String fileName;
-  private static String fileNameWithEmptyContent;
-  private static File file;
-  private static File fileWithEmptyContent;
-
-  @BeforeClass public static void setup() {
-    dfsFileHolder = new DFSFileHolderImpl();
-    file = new File("Test.carbondata");
-    fileWithEmptyContent = new File("TestEXception.carbondata");
-
-    if (!file.exists()) try {
-      file.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    if (!fileWithEmptyContent.exists()) try {
-      fileWithEmptyContent.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    try {
-      FileOutputStream of = new FileOutputStream(file, true);
-      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, 
"UTF-8"));
-      br.write("Hello World");
-      br.close();
-    } catch (Exception e) {
-      e.getMessage();
-    }
-    fileName = file.getAbsolutePath();
-    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
-  }
-
-  @AfterClass public static void tearDown() throws IOException  {
-    file.delete();
-    fileWithEmptyContent.delete();
-    dfsFileHolder.finish();
-  }
-
-  @Test public void testReadByteArray() throws IOException  {
-    byte[] result = dfsFileHolder.readByteArray(fileName, 1);
-    byte[] expected_result = new byte[] { 72 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadByteArrayWithFilePath() throws IOException  {
-    byte[] result = dfsFileHolder.readByteArray(fileName, 2L, 2);
-    byte[] expected_result = { 108, 108 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadLong() throws IOException  {
-    long actualResult = dfsFileHolder.readLong(fileName, 1L);
-    long expectedResult = 7308335519855243122L;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadLongForIoException() throws IOException {
-    dfsFileHolder.readLong(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadIntForIoException() throws IOException{
-    dfsFileHolder.readInt(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadInt() throws IOException  {
-    int actualResult = dfsFileHolder.readInt(fileName, 1L);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileName() throws IOException {
-    int actualResult = dfsFileHolder.readInt(fileName);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadIntWithFileNameForIOException() throws IOException {
-    dfsFileHolder.readInt(fileNameWithEmptyContent);
-  }
-
-  @Test public void testDouble() throws IOException  {
-    double actualResult = dfsFileHolder.readDouble(fileName, 1L);
-    double expectedResult = 7.3083355198552433E18;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testDoubleForIoException() throws IOException {
-    dfsFileHolder.readDouble(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test
-  public void testDoubleForIoExceptionwithUpdateCache() throws IOException {
-    new MockUp<FileSystem>() {
-      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path 
file)
-          throws IOException {
-        throw new IOException();
-      }
-
-    };
-    dfsFileHolder.readDouble(fileName, 1L);
-  }
-
-}

Reply via email to