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 2018/03/08 16:55:04 UTC

[03/54] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index 94a041a..b74c279 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -378,7 +378,6 @@ public abstract class AbstractDataFileFooterConverter {
       cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
     }
     info.setColumnCardinality(cardinality);
-    info.setNumberOfColumns(segmentInfo.getNum_cols());
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/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 52305bd..0cc783e 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
@@ -48,10 +48,10 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
@@ -82,7 +82,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
@@ -248,16 +248,13 @@ public final class CarbonUtil {
   public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
     int[] columnSplit = new int[columnGroups.length];
     int noOfColumnStore = columnSplit.length;
-    boolean[] columnarStore = new boolean[noOfColumnStore];
 
     for (int i = 0; i < columnGroups.length; i++) {
       columnSplit[i] = columnGroups[i].length;
-      columnarStore[i] = columnGroups[i].length <= 1;
     }
     ColumnGroupModel colGroupModel = new ColumnGroupModel();
     colGroupModel.setNoOfColumnStore(noOfColumnStore);
     colGroupModel.setColumnSplit(columnSplit);
-    colGroupModel.setColumnarStore(columnarStore);
     colGroupModel.setColumnGroup(columnGroups);
     return colGroupModel;
   }
@@ -418,7 +415,7 @@ public final class CarbonUtil {
     }
   }
 
-  public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk dimColumnDataChunk,
+  public static int getFirstIndexUsingBinarySearch(DimensionColumnPage dimColumnDataChunk,
       int low, int high, byte[] compareValue, boolean matchUpLimit) {
     int cmpResult = 0;
     while (high >= low) {
@@ -457,7 +454,7 @@ public final class CarbonUtil {
    * @return the compareValue's range index in the dimColumnDataChunk
    */
   public static int[] getRangeIndexUsingBinarySearch(
-      DimensionColumnDataChunk dimColumnDataChunk, int low, int high, byte[] compareValue) {
+      DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] compareValue) {
 
     int[] rangeIndex = new int[2];
     int cmpResult = 0;
@@ -551,7 +548,7 @@ public final class CarbonUtil {
    * @return index value
    */
   public static int nextLesserValueToTarget(int currentIndex,
-      DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue) {
+      DimensionColumnPage dimColumnDataChunk, byte[] compareValue) {
     while (currentIndex - 1 >= 0
         && dimColumnDataChunk.compareTo(currentIndex - 1, compareValue) >= 0) {
       --currentIndex;
@@ -571,7 +568,7 @@ public final class CarbonUtil {
    * @return index value
    */
   public static int nextGreaterValueToTarget(int currentIndex,
-      DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
+      DimensionColumnPage dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
     while (currentIndex + 1 < numerOfRows
         && dimColumnDataChunk.compareTo(currentIndex + 1, compareValue) <= 0) {
       ++currentIndex;
@@ -934,7 +931,7 @@ public final class CarbonUtil {
     return false;
   }
 
-  public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getDictionaryEncodingArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -943,7 +940,7 @@ public final class CarbonUtil {
     return dictionaryEncodingArray;
   }
 
-  public static boolean[] getDirectDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getDirectDictionaryEncodingArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -952,7 +949,7 @@ public final class CarbonUtil {
     return dictionaryEncodingArray;
   }
 
-  public static boolean[] getImplicitColumnArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getImplicitColumnArray(ProjectionDimension[] queryDimensions) {
     boolean[] implicitColumnArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       implicitColumnArray[i] = queryDimensions[i].getDimension().hasEncoding(Encoding.IMPLICIT);
@@ -960,7 +957,7 @@ public final class CarbonUtil {
     return implicitColumnArray;
   }
 
-  public static boolean[] getComplexDataTypeArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getComplexDataTypeArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -1008,7 +1005,6 @@ public final class CarbonUtil {
       fileFooter.setColumnInTable(schema);
       SegmentInfo segmentInfo = new SegmentInfo();
       segmentInfo.setColumnCardinality(detailInfo.getDimLens());
-      segmentInfo.setNumberOfColumns(detailInfo.getRowCount());
       fileFooter.setSegmentInfo(segmentInfo);
       return fileFooter;
     }
@@ -1048,7 +1044,7 @@ public final class CarbonUtil {
    * @return
    */
   public static long calculateMetaSize(TableBlockInfo tableBlockInfo) throws IOException {
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
@@ -2183,19 +2179,19 @@ public final class CarbonUtil {
   /**
    * Below method will be used to check filter value is present in the data chunk or not
    * @param filterValues
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param low
    * @param high
    * @param chunkRowIndex
    * @return
    */
   public static int isFilterPresent(byte[][] filterValues,
-      DimensionColumnDataChunk dimensionColumnDataChunk, int low, int high, int chunkRowIndex) {
+      DimensionColumnPage dimensionColumnPage, int low, int high, int chunkRowIndex) {
     int compareResult = 0;
     int mid = 0;
     while (low <= high) {
       mid = (low + high) >>> 1;
-      compareResult = dimensionColumnDataChunk.compareTo(chunkRowIndex, filterValues[mid]);
+      compareResult = dimensionColumnPage.compareTo(chunkRowIndex, filterValues[mid]);
       if (compareResult < 0) {
         high = mid - 1;
       } else if (compareResult > 0) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
index e61b477..d665379 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
@@ -45,7 +45,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
       throws IOException {
     DataFileFooter dataFileFooter = new DataFileFooter();
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
@@ -123,7 +123,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   }
 
   @Override public List<ColumnSchema> getSchema(TableBlockInfo tableBlockInfo) throws IOException {
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
index 74fd09a..afacc0b 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
@@ -36,17 +36,6 @@ public interface CarbonDictionaryWriter extends Closeable {
   void write(String value) throws IOException;
 
   /**
-   * write method that accepts one value at a time
-   * This method can be used when data is huge and memory is les. In that
-   * case data can be stored to a file and an iterator can iterate over it and
-   * pass one value at a time
-   *
-   * @param value unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  void write(byte[] value) throws IOException;
-
-  /**
    * write method that accepts list of byte arrays as value
    * This can be used when data is less, then string can be converted
    * to byte array for each value and added to a list

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 82baccc..53411e9 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -144,7 +144,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * @param value unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  @Override public void write(byte[] value) throws IOException {
+  private void write(byte[] value) throws IOException {
     if (isFirstTime) {
       init();
       isFirstTime = false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
index 8bf8a07..08dd791 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
@@ -87,7 +87,7 @@ public class DictionaryCacheLoaderImplTest {
         return 9999;
       }
     };
-    dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true);
+    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
     assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 4);
   }
 
@@ -97,7 +97,7 @@ public class DictionaryCacheLoaderImplTest {
         return 10000;
       }
     };
-    dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true);
+    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
     assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 2);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/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 5c51c87..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
+++ /dev/null
@@ -1,149 +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.datastore.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.assertThat;
-
-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);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
new file mode 100644
index 0000000..da61a94
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.datastore.impl.DFSFileReaderImpl;
+
+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.assertThat;
+
+public class DFSFileReaderImplUnitTest {
+
+  private static DFSFileReaderImpl dfsFileHolder;
+  private static String fileName;
+  private static String fileNameWithEmptyContent;
+  private static File file;
+  private static File fileWithEmptyContent;
+
+  @BeforeClass public static void setup() {
+    dfsFileHolder = new DFSFileReaderImpl();
+    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);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
deleted file mode 100644
index ed50d63..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.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.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.datastore.impl.FileHolderImpl;
-
-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.assertNull;
-import static org.junit.Assert.assertThat;
-
-public class FileHolderImplUnitTest {
-
-  private static FileHolderImpl fileHolder;
-  private static FileHolderImpl fileHolderWithCapacity;
-  private static String fileName;
-  private static String fileNameWithEmptyContent;
-  private static File file;
-  private static File fileWithEmptyContent;
-
-  @BeforeClass public static void setup() {
-    fileHolder = new FileHolderImpl();
-    fileHolderWithCapacity = new FileHolderImpl(50);
-    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();
-    fileHolder.finish();
-  }
-
-  @Test public void testReadByteArray() throws IOException  {
-    byte[] result = fileHolder.readByteArray(fileName, 1);
-    byte[] expected_result = new byte[] { 72 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadByteArrayWithFilePath() throws IOException  {
-    byte[] result = fileHolder.readByteArray(fileName, 2L, 2);
-    byte[] expected_result = { 108, 108 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadLong() throws IOException  {
-    long actualResult = fileHolder.readLong(fileName, 1L);
-    long expectedResult = 7308335519855243122L;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadLongForIoException() throws IOException {
-    fileHolder.readLong(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadIntForIoException() throws IOException {
-    fileHolder.readInt(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadInt() throws IOException  {
-    int actualResult = fileHolder.readInt(fileName, 1L);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileName() throws IOException  {
-    int actualResult = fileHolder.readInt(fileName);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileNameForIOException() throws IOException  {
-    fileHolder.readInt(fileNameWithEmptyContent);
-
-  }
-
-  @Test public void testDouble() throws IOException  {
-    double actualResult = fileHolder.readDouble(fileName, 1L);
-    double expectedResult = 7.3083355198552433E18;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testDoubleForIoException() throws IOException {
-    fileHolder.readDouble(fileNameWithEmptyContent, 1L);
-
-  }
-
-  @Test public void testDoubleForIoExceptionwithUpdateCache() throws Exception {
-    new MockUp<FileSystem>() {
-      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
-          throws IOException {
-        throw new IOException();
-      }
-
-    };
-    try {
-      fileHolder.readDouble(fileName, 1L);
-    } catch (Exception e) {
-      assertNull(e.getMessage());
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
new file mode 100644
index 0000000..a6d3235
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.datastore.impl.FileReaderImpl;
+
+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.assertNull;
+import static org.junit.Assert.assertThat;
+
+public class FileReaderImplUnitTest {
+
+  private static FileReaderImpl fileHolder;
+  private static FileReaderImpl fileHolderWithCapacity;
+  private static String fileName;
+  private static String fileNameWithEmptyContent;
+  private static File file;
+  private static File fileWithEmptyContent;
+
+  @BeforeClass public static void setup() {
+    fileHolder = new FileReaderImpl();
+    fileHolderWithCapacity = new FileReaderImpl(50);
+    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();
+    fileHolder.finish();
+  }
+
+  @Test public void testReadByteArray() throws IOException  {
+    byte[] result = fileHolder.readByteArray(fileName, 1);
+    byte[] expected_result = new byte[] { 72 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadByteArrayWithFilePath() throws IOException  {
+    byte[] result = fileHolder.readByteArray(fileName, 2L, 2);
+    byte[] expected_result = { 108, 108 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadLong() throws IOException  {
+    long actualResult = fileHolder.readLong(fileName, 1L);
+    long expectedResult = 7308335519855243122L;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadLongForIoException() throws IOException {
+    fileHolder.readLong(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadIntForIoException() throws IOException {
+    fileHolder.readInt(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadInt() throws IOException  {
+    int actualResult = fileHolder.readInt(fileName, 1L);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileName() throws IOException  {
+    int actualResult = fileHolder.readInt(fileName);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileNameForIOException() throws IOException  {
+    fileHolder.readInt(fileNameWithEmptyContent);
+
+  }
+
+  @Test public void testDouble() throws IOException  {
+    double actualResult = fileHolder.readDouble(fileName, 1L);
+    double expectedResult = 7.3083355198552433E18;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testDoubleForIoException() throws IOException {
+    fileHolder.readDouble(fileNameWithEmptyContent, 1L);
+
+  }
+
+  @Test public void testDoubleForIoExceptionwithUpdateCache() throws Exception {
+    new MockUp<FileSystem>() {
+      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
+          throws IOException {
+        throw new IOException();
+      }
+
+    };
+    try {
+      fileHolder.readDouble(fileName, 1L);
+    } catch (Exception e) {
+      assertNull(e.getMessage());
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
index 9c43553..20036ec 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
@@ -84,7 +84,7 @@ public class SegmentPropertiesTest extends TestCase {
     assertTrue(true);
   }
 
-  @Test public void testBlockMetadataHasProperDimensionBlockMapping() {
+  @Test public void testBlockMetadataHasProperDimensionChunkMapping() {
     Map<Integer, Integer> dimensionOrdinalToBlockMapping = new HashMap<Integer, Integer>();
     dimensionOrdinalToBlockMapping.put(0, 0);
     dimensionOrdinalToBlockMapping.put(1, 1);
@@ -97,7 +97,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimensionOrdinalToBlockMapping.put(8, 5);
     dimensionOrdinalToBlockMapping.put(9, 6);
     Map<Integer, Integer> dimensionOrdinalToBlockMappingActual =
-        blockMetadataInfos.getDimensionOrdinalToBlockMapping();
+        blockMetadataInfos.getDimensionOrdinalToChunkMapping();
     assertEquals(dimensionOrdinalToBlockMapping.size(),
         dimensionOrdinalToBlockMappingActual.size());
     Iterator<Entry<Integer, Integer>> iterator =
@@ -112,12 +112,12 @@ public class SegmentPropertiesTest extends TestCase {
     assertTrue(true);
   }
 
-  @Test public void testBlockMetadataHasProperMeasureBlockMapping() {
+  @Test public void testBlockMetadataHasProperMeasureChunkMapping() {
     Map<Integer, Integer> measureOrdinalToBlockMapping = new HashMap<Integer, Integer>();
     measureOrdinalToBlockMapping.put(0, 0);
     measureOrdinalToBlockMapping.put(1, 1);
     Map<Integer, Integer> measureOrdinalToBlockMappingActual =
-        blockMetadataInfos.getMeasuresOrdinalToBlockMapping();
+        blockMetadataInfos.getMeasuresOrdinalToChunkMapping();
     assertEquals(measureOrdinalToBlockMapping.size(), measureOrdinalToBlockMappingActual.size());
     Iterator<Entry<Integer, Integer>> iterator = measureOrdinalToBlockMapping.entrySet().iterator();
     while (iterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
index bdb83cd..54b66a6 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
@@ -37,7 +37,7 @@ import org.junit.Test;
 
 public class ColumnGroupDimensionDataChunkTest {
 
-  static ColumnGroupDimensionDataChunk columnGroupDimensionDataChunk;
+  static ColumnGroupDimensionColumnPage columnGroupDimensionDataChunk;
   static KeyGenerator keyGenerator;
 
   @BeforeClass public static void setup() {
@@ -56,7 +56,7 @@ public class ColumnGroupDimensionDataChunkTest {
       position += keyGenerator.getKeySizeInBytes();
     }
     columnGroupDimensionDataChunk =
-        new ColumnGroupDimensionDataChunk(data, keyGenerator.getKeySizeInBytes(), 3);
+        new ColumnGroupDimensionColumnPage(data, keyGenerator.getKeySizeInBytes(), 3);
   }
 
   @Test public void fillChunkDataTest() {
@@ -64,7 +64,7 @@ public class ColumnGroupDimensionDataChunkTest {
     ordinals.add(1);
     KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator);
     byte[] buffer = new byte[1];
-    columnGroupDimensionDataChunk.fillChunkData(buffer, 0, 1, keyStructureInfo);
+    columnGroupDimensionDataChunk.fillRawData(1, 0, buffer, keyStructureInfo);
     assertEquals(buffer[0], 2);
   }
 
@@ -81,7 +81,7 @@ public class ColumnGroupDimensionDataChunkTest {
     ordinals.add(2);
     KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator);
     keyStructureInfo.setMdkeyQueryDimensionOrdinal(new int[] { 2 });
-    int res = columnGroupDimensionDataChunk.fillConvertedChunkData(2, 2, row, keyStructureInfo);
+    int res = columnGroupDimensionDataChunk.fillSurrogateKey(2, 2, row, keyStructureInfo);
     Assert.assertTrue(Arrays.equals(row, expected));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
index 04b7a80..f327ef6 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
@@ -30,7 +30,7 @@ import org.junit.Test;
 
 public class FixedLengthDimensionDataChunkTest {
 
-  static FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+  static FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
   static byte[] data;
 
   @BeforeClass public static void setup() {
@@ -40,7 +40,7 @@ public class FixedLengthDimensionDataChunkTest {
 
     int invertedIndexReverse[] = { 1, 0, 5, 7, 8 };
     fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(data, invertedIndex, invertedIndexReverse, 5, 4);
+        new FixedLengthDimensionColumnPage(data, invertedIndex, invertedIndexReverse, 5, 4);
   }
 
   @Test public void fillChunkDataTest() {
@@ -48,7 +48,7 @@ public class FixedLengthDimensionDataChunkTest {
     int[] maskByteRanges = { 1, 2, 4, 6, 5 };
     keyStructureInfo.setMaskByteRanges(maskByteRanges);
     keyStructureInfo.setMaxKey("1234567".getBytes());
-    int res = fixedLengthDimensionDataChunk.fillChunkData(data, 0, 0, keyStructureInfo);
+    int res = fixedLengthDimensionDataChunk.fillRawData(0, 0, data, keyStructureInfo);
     int expectedResult = 4 ;
     assertEquals(res, expectedResult);
   }
@@ -62,7 +62,7 @@ public class FixedLengthDimensionDataChunkTest {
   @Test public void fillConvertedChunkDataTest() {
     int[] row = { 1, 2, 4, 6 };
     KeyStructureInfo keyStructureInfo = new KeyStructureInfo();
-    int res = fixedLengthDimensionDataChunk.fillConvertedChunkData(1, 0, row, keyStructureInfo);
+    int res = fixedLengthDimensionDataChunk.fillSurrogateKey(1, 0, row, keyStructureInfo);
     int expectedResult = 1;
     assertEquals(res, expectedResult);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
index d874037..7f7719c 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
@@ -76,9 +76,9 @@ public class BTreeBlockFinderTest extends TestCase {
     buffer.array();
     IndexKey key = new IndexKey(null, buffer.array());
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(1, findFirstBlock.nodeNumber());
+    assertEquals(1, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(1, findLastBlock.nodeNumber());
+    assertEquals(1, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithNoDictionary() {
@@ -96,9 +96,9 @@ public class BTreeBlockFinderTest extends TestCase {
     buffer.array();
     IndexKey key = new IndexKey(null, buffer.array());
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey1()
@@ -118,9 +118,9 @@ public class BTreeBlockFinderTest extends TestCase {
     IndexKey key =
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 1, 1 }), null);
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey2()
@@ -141,9 +141,9 @@ public class BTreeBlockFinderTest extends TestCase {
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 0, 0 }), null);
 
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   /**
@@ -169,9 +169,9 @@ public class BTreeBlockFinderTest extends TestCase {
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 10001, 10001 }), null);
 
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(99, findFirstBlock.nodeNumber());
+    assertEquals(99, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(99, findLastBlock.nodeNumber());
+    assertEquals(99, findLastBlock.nodeIndex());
   }
 
   private List<DataFileFooter> getDataFileFooterList() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/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 85bdfd8..be91410 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
@@ -50,32 +50,32 @@ public class RawBasedResultCollectorTest {
 //    aggregatorInfo.setDefaultValues(new Object[] { 1, 2, 3, 4 });
 //    aggregatorInfo.setMeasureDataTypes(
 //        new DataType[] { DataTypes.INT, DataTypes.TIMESTAMP, DataTypes.INT, DataTypes.INT });
-//    QueryMeasure queryMeasure1 = new QueryMeasure("QMCol1");
+//    ProjectionMeasure queryMeasure1 = new ProjectionMeasure("QMCol1");
 //    queryMeasure1.setQueryOrder(1);
-//    QueryMeasure queryMeasure2 = new QueryMeasure("QMCol2");
+//    ProjectionMeasure queryMeasure2 = new ProjectionMeasure("QMCol2");
 //    queryMeasure1.setQueryOrder(2);
-//    QueryMeasure queryMeasure3 = new QueryMeasure("QMCol3");
+//    ProjectionMeasure queryMeasure3 = new ProjectionMeasure("QMCol3");
 //    queryMeasure1.setQueryOrder(3);
-//    QueryMeasure queryMeasure4 = new QueryMeasure("QMCol4");
+//    ProjectionMeasure queryMeasure4 = new ProjectionMeasure("QMCol4");
 //    queryMeasure1.setQueryOrder(4);
-//    QueryDimension queryDimension1 = new QueryDimension("QDCol1");
+//    ProjectionDimension queryDimension1 = new ProjectionDimension("QDCol1");
 //    queryDimension1.setQueryOrder(1);
 //    ColumnSchema columnSchema = new ColumnSchema();
 //    queryDimension1.setDimension(new CarbonDimension(columnSchema, 0, 0, 0, 0));
-//    QueryDimension queryDimension2 = new QueryDimension("QDCol2");
+//    ProjectionDimension queryDimension2 = new ProjectionDimension("QDCol2");
 //    queryDimension2.setQueryOrder(2);
 //    queryDimension2.setDimension(new CarbonDimension(columnSchema, 1, 1, 1, 1));
-//    QueryDimension queryDimension3 = new QueryDimension("QDCol3");
+//    ProjectionDimension queryDimension3 = new ProjectionDimension("QDCol3");
 //    queryDimension3.setQueryOrder(3);
 //    queryDimension3.setDimension(new CarbonDimension(columnSchema, 2, 0, 0, 0));
-//    QueryDimension queryDimension4 = new QueryDimension("QDCol4");
+//    ProjectionDimension queryDimension4 = new ProjectionDimension("QDCol4");
 //    queryDimension4.setQueryOrder(4);
 //    queryDimension4.setDimension(new CarbonDimension(columnSchema, 3, 0, 0, 0));
-//    blockExecutionInfo.setQueryDimensions(
-//        new QueryDimension[] { queryDimension1, queryDimension2, queryDimension3,
+//    blockExecutionInfo.setProjectionDimensions(
+//        new ProjectionDimension[] { queryDimension1, queryDimension2, queryDimension3,
 //            queryDimension4 });
-//    blockExecutionInfo.setQueryMeasures(
-//        new QueryMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
+//    blockExecutionInfo.setProjectionMeasures(
+//        new ProjectionMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
 //    blockExecutionInfo.setFixedKeyUpdateRequired(true);
 //    blockExecutionInfo.setMeasureInfo(aggregatorInfo);
 //    blockExecutionInfo.setMaskedByteForBlock(new int[] { 1, 2 });
@@ -99,7 +99,7 @@ public class RawBasedResultCollectorTest {
 //        return new byte[][] { { 1, 2 }, { 1, 2 } };
 //      }
 //
-//      @SuppressWarnings("unused") @Mock public ColumnPage getMeasureChunk(int ordinal) {
+//      @SuppressWarnings("unused") @Mock public ColumnPage readMeasureChunk(int ordinal) {
 //        ColumnPage ColumnPage = new ColumnPage();
 //        PresenceMeta presenceMeta = new PresenceMeta();
 //        BitSet bitSet = new BitSet();
@@ -120,10 +120,10 @@ public class RawBasedResultCollectorTest {
 //      }
 //    };
 //
-//    AbstractScannedResult abstractScannedResult =
+//    BlockletScannedResult abstractScannedResult =
 //        new NonFilterQueryScannedResult(blockExecutionInfo);
-//    abstractScannedResult.setNumberOfRows(2);
-//    List<Object[]> result = rawBasedResultCollector.collectData(abstractScannedResult, 2);
+//    abstractScannedResult.setPageFilteredRowCount(2);
+//    List<Object[]> result = rawBasedResultCollector.collectResultInRow(abstractScannedResult, 2);
 //    int expectedResult = 2;
 //    assertThat(result.size(), is(equalTo(expectedResult)));
 //  }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
index 45e86f2..e8b75b7 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesTestUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import junit.framework.TestCase;
 import mockit.Mock;
@@ -45,9 +45,8 @@ public class QueryUtilTest extends TestCase {
 
   @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() {
 
-    QueryDimension dimension =
-        new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
+    ProjectionDimension dimension =
+        new ProjectionDimension(segmentProperties.getDimensions().get(0));
     int[] maskedByteRange = QueryUtil
         .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
     int[] expectedMaskedByteRange = { 0 };
@@ -57,11 +56,10 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       dimensions.add(dimension);
     }
     int[] maskedByteRange =
@@ -84,11 +82,10 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaxKeyBasedOnDimensions() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       dimensions.add(dimension);
     }
     byte[] maxKeyBasedOnDimensions = null;
@@ -115,10 +112,8 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaksedByte() {
-    QueryDimension dimension =
-        new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
+    ProjectionDimension dimension =
+        new ProjectionDimension(segmentProperties.getDimensions().get(0));
     int[] maskedByteRange = QueryUtil
         .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
     int[] maskedByte = QueryUtil
@@ -203,18 +198,16 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetSortDimensionIndexes() {
-    List<QueryDimension> sortedDimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> sortedDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       sortedDimensions.add(dimension);
     }
-    List<QueryDimension> queryDimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> queryDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       queryDimensions.add(dimension);
     }
     byte[] actualValue = QueryUtil.getSortDimensionIndexes(sortedDimensions, queryDimensions);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
index 90fcb74..163580d 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
@@ -28,8 +28,8 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.MeasureInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 
 import org.junit.Test;
 
@@ -79,27 +79,22 @@ public class RestructureUtilTest {
     List<CarbonDimension> tableComplexDimensions =
         Arrays.asList(tableComplexDimension1, tableComplexDimension2);
 
-    QueryDimension queryDimension1 = new QueryDimension("Id");
-    queryDimension1.setDimension(tableBlockDimension1);
-    QueryDimension queryDimension2 = new QueryDimension("Name");
-    queryDimension2.setDimension(tableComplexDimension2);
-    QueryDimension queryDimension3 = new QueryDimension("Address");
-    queryDimension3.setDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3));
-    QueryMeasure queryMeasure1 = new QueryMeasure("Age");
-    QueryMeasure queryMeasure2 = new QueryMeasure("Salary");
-    queryMeasure1.setMeasure(new CarbonMeasure(columnSchema3, 2));
-    queryMeasure2.setMeasure(new CarbonMeasure(columnSchema4, 4));
-    List<QueryMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2);
+    ProjectionDimension queryDimension1 = new ProjectionDimension(tableBlockDimension1);
+    ProjectionDimension queryDimension2 = new ProjectionDimension(tableComplexDimension2);
+    ProjectionDimension queryDimension3 = new ProjectionDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3));
+    ProjectionMeasure queryMeasure1 = new ProjectionMeasure(new CarbonMeasure(columnSchema3, 2));
+    ProjectionMeasure queryMeasure2 = new ProjectionMeasure(new CarbonMeasure(columnSchema4, 4));
+    List<ProjectionMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2);
 
-    List<QueryDimension> queryDimensions =
+    List<ProjectionDimension> queryDimensions =
         Arrays.asList(queryDimension1, queryDimension2, queryDimension3);
 
-    List<QueryDimension> result = null;
+    List<ProjectionDimension> result = null;
     result = RestructureUtil
         .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo, queryDimensions,
             tableBlockDimensions, tableComplexDimensions, queryMeasures.size());
     List<CarbonDimension> resultDimension = new ArrayList<>(result.size());
-    for (QueryDimension queryDimension : result) {
+    for (ProjectionDimension queryDimension : result) {
       resultDimension.add(queryDimension.getDimension());
     }
     assertThat(resultDimension,
@@ -126,13 +121,10 @@ public class RestructureUtilTest {
     carbonMeasure3.getColumnSchema().setDefaultValue("3".getBytes());
     List<CarbonMeasure> currentBlockMeasures = Arrays.asList(carbonMeasure1, carbonMeasure2);
 
-    QueryMeasure queryMeasure1 = new QueryMeasure("Id");
-    queryMeasure1.setMeasure(carbonMeasure1);
-    QueryMeasure queryMeasure2 = new QueryMeasure("Name");
-    queryMeasure2.setMeasure(carbonMeasure2);
-    QueryMeasure queryMeasure3 = new QueryMeasure("Age");
-    queryMeasure3.setMeasure(carbonMeasure3);
-    List<QueryMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3);
+    ProjectionMeasure queryMeasure1 = new ProjectionMeasure(carbonMeasure1);
+    ProjectionMeasure queryMeasure2 = new ProjectionMeasure(carbonMeasure2);
+    ProjectionMeasure queryMeasure3 = new ProjectionMeasure(carbonMeasure3);
+    List<ProjectionMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3);
     BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
     RestructureUtil.createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo, queryMeasures,
         currentBlockMeasures);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 565da04..9c7c26c 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -328,7 +328,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
         return "test";
       }
     };
-    assertTrue(FilterUtil.getFilterListForRS(expression, columnExpression, defaultValues,
+    assertTrue(FilterUtil.getFilterListForRS(expression, defaultValues,
         defaultSurrogate) instanceof ColumnFilterInfo);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
index e3ae42c..11c4980 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
@@ -18,12 +18,12 @@ package org.apache.carbondata.core.scan.filter.executer;
 
 import java.util.BitSet;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest {
 
- @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimColumnDataChunk,
+ @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimColumnDataChunk,
      int numerOfRows, byte[][] filterValues) {
    BitSet bitSet = new BitSet(numerOfRows);
    bitSet.flip(0, numerOfRows);
@@ -46,7 +46,7 @@ public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest
    return bitSet;
  }
 
- @Override public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimColumnDataChunk,
+ @Override public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
index 29dda52..1bfa875 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
@@ -18,8 +18,8 @@ package org.apache.carbondata.core.scan.filter.executer;
 
 import java.util.BitSet;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.junit.Assert;
@@ -38,22 +38,22 @@ public class IncludeFilterExecuterImplTest extends TestCase {
 
   }
 
-  public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimensionColumnDataChunk,
+  public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
       // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.set(i);
           }
         }
       } else if (filterValues.length == 1) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (dimensionColumnDataChunk.compareTo(i, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(i, filterValues[0]) == 0) {
             bitSet.set(i);
           }
         }
@@ -62,14 +62,14 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     return bitSet;
   }
 
-  public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+  public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
       // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       for (int k = 0; k < filterValues.length; k++) {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[k]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[k]) == 0) {
             bitSet.set(j);
           }
         }
@@ -163,7 +163,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     System.out.println("dimColumnSize: " + dimColumnSize);
     
-    FixedLengthDimensionDataChunk dimensionColumnDataChunk;
+    FixedLengthDimensionColumnPage dimensionColumnDataChunk;
     DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo();
 
     byte[] dataChunk = new byte[dataChunkSize * dimColumnSize];
@@ -183,7 +183,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     dim.setFilterKeys(filterKeys);
 
-    dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunkSize, dimColumnSize);
 
     // repeat query and compare 2 result between old code and new optimized code
@@ -194,7 +194,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
       oldTime = oldTime + System.currentTimeMillis() - start;
 
       start = System.currentTimeMillis();
-      BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionDataChunk) dimensionColumnDataChunk, dataChunkSize,
+      BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionColumnPage) dimensionColumnDataChunk, dataChunkSize,
           filterKeys);
       newTime = newTime + System.currentTimeMillis() - start;
 
@@ -212,7 +212,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
   }
 
 
-  private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionColumnPage dimensionColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -243,7 +243,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionColumnPage dimensionColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     int startIndex = 0;
@@ -281,7 +281,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     int filteredValueCnt = 800;
     // column dictionary size
     int dimColumnSize = 2;
-    FixedLengthDimensionDataChunk dimensionColumnDataChunk;
+    FixedLengthDimensionColumnPage dimensionColumnDataChunk;
     DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo();
 
     byte[] dataChunk = new byte[dataChunkSize * dimColumnSize];
@@ -303,7 +303,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     dim.setFilterKeys(filterKeys);
 
-    dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / dimColumnSize, dimColumnSize);
 
     // initial to run

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
deleted file mode 100644
index 541ce48..0000000
--- a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
+++ /dev/null
@@ -1,94 +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;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-public class BatchResultTest {
-  private static BatchResult batchResult;
-  private static List<Object[]> rowsList = new ArrayList(2);
-
-  @BeforeClass public static void setUp() {
-    batchResult = new BatchResult();
-    rowsList.add(0, new Integer[] { 1, 2 });
-    rowsList.add(1, new Integer[] { 3 });
-  }
-
-  @Test public void testNext() throws NoSuchElementException {
-    BatchResult rows = new BatchResult();
-    rows.setRows(rowsList);
-    Object[] result = rows.next();
-    Assert.assertTrue(result.equals(rowsList.get(0)));
-  }
-
-  @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() {
-    BatchResult rows = new BatchResult();
-    List emptyList = new ArrayList(2);
-    rows.setRows(emptyList);
-    rows.next();
-  }
-
-  @Test public void testGetRows() {
-    new MockUp<BatchResult>() {
-      @Mock public void $init() {
-        //to be left blank
-      }
-    };
-    BatchResult batchResult = new BatchResult();
-    List<Object[]> list = batchResult.getRows();
-    assertNull("Number of rows is null", list);
-  }
-
-  @Test public void testHasNext() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    list.add(1, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    boolean result = batchResult.hasNext();
-    Assert.assertTrue(result);
-  }
-
-  @Test public void testGetRawRow() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    Object[] actualValue = batchResult.getRawRow(0);
-    Assert.assertTrue(list.get(0) == actualValue);
-  }
-
-  @Test public void testGetSize() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    list.add(1, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    int actualValue = batchResult.getSize();
-    int expectedValue = 2;
-    assertEquals(expectedValue, actualValue);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/daa64650/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
new file mode 100644
index 0000000..123d64e
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class RowBatchTest {
+  private static RowBatch rowBatch;
+  private static List<Object[]> rowsList = new ArrayList(2);
+
+  @BeforeClass public static void setUp() {
+    rowBatch = new RowBatch();
+    rowsList.add(0, new Integer[] { 1, 2 });
+    rowsList.add(1, new Integer[] { 3 });
+  }
+
+  @Test public void testNext() throws NoSuchElementException {
+    RowBatch rows = new RowBatch();
+    rows.setRows(rowsList);
+    Object[] result = rows.next();
+    Assert.assertTrue(result.equals(rowsList.get(0)));
+  }
+
+  @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() {
+    RowBatch rows = new RowBatch();
+    List emptyList = new ArrayList(2);
+    rows.setRows(emptyList);
+    rows.next();
+  }
+
+  @Test public void testGetRows() {
+    new MockUp<RowBatch>() {
+      @Mock public void $init() {
+        //to be left blank
+      }
+    };
+    RowBatch rowBatch = new RowBatch();
+    List<Object[]> list = rowBatch.getRows();
+    assertNull("Number of rows is null", list);
+  }
+
+  @Test public void testHasNext() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    list.add(1, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    boolean result = rowBatch.hasNext();
+    Assert.assertTrue(result);
+  }
+
+  @Test public void testGetRawRow() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    Object[] actualValue = rowBatch.getRawRow(0);
+    Assert.assertTrue(list.get(0) == actualValue);
+  }
+
+  @Test public void testGetSize() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    list.add(1, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    int actualValue = rowBatch.getSize();
+    int expectedValue = 2;
+    assertEquals(expectedValue, actualValue);
+  }
+
+}