You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/08/15 07:09:24 UTC

[39/52] [partial] incubator-carbondata git commit: Renamed packages to org.apache.carbondata and fixed errors

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/data/uncompressed/DoubleArrayDataInMemoryStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/data/uncompressed/DoubleArrayDataInMemoryStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/data/uncompressed/DoubleArrayDataInMemoryStore.java
new file mode 100644
index 0000000..1effb25
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/data/uncompressed/DoubleArrayDataInMemoryStore.java
@@ -0,0 +1,163 @@
+/*
+ * 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.datastorage.store.impl.data.uncompressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.MeasureDataWrapper;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.datastorage.store.impl.CompressedDataMeasureDataWrapper;
+
+/**
+ * DoubleArrayDataInMemoryStore.
+ *
+ * @author S71955
+ */
+public class DoubleArrayDataInMemoryStore extends AbstractDoubleArrayDataStore {
+
+  // /**
+  // * DoubleArrayDataInMemoryStore.
+  // * @param size
+  // * @param elementSize
+  // * @param compressionModel
+  // */
+  // public DoubleArrayDataInMemoryStore(int size, int elementSize,
+  // ValueCompressionModel compressionModel)
+  // {
+  // super(size, elementSize, compressionModel);
+  // }
+  //
+  // /**
+  // * DoubleArrayDataInMemoryStore.
+  // * @param size
+  // * @param elementSize
+  // */
+  // public DoubleArrayDataInMemoryStore(int size, int elementSize)
+  // {
+  // super(size, elementSize);
+  // }
+
+  // /**
+  // * DoubleArrayDataInMemoryStore.
+  // * @param size
+  // * @param elementSize
+  // * @param compressionModel
+  // * @param measuresOffsetsArray
+  // * @param measuresLengthArray
+  // * @param fileName
+  // * @param fileHolder
+  // */
+  // public DoubleArrayDataInMemoryStore(int size, int elementSize,
+  // ValueCompressionModel compressionModel,
+  // long[] measuresOffsetsArray, int[] measuresLengthArray, String fileName,
+  // FileHolder fileHolder)
+  // {
+  // super(size, elementSize, compressionModel);
+  // UnCompressValue[] unCompValues = compressionModel.getUnCompressValues();
+  // if(null != unCompValues)
+  // {
+  // for(int i = 0;i < measuresLengthArray.length;i++)
+  // {
+  //
+  // values[i] = unCompValues[i].getNew();
+  // values[i].setValueInBytes(fileHolder.readByteArray(fileName,
+  // measuresOffsetsArray[i],
+  // measuresLengthArray[i]));
+  // }
+  // }
+  // }
+
+  /**
+   * DoubleArrayDataInMemoryStore.
+   *
+   * @param size
+   * @param elementSize
+   * @param compressionModel
+   * @param measuresOffsetsArray
+   * @param measuresLengthArray
+   * @param fileName
+   * @param fileHolder
+   */
+  public DoubleArrayDataInMemoryStore(ValueCompressionModel compressionModel,
+      long[] measuresOffsetsArray, int[] measuresLengthArray, String fileName,
+      FileHolder fileHolder) {
+    super(compressionModel);
+    if (null != compressionModel) {
+      UnCompressValue[] unCompValues = compressionModel.getUnCompressValues();
+      if (null != unCompValues) {
+        for (int i = 0; i < measuresLengthArray.length; i++) {
+
+          values[i] = unCompValues[i].getNew();
+          values[i].setValueInBytes(
+              fileHolder.readByteArray(fileName, measuresOffsetsArray[i], measuresLengthArray[i]));
+        }
+      }
+    }
+  }
+
+  /**
+   * DoubleArrayDataInMemoryStore.
+   *
+   * @param size
+   * @param elementSize
+   * @param compressionModel
+   * @param measuresOffsetsArray
+   * @param measuresLengthArray
+   * @param fileName
+   * @param fileHolder
+   */
+  public DoubleArrayDataInMemoryStore(ValueCompressionModel compressionModel) {
+    super(compressionModel);
+  }
+
+  @Override public MeasureDataWrapper getBackData(int[] cols, FileHolder fileHolder) {
+    if (null == compressionModel) {
+      return null;
+    }
+    CarbonReadDataHolder[] vals = new CarbonReadDataHolder[values.length];
+    if (null == cols) {
+      for (int i = 0; i < vals.length; i++) {
+        vals[i] = values[i]
+            .getValues(compressionModel.getDecimal()[i], compressionModel.getMaxValue()[i]);
+      }
+    } else {
+      for (int i = 0; i < cols.length; i++) {
+        vals[cols[i]] = values[cols[i]].getValues(compressionModel.getDecimal()[cols[i]],
+            compressionModel.getMaxValue()[cols[i]]);
+      }
+    }
+    // return new CompressedDataMeasureDataWrapper(values,
+    // compressionModel.getDecimal(), compressionModel.getMaxValue());
+    return new CompressedDataMeasureDataWrapper(vals);
+  }
+
+  @Override public MeasureDataWrapper getBackData(int cols, FileHolder fileHolder) {
+    if (null == compressionModel) {
+      return null;
+    }
+    CarbonReadDataHolder[] vals = new CarbonReadDataHolder[values.length];
+
+    vals[cols] = values[cols]
+        .getValues(compressionModel.getDecimal()[cols], compressionModel.getMaxValue()[cols]);
+    return new CompressedDataMeasureDataWrapper(vals);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/AbstractColumnarKeyStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/AbstractColumnarKeyStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/AbstractColumnarKeyStore.java
new file mode 100644
index 0000000..51c5fb8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/AbstractColumnarKeyStore.java
@@ -0,0 +1,106 @@
+/*
+ * 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.datastorage.store.impl.key.columnar;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStore;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression;
+
+public abstract class AbstractColumnarKeyStore implements ColumnarKeyStore {
+
+  /**
+   * compressor will be used to compress the data
+   */
+  protected static final Compressor<byte[]> COMPRESSOR =
+      SnappyCompression.SnappyByteCompression.INSTANCE;
+
+  protected ColumnarKeyStoreInfo columnarStoreInfo;
+
+  protected byte[][] columnarKeyBlockDataIndex;
+
+  protected byte[][] columnarKeyBlockData;
+
+  protected Map<Integer, Integer> mapOfColumnIndexAndColumnBlockIndex;
+
+  protected Map<Integer, Integer> mapOfAggDataIndex;
+
+  protected byte[][] columnarUniqueblockKeyBlockIndex;
+
+  public AbstractColumnarKeyStore(ColumnarKeyStoreInfo columnarStoreInfo, boolean isInMemory,
+      FileHolder fileHolder) {
+    this.columnarStoreInfo = columnarStoreInfo;
+    this.mapOfColumnIndexAndColumnBlockIndex =
+        new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.mapOfAggDataIndex =
+        new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    int index = 0;
+    for (int i = 0; i < this.columnarStoreInfo.getIsSorted().length; i++) {
+      if (!this.columnarStoreInfo.getIsSorted()[i]) {
+        this.mapOfColumnIndexAndColumnBlockIndex.put(i, index++);
+      }
+    }
+    index = 0;
+    for (int i = 0; i < this.columnarStoreInfo.getAggKeyBlock().length; i++) {
+      if (this.columnarStoreInfo.getAggKeyBlock()[i]) {
+        mapOfAggDataIndex.put(i, index++);
+      }
+    }
+    if (isInMemory) {
+      this.columnarKeyBlockData = new byte[this.columnarStoreInfo.getIsSorted().length][];
+      this.columnarKeyBlockDataIndex = new byte[this.mapOfColumnIndexAndColumnBlockIndex.size()][];
+      this.columnarUniqueblockKeyBlockIndex = new byte[this.mapOfAggDataIndex.size()][];
+      for (int i = 0; i < columnarStoreInfo.getSizeOfEachBlock().length; i++) {
+        columnarKeyBlockData[i] = fileHolder.readByteArray(columnarStoreInfo.getFilePath(),
+            columnarStoreInfo.getKeyBlockOffsets()[i], columnarStoreInfo.getKeyBlockLengths()[i]);
+
+        if (!this.columnarStoreInfo.getIsSorted()[i]) {
+          this.columnarKeyBlockDataIndex[mapOfColumnIndexAndColumnBlockIndex.get(i)] = fileHolder
+              .readByteArray(columnarStoreInfo.getFilePath(),
+                  columnarStoreInfo.getKeyBlockIndexOffsets()[mapOfColumnIndexAndColumnBlockIndex
+                      .get(i)],
+                  columnarStoreInfo.getKeyBlockIndexLength()[mapOfColumnIndexAndColumnBlockIndex
+                      .get(i)]);
+        }
+
+        if (this.columnarStoreInfo.getAggKeyBlock()[i]) {
+          this.columnarUniqueblockKeyBlockIndex[mapOfAggDataIndex.get(i)] = fileHolder
+              .readByteArray(columnarStoreInfo.getFilePath(),
+                  columnarStoreInfo.getDataIndexMapOffsets()[mapOfAggDataIndex.get(i)],
+                  columnarStoreInfo.getDataIndexMapLength()[mapOfAggDataIndex.get(i)]);
+        }
+      }
+    }
+  }
+
+  protected int[] getColumnIndexForNonFilter(int[] columnIndex) {
+    int[] columnIndexTemp = new int[columnIndex.length];
+
+    for (int i = 0; i < columnIndex.length; i++) {
+      columnIndexTemp[columnIndex[i]] = i;
+    }
+    return columnIndexTemp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarFileKeyStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarFileKeyStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarFileKeyStore.java
new file mode 100644
index 0000000..94d4b8f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarFileKeyStore.java
@@ -0,0 +1,168 @@
+/*
+ * 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.datastorage.store.impl.key.columnar.compressed;
+
+import java.util.List;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreMetadata;
+import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.datastorage.store.impl.key.columnar.AbstractColumnarKeyStore;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+public class CompressedColumnarFileKeyStore extends AbstractColumnarKeyStore {
+
+  public CompressedColumnarFileKeyStore(ColumnarKeyStoreInfo columnarStoreInfo) {
+    super(columnarStoreInfo, false, null);
+  }
+
+  @Override public ColumnarKeyStoreDataHolder[] getUnCompressedKeyArray(FileHolder fileHolder,
+      int[] blockIndex, boolean[] needCompressedData, int[] noDictionaryColIndexes) {
+    ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder[blockIndex.length];
+
+    for (int i = 0; i < columnarKeyStoreDataHolders.length; i++) {
+      byte[] columnarKeyBlockData = null;
+      int[] columnKeyBlockIndex = null;
+      int[] columnKeyBlockReverseIndexes = null;
+      ColumnarKeyStoreMetadata columnarKeyStoreMetadata = null;
+      int[] dataIndex = null;
+      boolean isUnCompressed = true;
+      columnarKeyBlockData = COMPRESSOR.unCompress(fileHolder
+          .readByteArray(columnarStoreInfo.getFilePath(),
+              columnarStoreInfo.getKeyBlockOffsets()[blockIndex[i]],
+              columnarStoreInfo.getKeyBlockLengths()[blockIndex[i]]));
+      boolean isNoDictionaryBlock =
+          CompressedColumnarKeyStoreUtil.isNoDictionaryBlock(noDictionaryColIndexes, blockIndex[i]);
+      if (!isNoDictionaryBlock && this.columnarStoreInfo.getAggKeyBlock()[blockIndex[i]]) {
+        dataIndex = columnarStoreInfo.getNumberCompressor().unCompress(fileHolder
+            .readByteArray(columnarStoreInfo.getFilePath(),
+                columnarStoreInfo.getDataIndexMapOffsets()[mapOfAggDataIndex.get(blockIndex[i])],
+                columnarStoreInfo.getDataIndexMapLength()[mapOfAggDataIndex.get(blockIndex[i])]));
+        if (!needCompressedData[i]) {
+          columnarKeyBlockData = UnBlockIndexer.uncompressData(columnarKeyBlockData, dataIndex,
+              columnarStoreInfo.getSizeOfEachBlock()[blockIndex[i]]);
+          dataIndex = null;
+        } else {
+          isUnCompressed = false;
+        }
+      }
+      if (!columnarStoreInfo.getIsSorted()[blockIndex[i]]) {
+        columnKeyBlockIndex = CarbonUtil
+            .getUnCompressColumnIndex(columnarStoreInfo.getKeyBlockIndexLength()[blockIndex[i]],
+                fileHolder.readByteArray(columnarStoreInfo.getFilePath(),
+                    columnarStoreInfo.getKeyBlockIndexOffsets()[blockIndex[i]],
+                    columnarStoreInfo.getKeyBlockIndexLength()[blockIndex[i]]),
+                columnarStoreInfo.getNumberCompressor());
+        columnKeyBlockReverseIndexes = getColumnIndexForNonFilter(columnKeyBlockIndex);
+      }
+      //Since its an high cardinality dimension adding the direct surrogates as part of
+      //columnarKeyStoreMetadata so that later it will be used with bytearraywrapper instance.
+      if (isNoDictionaryBlock) {
+        columnarKeyStoreMetadata = new ColumnarKeyStoreMetadata(0);
+        columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+        columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndexes);
+        columnarKeyStoreMetadata.setNoDictionaryValColumn(true);
+        columnarKeyStoreMetadata.setUnCompressed(true);
+        columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex[i]]);
+        //System is reading the direct surrogates data from byte array which contains both
+        // length and the direct surrogates data
+        List<byte[]> noDictionaryValBasedKeyBlockData = CompressedColumnarKeyStoreUtil
+            .readColumnarKeyBlockDataForNoDictionaryCols(columnarKeyBlockData);
+        columnarKeyStoreDataHolders[i] =
+            new ColumnarKeyStoreDataHolder(noDictionaryValBasedKeyBlockData,
+                columnarKeyStoreMetadata);
+      } else {
+        columnarKeyStoreMetadata =
+            new ColumnarKeyStoreMetadata(columnarStoreInfo.getSizeOfEachBlock()[blockIndex[i]]);
+        columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+        columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex[i]]);
+        columnarKeyStoreMetadata.setDataIndex(dataIndex);
+        columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndexes);
+        columnarKeyStoreMetadata.setUnCompressed(isUnCompressed);
+        columnarKeyStoreDataHolders[i] =
+            new ColumnarKeyStoreDataHolder(columnarKeyBlockData, columnarKeyStoreMetadata);
+      }
+    }
+    return columnarKeyStoreDataHolders;
+  }
+
+  @Override
+  public ColumnarKeyStoreDataHolder getUnCompressedKeyArray(FileHolder fileHolder, int blockIndex,
+      boolean needCompressedData, int[] noDictionaryColIndexes) {
+    byte[] columnarKeyBlockData = null;
+    int[] columnKeyBlockIndex = null;
+    int[] columnKeyBlockReverseIndex = null;
+    ColumnarKeyStoreMetadata columnarKeyStoreMetadata = null;
+    int[] dataIndex = null;
+    boolean isUnCompressed = true;
+    columnarKeyBlockData = COMPRESSOR.unCompress(fileHolder
+        .readByteArray(columnarStoreInfo.getFilePath(),
+            columnarStoreInfo.getKeyBlockOffsets()[blockIndex],
+            columnarStoreInfo.getKeyBlockLengths()[blockIndex]));
+    boolean isNoDictionaryBlock =
+        CompressedColumnarKeyStoreUtil.isNoDictionaryBlock(noDictionaryColIndexes, blockIndex);
+    if (!isNoDictionaryBlock && this.columnarStoreInfo.getAggKeyBlock()[blockIndex]) {
+      dataIndex = columnarStoreInfo.getNumberCompressor().unCompress(fileHolder
+          .readByteArray(columnarStoreInfo.getFilePath(),
+              columnarStoreInfo.getDataIndexMapOffsets()[mapOfAggDataIndex.get(blockIndex)],
+              columnarStoreInfo.getDataIndexMapLength()[mapOfAggDataIndex.get(blockIndex)]));
+      if (!needCompressedData) {
+        columnarKeyBlockData = UnBlockIndexer.uncompressData(columnarKeyBlockData, dataIndex,
+            columnarStoreInfo.getSizeOfEachBlock()[blockIndex]);
+        dataIndex = null;
+      } else {
+        isUnCompressed = false;
+      }
+    }
+    if (!columnarStoreInfo.getIsSorted()[blockIndex]) {
+      columnKeyBlockIndex = CarbonUtil
+          .getUnCompressColumnIndex(columnarStoreInfo.getKeyBlockIndexLength()[blockIndex],
+              fileHolder.readByteArray(columnarStoreInfo.getFilePath(),
+                  columnarStoreInfo.getKeyBlockIndexOffsets()[blockIndex],
+                  columnarStoreInfo.getKeyBlockIndexLength()[blockIndex]),
+              columnarStoreInfo.getNumberCompressor());
+      columnKeyBlockReverseIndex = getColumnIndexForNonFilter(columnKeyBlockIndex);
+    }
+    //Since its an high cardinality dimension, For filter queries.
+    if (isNoDictionaryBlock) {
+      columnarKeyStoreMetadata = new ColumnarKeyStoreMetadata(0);
+      ColumnarKeyStoreDataHolder columnarKeyStoreDataHolders = CompressedColumnarKeyStoreUtil
+          .createColumnarKeyStoreMetadataForHCDims(blockIndex, columnarKeyBlockData,
+              columnKeyBlockIndex, columnKeyBlockReverseIndex, columnarStoreInfo);
+      new ColumnarKeyStoreDataHolder(columnarKeyBlockData, columnarKeyStoreMetadata);
+      return columnarKeyStoreDataHolders;
+    }
+    columnarKeyStoreMetadata =
+        new ColumnarKeyStoreMetadata(columnarStoreInfo.getSizeOfEachBlock()[blockIndex]);
+    columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+    columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex]);
+    columnarKeyStoreMetadata.setDataIndex(dataIndex);
+    columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndex);
+    columnarKeyStoreMetadata.setUnCompressed(isUnCompressed);
+
+    ColumnarKeyStoreDataHolder columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder(columnarKeyBlockData, columnarKeyStoreMetadata);
+    return columnarKeyStoreDataHolders;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarInMemoryStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarInMemoryStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarInMemoryStore.java
new file mode 100644
index 0000000..5d3d4b5
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarInMemoryStore.java
@@ -0,0 +1,155 @@
+/*
+ * 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.datastorage.store.impl.key.columnar.compressed;
+
+import java.util.List;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreMetadata;
+import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.datastorage.store.impl.key.columnar.AbstractColumnarKeyStore;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+public class CompressedColumnarInMemoryStore extends AbstractColumnarKeyStore {
+
+  public CompressedColumnarInMemoryStore(ColumnarKeyStoreInfo columnarStoreInfo,
+      FileHolder fileHolder) {
+    super(columnarStoreInfo, true, fileHolder);
+  }
+
+  @Override public ColumnarKeyStoreDataHolder[] getUnCompressedKeyArray(FileHolder fileHolder,
+      int[] blockIndex, boolean[] needCompressedData, int[] noDictionaryColIndexes) {
+    ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder[blockIndex.length];
+    for (int i = 0; i < columnarKeyStoreDataHolders.length; i++) {
+      byte[] columnarKeyBlockDataTemp = null;
+      int[] columnKeyBlockIndex = null;
+      int[] columnKeyBlockReverseIndexes = null;
+      ColumnarKeyStoreMetadata columnarKeyStoreMetadata = null;
+      int columnarKeyBlockIndex = 0;
+      int[] dataIndex = null;
+      boolean isUnCompressed = true;
+      columnarKeyBlockDataTemp = COMPRESSOR.unCompress(columnarKeyBlockData[blockIndex[i]]);
+      boolean isNoDictionaryBlock =
+          CompressedColumnarKeyStoreUtil.isNoDictionaryBlock(noDictionaryColIndexes, blockIndex[i]);
+      if (!isNoDictionaryBlock && this.columnarStoreInfo.getAggKeyBlock()[blockIndex[i]]) {
+        dataIndex = columnarStoreInfo.getNumberCompressor()
+            .unCompress(columnarUniqueblockKeyBlockIndex[mapOfAggDataIndex.get(blockIndex[i])]);
+        if (!needCompressedData[i]) {
+          columnarKeyBlockDataTemp = UnBlockIndexer
+              .uncompressData(columnarKeyBlockDataTemp, dataIndex,
+                  columnarStoreInfo.getSizeOfEachBlock()[blockIndex[i]]);
+          dataIndex = null;
+        } else {
+          isUnCompressed = false;
+        }
+      }
+      if (!columnarStoreInfo.getIsSorted()[blockIndex[i]]) {
+        columnarKeyBlockIndex = mapOfColumnIndexAndColumnBlockIndex.get(blockIndex[i]);
+        columnKeyBlockIndex = CarbonUtil.getUnCompressColumnIndex(
+            columnarStoreInfo.getKeyBlockIndexLength()[columnarKeyBlockIndex],
+            columnarKeyBlockDataIndex[columnarKeyBlockIndex],
+            columnarStoreInfo.getNumberCompressor());
+        columnKeyBlockReverseIndexes = getColumnIndexForNonFilter(columnKeyBlockIndex);
+      }
+      if (isNoDictionaryBlock) {
+        columnarKeyStoreMetadata = new ColumnarKeyStoreMetadata(0);
+        columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+        columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndexes);
+        columnarKeyStoreMetadata.setNoDictionaryValColumn(true);
+        columnarKeyStoreMetadata.setUnCompressed(true);
+        columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex[i]]);
+        //System is reading the direct surrogates data from byte array which contains both
+        // length and the direct surrogates data
+        List<byte[]> noDictionaryValBasedKeyBlockData = CompressedColumnarKeyStoreUtil
+            .readColumnarKeyBlockDataForNoDictionaryCols(columnarKeyBlockDataTemp);
+        columnarKeyStoreDataHolders[i] =
+            new ColumnarKeyStoreDataHolder(noDictionaryValBasedKeyBlockData,
+                columnarKeyStoreMetadata);
+      }
+      columnarKeyStoreMetadata =
+          new ColumnarKeyStoreMetadata(columnarStoreInfo.getSizeOfEachBlock()[blockIndex[i]]);
+      columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+      columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex[i]]);
+      columnarKeyStoreMetadata.setDataIndex(dataIndex);
+      columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndexes);
+      columnarKeyStoreMetadata.setUnCompressed(isUnCompressed);
+      columnarKeyStoreDataHolders[i] =
+          new ColumnarKeyStoreDataHolder(columnarKeyBlockDataTemp, columnarKeyStoreMetadata);
+    }
+    return columnarKeyStoreDataHolders;
+  }
+
+  @Override
+  public ColumnarKeyStoreDataHolder getUnCompressedKeyArray(FileHolder fileHolder, int blockIndex,
+      boolean needCompressedData, int[] noDictionaryVals) {
+
+    byte[] columnarKeyBlockDataTemp = null;
+    int[] columnKeyBlockIndex = null;
+    int[] columnKeyBlockReverseIndex = null;
+    ColumnarKeyStoreMetadata columnarKeyStoreMetadata = null;
+    int columnarKeyBlockIndex = 0;
+    int[] dataIndex = null;
+    boolean isUnCompressed = true;
+    columnarKeyBlockDataTemp = COMPRESSOR.unCompress(columnarKeyBlockData[blockIndex]);
+    boolean isNoDictionaryBlock =
+        CompressedColumnarKeyStoreUtil.isNoDictionaryBlock(noDictionaryVals, blockIndex);
+    if (!isNoDictionaryBlock && this.columnarStoreInfo.getAggKeyBlock()[blockIndex]) {
+      dataIndex = columnarStoreInfo.getNumberCompressor()
+          .unCompress(columnarUniqueblockKeyBlockIndex[mapOfAggDataIndex.get(blockIndex)]);
+      if (!needCompressedData) {
+        columnarKeyBlockDataTemp = UnBlockIndexer
+            .uncompressData(columnarKeyBlockDataTemp, dataIndex,
+                columnarStoreInfo.getSizeOfEachBlock()[blockIndex]);
+        dataIndex = null;
+      } else {
+        isUnCompressed = false;
+      }
+    }
+    if (!columnarStoreInfo.getIsSorted()[blockIndex]) {
+      columnarKeyBlockIndex = mapOfColumnIndexAndColumnBlockIndex.get(blockIndex);
+      columnKeyBlockIndex = CarbonUtil.getUnCompressColumnIndex(
+          columnarStoreInfo.getKeyBlockIndexLength()[columnarKeyBlockIndex],
+          columnarKeyBlockDataIndex[columnarKeyBlockIndex],
+          columnarStoreInfo.getNumberCompressor());
+      columnKeyBlockReverseIndex = getColumnIndexForNonFilter(columnKeyBlockIndex);
+    }
+    if (isNoDictionaryBlock) {
+      ColumnarKeyStoreDataHolder colKeystoreDataHolders = CompressedColumnarKeyStoreUtil
+          .createColumnarKeyStoreMetadataForHCDims(blockIndex, columnarKeyBlockDataTemp,
+              columnKeyBlockIndex, columnKeyBlockReverseIndex, columnarStoreInfo);
+      return colKeystoreDataHolders;
+    }
+    columnarKeyStoreMetadata =
+        new ColumnarKeyStoreMetadata(columnarStoreInfo.getSizeOfEachBlock()[blockIndex]);
+    columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+    columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex]);
+    columnarKeyStoreMetadata.setDataIndex(dataIndex);
+    columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndex);
+    columnarKeyStoreMetadata.setUnCompressed(isUnCompressed);
+    ColumnarKeyStoreDataHolder columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder(columnarKeyBlockDataTemp, columnarKeyStoreMetadata);
+    return columnarKeyStoreDataHolders;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarKeyStoreUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarKeyStoreUtil.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarKeyStoreUtil.java
new file mode 100644
index 0000000..b0d7ff8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/compressed/CompressedColumnarKeyStoreUtil.java
@@ -0,0 +1,108 @@
+/*
+ * 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.datastorage.store.impl.key.columnar.compressed;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreMetadata;
+
+/**
+ * Utility helper class for managing the processing of columnar key store block.
+ */
+public final class CompressedColumnarKeyStoreUtil {
+
+  private CompressedColumnarKeyStoreUtil() {
+
+  }
+
+  /**
+   * @param columnarKeyBlockData
+   * @param columnarKeyStoreMetadata
+   * @return
+   * @author s71955 The high cardinality dimensions rows will be send in byte
+   * array with its data length appended in the
+   * ColumnarKeyStoreDataHolder byte array since high cardinality dim
+   * data will not be part of MDKey/Surrogate keys. In this method the
+   * byte array will be scanned and the length which is stored in
+   * short will be removed.
+   */
+  public static List<byte[]> readColumnarKeyBlockDataForNoDictionaryCols(
+      byte[] columnarKeyBlockData) {
+    List<byte[]> columnarKeyBlockDataList = new ArrayList<byte[]>(50);
+    ByteBuffer noDictionaryValKeyStoreDataHolder = ByteBuffer.allocate(columnarKeyBlockData.length);
+    noDictionaryValKeyStoreDataHolder.put(columnarKeyBlockData);
+    noDictionaryValKeyStoreDataHolder.flip();
+    while (noDictionaryValKeyStoreDataHolder.hasRemaining()) {
+      short dataLength = noDictionaryValKeyStoreDataHolder.getShort();
+      byte[] noDictionaryValKeyData = new byte[dataLength];
+      noDictionaryValKeyStoreDataHolder.get(noDictionaryValKeyData);
+      columnarKeyBlockDataList.add(noDictionaryValKeyData);
+    }
+    return columnarKeyBlockDataList;
+
+  }
+
+  /**
+   * @param blockIndex
+   * @param columnarKeyBlockData
+   * @param columnKeyBlockIndex
+   * @param columnKeyBlockReverseIndex
+   * @param columnarStoreInfo
+   * @return
+   */
+  public static ColumnarKeyStoreDataHolder createColumnarKeyStoreMetadataForHCDims(int blockIndex,
+      byte[] columnarKeyBlockData, int[] columnKeyBlockIndex, int[] columnKeyBlockReverseIndex,
+      ColumnarKeyStoreInfo columnarStoreInfo) {
+    ColumnarKeyStoreMetadata columnarKeyStoreMetadata;
+    columnarKeyStoreMetadata = new ColumnarKeyStoreMetadata(0);
+    columnarKeyStoreMetadata.setNoDictionaryValColumn(true);
+    columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+    columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndex);
+    columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex]);
+    columnarKeyStoreMetadata.setUnCompressed(true);
+    List<byte[]> noDictionaryValBasedKeyBlockData = CompressedColumnarKeyStoreUtil
+        .readColumnarKeyBlockDataForNoDictionaryCols(columnarKeyBlockData);
+    ColumnarKeyStoreDataHolder columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder(noDictionaryValBasedKeyBlockData, columnarKeyStoreMetadata);
+    return columnarKeyStoreDataHolders;
+  }
+
+  /**
+   * This API will determine whether the requested block index is a  No dictionary
+   * column index.
+   *
+   * @param noDictionaryColIndexes
+   * @param blockIndex
+   * @return
+   */
+  public static boolean isNoDictionaryBlock(int[] noDictionaryColIndexes, int blockIndex) {
+    if (null != noDictionaryColIndexes) {
+      for (int noDictionaryValIndex : noDictionaryColIndexes) {
+        if (noDictionaryValIndex == blockIndex) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarFileKeyStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarFileKeyStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarFileKeyStore.java
new file mode 100644
index 0000000..d0b17dc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarFileKeyStore.java
@@ -0,0 +1,88 @@
+/*
+ * 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.datastorage.store.impl.key.columnar.uncompressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreMetadata;
+import org.apache.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.datastorage.store.impl.key.columnar.AbstractColumnarKeyStore;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+public class UnCompressedColumnarFileKeyStore extends AbstractColumnarKeyStore {
+
+  public UnCompressedColumnarFileKeyStore(ColumnarKeyStoreInfo columnarStoreInfo) {
+    super(columnarStoreInfo, false, null);
+  }
+
+  @Override public ColumnarKeyStoreDataHolder[] getUnCompressedKeyArray(FileHolder fileHolder,
+      int[] blockIndex, boolean[] needCompressedData, int[] noDictionaryColIndexes) {
+    ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder[blockIndex.length];
+    byte[] columnarKeyBlockData = null;
+    int[] columnKeyBlockIndex = null;
+    ColumnarKeyStoreMetadata columnarKeyStoreMetadata = null;
+    int columnarKeyBlockIndex = 0;
+    int[] dataIndex = null;
+    int[] columnKeyBlockReverseIndex = null;
+    for (int j = 0; j < columnarKeyStoreDataHolders.length; j++) {
+      columnarKeyBlockData = fileHolder.readByteArray(columnarStoreInfo.getFilePath(),
+          columnarStoreInfo.getKeyBlockOffsets()[blockIndex[j]],
+          columnarStoreInfo.getKeyBlockLengths()[blockIndex[j]]);
+      if (this.columnarStoreInfo.getAggKeyBlock()[blockIndex[j]]) {
+        dataIndex = columnarStoreInfo.getNumberCompressor().unCompress(fileHolder
+            .readByteArray(columnarStoreInfo.getFilePath(),
+                columnarStoreInfo.getDataIndexMapOffsets()[mapOfAggDataIndex.get(blockIndex[j])],
+                columnarStoreInfo.getDataIndexMapLength()[mapOfAggDataIndex.get(blockIndex[j])]));
+        if (!needCompressedData[j]) {
+          columnarKeyBlockData = UnBlockIndexer.uncompressData(columnarKeyBlockData, dataIndex,
+              columnarStoreInfo.getSizeOfEachBlock()[blockIndex[j]]);
+          dataIndex = null;
+        }
+      }
+      if (!columnarStoreInfo.getIsSorted()[blockIndex[j]]) {
+        columnarKeyBlockIndex = mapOfColumnIndexAndColumnBlockIndex.get(blockIndex[j]);
+        columnKeyBlockIndex = CarbonUtil.getUnCompressColumnIndex(
+            columnarStoreInfo.getKeyBlockIndexLength()[columnarKeyBlockIndex], fileHolder
+                .readByteArray(columnarStoreInfo.getFilePath(),
+                    columnarStoreInfo.getKeyBlockIndexOffsets()[columnarKeyBlockIndex],
+                    columnarStoreInfo.getKeyBlockIndexLength()[columnarKeyBlockIndex]),
+            columnarStoreInfo.getNumberCompressor());
+        columnKeyBlockReverseIndex = getColumnIndexForNonFilter(columnKeyBlockIndex);
+      }
+      columnarKeyStoreMetadata =
+          new ColumnarKeyStoreMetadata(columnarStoreInfo.getSizeOfEachBlock()[blockIndex[j]]);
+      columnarKeyStoreMetadata.setSorted(columnarStoreInfo.getIsSorted()[blockIndex[j]]);
+      columnarKeyStoreMetadata.setColumnIndex(columnKeyBlockIndex);
+      columnarKeyStoreMetadata.setDataIndex(dataIndex);
+      columnarKeyStoreMetadata.setColumnReverseIndex(columnKeyBlockReverseIndex);
+      columnarKeyStoreDataHolders[j] =
+          new ColumnarKeyStoreDataHolder(columnarKeyBlockData, columnarKeyStoreMetadata);
+    }
+    return columnarKeyStoreDataHolders;
+  }
+
+  @Override
+  public ColumnarKeyStoreDataHolder getUnCompressedKeyArray(FileHolder fileHolder, int blockIndex,
+      boolean needCompressedData, int[] noDictionaryColIndexes) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarInMemoryStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarInMemoryStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarInMemoryStore.java
new file mode 100644
index 0000000..da69e01
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/columnar/uncompressed/UnCompressedColumnarInMemoryStore.java
@@ -0,0 +1,70 @@
+/*
+ * 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.datastorage.store.impl.key.columnar.uncompressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
+import org.apache.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreMetadata;
+import org.apache.carbondata.core.datastorage.store.impl.key.columnar.AbstractColumnarKeyStore;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+public class UnCompressedColumnarInMemoryStore extends AbstractColumnarKeyStore {
+
+  public UnCompressedColumnarInMemoryStore(ColumnarKeyStoreInfo columnarStoreInfo,
+      FileHolder fileHolder) {
+    super(columnarStoreInfo, true, fileHolder);
+  }
+
+  @Override public ColumnarKeyStoreDataHolder[] getUnCompressedKeyArray(FileHolder fileHolder,
+      int[] blockIndex, boolean[] needCompressedData, int[] noDictionaryColIndexes) {
+    int columnarKeyBlockIndex = 0;
+    int[] columnIndex = null;
+    ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolders =
+        new ColumnarKeyStoreDataHolder[blockIndex.length];
+    ColumnarKeyStoreMetadata columnarKeyStoreMetadataTemp = null;
+    for (int i = 0; i < columnarKeyStoreDataHolders.length; i++) {
+      columnarKeyStoreMetadataTemp = new ColumnarKeyStoreMetadata(0);
+      if (!columnarStoreInfo.getIsSorted()[blockIndex[i]]) {
+        columnarKeyBlockIndex = mapOfColumnIndexAndColumnBlockIndex.get(blockIndex[i]);
+        columnIndex = CarbonUtil.getUnCompressColumnIndex(
+            columnarStoreInfo.getKeyBlockIndexLength()[columnarKeyBlockIndex], fileHolder
+                .readByteArray(columnarStoreInfo.getFilePath(),
+                    columnarStoreInfo.getKeyBlockIndexOffsets()[columnarKeyBlockIndex],
+                    columnarStoreInfo.getKeyBlockIndexLength()[columnarKeyBlockIndex]),
+            columnarStoreInfo.getNumberCompressor());
+        columnIndex = getColumnIndexForNonFilter(columnIndex);
+        columnarKeyStoreMetadataTemp.setColumnIndex(columnIndex);
+      }
+      columnarKeyStoreMetadataTemp.setSorted(columnarStoreInfo.getIsSorted()[blockIndex[i]]);
+      columnarKeyStoreDataHolders[i] =
+          new ColumnarKeyStoreDataHolder(columnarKeyBlockData[blockIndex[i]],
+              columnarKeyStoreMetadataTemp);
+    }
+    return columnarKeyStoreDataHolders;
+  }
+
+  @Override
+  public ColumnarKeyStoreDataHolder getUnCompressedKeyArray(FileHolder fileHolder, int blockIndex,
+      boolean needCompressedData, int[] noDictionaryVals) {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/AbstractCompressedSingleArrayStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/AbstractCompressedSingleArrayStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/AbstractCompressedSingleArrayStore.java
new file mode 100644
index 0000000..493b61f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/AbstractCompressedSingleArrayStore.java
@@ -0,0 +1,119 @@
+/*
+ * 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.datastorage.store.impl.key.compressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.NodeKeyStore;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.SnappyCompression;
+
+public abstract class AbstractCompressedSingleArrayStore implements NodeKeyStore {
+
+  /**
+   * compressor will be used to compress the data
+   */
+  protected static final Compressor<byte[]> COMPRESSOR =
+      SnappyCompression.SnappyByteCompression.INSTANCE;
+  /**
+   * size of each element
+   */
+  protected final int sizeOfEachElement;
+  /**
+   * data store which will hold the data
+   */
+  protected byte[] datastore;
+  /**
+   * total number of elements;
+   */
+  protected int totalNumberOfElements;
+
+  public AbstractCompressedSingleArrayStore(int size, int elementSize) {
+    this(size, elementSize, true);
+  }
+
+  public AbstractCompressedSingleArrayStore(int size, int elementSize, boolean createDataStore) {
+    this.sizeOfEachElement = elementSize;
+    this.totalNumberOfElements = size;
+    if (createDataStore) {
+      datastore = new byte[this.totalNumberOfElements * this.sizeOfEachElement];
+    }
+  }
+
+  /**
+   * This method will be used to insert key to store
+   */
+  @Override public void put(int index, byte[] value) {
+    System.arraycopy(value, 0, datastore, ((index) * sizeOfEachElement), sizeOfEachElement);
+  }
+
+  /**
+   * This method will be used to get the writable key array.
+   * writable key array will hold below information:
+   * <size of key array><key array>
+   * total length will be stored in 4 bytes+ key array length for key array
+   *
+   * @return writable array (compressed or normal)
+   */
+  @Override public byte[] getWritableKeyArray() {
+    // compress the data store
+    byte[] compressedKeys = COMPRESSOR.compress(datastore);
+    return compressedKeys;
+  }
+
+  /**
+   * This method will be used to get the actual key array present in the
+   * store .
+   * Here back array will be uncompress array
+   *
+   * @param fileHolder file holder will be used to read the file
+   * @return uncompressed keys
+   * will return uncompressed key
+   */
+  @Override public byte[] getBackArray(FileHolder fileHolder) {
+    return COMPRESSOR.unCompress(datastore);
+  }
+
+  /**
+   * This method will be used to get the key array based on index
+   *
+   * @param index      index in store
+   * @param fileHolder file holder will be used to read the file
+   * @return key
+   */
+  @Override public byte[] get(int index, FileHolder fileHolder) {
+    // uncompress the store data
+    byte[] unCompress = COMPRESSOR.unCompress(datastore);
+    // create new array of size of each element
+    byte[] copy = new byte[sizeOfEachElement];
+    // copy array for given index
+    // copy will done based on below calculation
+    // eg: index is 4 and size of each key is 6 then copy from 6*4= 24th
+    // index till 29th index
+    System.arraycopy(unCompress, ((index) * sizeOfEachElement), copy, 0, sizeOfEachElement);
+    return copy;
+  }
+
+  /**
+   * This method will clear the store and create the new empty store
+   */
+  @Override public void clear() {
+    datastore = new byte[this.totalNumberOfElements * this.sizeOfEachElement];
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyFileStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyFileStore.java
new file mode 100644
index 0000000..0d113d6
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyFileStore.java
@@ -0,0 +1,92 @@
+/*
+ * 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.datastorage.store.impl.key.compressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+
+public class CompressedSingleArrayKeyFileStore extends AbstractCompressedSingleArrayStore {
+
+  /**
+   * offset, this will be used for seek position
+   */
+  private long offset;
+
+  /**
+   * fully qualified file path
+   */
+  private String filePath;
+
+  /**
+   * length to be read
+   */
+  private int length;
+
+  public CompressedSingleArrayKeyFileStore(int size, int elementSize, long offset, String filePath,
+      int length) {
+    super(size, elementSize, false);
+    this.offset = offset;
+    this.filePath = filePath;
+    this.length = length;
+  }
+
+  /**
+   * This method will be used to get the actual keys array present in the
+   * store . Here back array will be uncompress array. This method will first read
+   * the data from file based on offset and length then uncompress the array
+   * to get the actual array
+   *
+   * @param fileHolder file holder will be used to read the file
+   * @return uncompressed
+   * keys will return uncompressed key
+   */
+  @Override public byte[] getBackArray(FileHolder fileHolder) {
+    if (null != fileHolder) {
+      // read from file based on offset and index, fileholder will read that
+      // much byte from that offset, then uncompress and return
+      return COMPRESSOR.unCompress(fileHolder.readByteArray(filePath, offset, length));
+    } else {
+      return new byte[0];
+    }
+  }
+
+  /**
+   * This method will be used to get the key array based on index
+   * This method will first read
+   * the data from file based on offset and length then uncompress the array
+   * to get the actual array, then get the array for index and return
+   *
+   * @param index      index in store
+   * @param fileHolder file holder will be used to read the file
+   * @return key
+   */
+  @Override public byte[] get(int index, FileHolder fileHolder) {
+    // read from file based on offset and index, fileholder will read that
+    // much byte from that offset, then uncompress to get the actual array
+    byte[] unCompress = COMPRESSOR.unCompress(fileHolder.readByteArray(filePath, offset, length));
+    // create new array of size of each element
+    byte[] copy = new byte[sizeOfEachElement];
+    // copy array for given index
+    // copy will done based on below calculation
+    // eg: index is 4 and size of each key is 6 then copy from 6*4= 24th
+    // index till 29th index
+    System.arraycopy(unCompress, ((index) * sizeOfEachElement), copy, 0, sizeOfEachElement);
+    return copy;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyInMemoryStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyInMemoryStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyInMemoryStore.java
new file mode 100644
index 0000000..612d434
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/compressed/CompressedSingleArrayKeyInMemoryStore.java
@@ -0,0 +1,46 @@
+/*
+ * 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.datastorage.store.impl.key.compressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+
+public class CompressedSingleArrayKeyInMemoryStore extends AbstractCompressedSingleArrayStore {
+  /**
+   * @param size
+   * @param elementSize
+   */
+  public CompressedSingleArrayKeyInMemoryStore(int size, int elementSize) {
+    super(size, elementSize);
+  }
+
+  /**
+   * @param size
+   * @param elementSize
+   * @param offset
+   * @param filePath
+   * @param fileHolder
+   * @param length
+   */
+  public CompressedSingleArrayKeyInMemoryStore(int size, int elementSize, long offset,
+      String filePath, FileHolder fileHolder, int length) {
+    this(size, elementSize);
+    datastore = fileHolder.readByteArray(filePath, offset, length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/AbstractSingleArrayKeyStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/AbstractSingleArrayKeyStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/AbstractSingleArrayKeyStore.java
new file mode 100644
index 0000000..e4141c3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/AbstractSingleArrayKeyStore.java
@@ -0,0 +1,107 @@
+/*
+ * 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.datastorage.store.impl.key.uncompressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.NodeKeyStore;
+
+public abstract class AbstractSingleArrayKeyStore implements NodeKeyStore {
+
+  /**
+   * size of each element
+   */
+  protected final int sizeOfEachElement;
+  /**
+   * total number of elements
+   */
+  protected final int totalNumberOfElements;
+  /**
+   * data store which will hold the data
+   */
+  protected byte[] datastore;
+
+  public AbstractSingleArrayKeyStore(int size, int elementSize) {
+    this.sizeOfEachElement = elementSize;
+    this.totalNumberOfElements = size;
+    datastore = new byte[size * elementSize];
+  }
+
+  /**
+   * This method will be used to insert mdkey to store
+   *
+   * @param index index of mdkey
+   * @param value mdkey
+   */
+  @Override public void put(int index, byte[] value) {
+    System.arraycopy(value, 0, datastore, ((index) * sizeOfEachElement), sizeOfEachElement);
+  }
+
+  /**
+   * This method will be used to get the writable key array.
+   * writable key array will hold below information:
+   * <size of key array><key array>
+   * total length will be stored in 4 bytes+ key array length for key array
+   *
+   * @return writable array
+   */
+  @Override public byte[] getWritableKeyArray() {
+    // create and allocate size for byte buffer
+    //  4 bytes for size of array(for array length) + size of array(for array)
+    return datastore;
+  }
+
+  /**
+   * This method will be used to get the actual key array present in the
+   * store.
+   *
+   * @param fileHolder file holder will be used to read the file
+   * @return uncompressed keys
+   * will return uncompressed key
+   */
+  @Override public byte[] getBackArray(FileHolder fileHolder) {
+    return datastore;
+  }
+
+  /**
+   * This method will be used to get the key array based on index
+   *
+   * @param index      index in store
+   * @param fileHolder file holder will be used to read the file
+   * @return key
+   */
+  @Override public byte[] get(int index, FileHolder fileHolder) {
+    // create new array of size of each element
+    byte[] copy = new byte[sizeOfEachElement];
+
+    // copy array for given index
+    // copy will done based on below calculation
+    // eg: index is 4 and size of each key is 6 then copy from 6*4= 24th
+    // index till 29th index
+    System.arraycopy(datastore, ((index) * sizeOfEachElement), copy, 0, sizeOfEachElement);
+    return copy;
+  }
+
+  /**
+   * This method will clear the store and create the new empty store
+   */
+  @Override public void clear() {
+    datastore = new byte[this.totalNumberOfElements * this.sizeOfEachElement];
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyFileStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyFileStore.java
new file mode 100644
index 0000000..2d44245
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyFileStore.java
@@ -0,0 +1,104 @@
+/*
+ * 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.datastorage.store.impl.key.uncompressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+
+public class SingleArrayKeyFileStore extends AbstractSingleArrayKeyStore {
+  /**
+   * offset, this will be used for seek position
+   */
+  private long offset;
+
+  /**
+   * fully qualified file path
+   */
+  private String filePath;
+
+  /**
+   * length to be read
+   */
+  private int length;
+
+  /**
+   * @param size
+   * @param elementSize
+   */
+  public SingleArrayKeyFileStore(int size, int elementSize) {
+    super(size, elementSize);
+  }
+
+  /**
+   * @param size
+   * @param elementSize
+   * @param offset
+   * @param filePath
+   * @param length
+   */
+  public SingleArrayKeyFileStore(int size, int elementSize, long offset, String filePath,
+      int length) {
+    this(size, elementSize);
+    this.offset = offset;
+    this.filePath = filePath;
+    this.length = length;
+    datastore = null;
+  }
+
+  /**
+   * This method will be used to get the actual keys array present in the
+   * store. This method will read
+   * the data from file based on offset and length then return the data read from file
+   *
+   * @param fileHolder file holder will be used to read the file
+   * @return uncompressed
+   * keys will return uncompressed key
+   */
+  @Override public byte[] getBackArray(FileHolder fileHolder) {
+    if (null != fileHolder) {
+      return fileHolder.readByteArray(filePath, offset, length);
+    } else {
+      return new byte[0];
+    }
+  }
+
+  /**
+   * This method will be used to get the key array based on index This method
+   * will first read the data from file based on offset and length then get
+   * the array for index and return
+   *
+   * @param index      index in store
+   * @param fileHolder file holder will be used to read the file
+   * @return key
+   */
+  @Override public byte[] get(int index, FileHolder fileHolder) {
+    // read from file based on offset and index, fileholder will read that
+    // much byte from that offset,
+    byte[] unCompress = fileHolder.readByteArray(filePath, offset, length);
+    // create new array of size of each element
+    byte[] copy = new byte[sizeOfEachElement];
+    // copy array for given index
+    // copy will done based on below calculation
+    // eg: index is 4 and size of each key is 6 then copy from 6*4= 24th
+    // index till 29th index
+    System.arraycopy(unCompress, ((index) * sizeOfEachElement), copy, 0, sizeOfEachElement);
+    return copy;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyInMemoryStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyInMemoryStore.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyInMemoryStore.java
new file mode 100644
index 0000000..6e0dde3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/impl/key/uncompressed/SingleArrayKeyInMemoryStore.java
@@ -0,0 +1,36 @@
+/*
+ * 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.datastorage.store.impl.key.uncompressed;
+
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+
+public class SingleArrayKeyInMemoryStore extends AbstractSingleArrayKeyStore {
+
+  public SingleArrayKeyInMemoryStore(int size, int elementSize) {
+    super(size, elementSize);
+  }
+
+  public SingleArrayKeyInMemoryStore(int size, int elementSize, long offset, String filePath,
+      FileHolder fileHolder, int length) {
+    this(size, elementSize);
+    datastore = fileHolder.readByteArray(filePath, offset, length);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/datastorage/util/StoreFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/util/StoreFactory.java b/core/src/main/java/org/apache/carbondata/core/datastorage/util/StoreFactory.java
new file mode 100644
index 0000000..408524f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/util/StoreFactory.java
@@ -0,0 +1,62 @@
+/*
+ * 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.datastorage.util;
+
+import org.apache.carbondata.core.datastorage.store.NodeMeasureDataStore;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.datastorage.store.impl.data.compressed.HeavyCompressedDoubleArrayDataInMemoryStore;
+import org.apache.carbondata.core.datastorage.store.impl.data.uncompressed.DoubleArrayDataInMemoryStore;
+
+public final class StoreFactory {
+  /**
+   * value type.
+   */
+  private static StoreType valueType;
+
+  static {
+    valueType = StoreType.HEAVY_VALUE_COMPRESSION;
+  }
+
+  private StoreFactory() {
+
+  }
+
+  public static NodeMeasureDataStore createDataStore(ValueCompressionModel compressionModel) {
+    switch (valueType) {
+      case COMPRESSED_DOUBLE_ARRAY:
+        return new DoubleArrayDataInMemoryStore(compressionModel);
+
+      case HEAVY_VALUE_COMPRESSION:
+        return new HeavyCompressedDoubleArrayDataInMemoryStore(compressionModel);
+      default:
+        return new HeavyCompressedDoubleArrayDataInMemoryStore(compressionModel);
+    }
+  }
+
+  /**
+   * enum defined.
+   */
+  public enum StoreType {
+    COMPRESSED_SINGLE_ARRAY,
+    COMPRESSED_DOUBLE_ARRAY,
+    HEAVY_VALUE_COMPRESSION
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenException.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenException.java
new file mode 100644
index 0000000..2824715
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenException.java
@@ -0,0 +1,45 @@
+/*
+ * 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.keygenerator;
+
+/**
+ * It can be thrown while generating the key.
+ */
+public class KeyGenException extends Exception {
+
+  private static final long serialVersionUID = 3105132151795358241L;
+
+  public KeyGenException() {
+    super();
+  }
+
+  public KeyGenException(Exception e) {
+    super(e);
+  }
+
+  public KeyGenException(Exception e, String msg) {
+    super(msg, e);
+  }
+
+  public KeyGenException(String msg) {
+    super(msg);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java
new file mode 100644
index 0000000..dc50b10
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java
@@ -0,0 +1,124 @@
+/*
+ * 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.keygenerator;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+/**
+ * It generates the key by using multiple keys(typically multiple dimension keys
+ * are combined to form a single key). And it can return the individual
+ * key(dimensional key) out of combined key.
+ */
+public interface KeyGenerator extends Serializable, Comparator<byte[]> {
+  /**
+   * It generates the single key aka byte array from multiple keys.
+   *
+   * @param keys
+   * @return byte array
+   * @throws KeyGenException
+   */
+  byte[] generateKey(long[] keys) throws KeyGenException;
+
+  /**
+   * It generates the single key aka byte array from multiple keys.
+   *
+   * @param keys
+   * @return
+   * @throws KeyGenException
+   */
+  byte[] generateKey(int[] keys) throws KeyGenException;
+
+  /**
+   * It gets array of keys out of single key aka byte array
+   *
+   * @param key
+   * @return array of keys.
+   */
+  long[] getKeyArray(byte[] key);
+
+  /**
+   * It gets array of keys out of single key aka byte array
+   *
+   * @param key
+   * @param offset
+   * @return array of keys.
+   */
+  long[] getKeyArray(byte[] key, int offset);
+
+  /**
+   * It gets array of keys out of single key aka byte array
+   *
+   * @param key
+   * @param maskedByteRanges
+   * @return array of keys
+   */
+  long[] getKeyArray(byte[] key, int[] maskedByteRanges);
+
+  /**
+   * It gets the key in the specified index from the single key aka byte array
+   *
+   * @param key
+   * @param index of key.
+   * @return key
+   */
+  long getKey(byte[] key, int index);
+
+  /**
+   * Set any extra properties if required.
+   */
+  void setProperty(Object key, Object value);
+
+  /**
+   * Gives the key size in number of bytes.
+   */
+  int getKeySizeInBytes();
+
+  /**
+   * It gets the specified index and size from the single key aka byte aray
+   *
+   * @param key
+   * @param index
+   * @param size
+   * @return
+   */
+  long[] getSubKeyArray(byte[] key, int index, int size);
+
+  /**
+   * returns key bytes offset
+   *
+   * @param index
+   * @return
+   */
+  int[] getKeyByteOffsets(int index);
+
+  int compare(byte[] key1, int offset1, int length1, byte[] key2, int offset2, int length2);
+
+  /**
+   * returns the dimension count
+   *
+   * @return
+   */
+  int getDimCount();
+
+  int getStartAndEndKeySizeWithOnlyPrimitives();
+
+  void setStartAndEndKeySizeWithOnlyPrimitives(int startAndEndKeySizeWithPrimitives);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java
new file mode 100644
index 0000000..b2ea154
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java
@@ -0,0 +1,103 @@
+/*
+ * 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.keygenerator.columnar;
+
+import org.apache.carbondata.core.keygenerator.KeyGenException;
+
+/**
+ * Splits the odometer key to columns.Further these columns can be stored in a columnar storage.
+ */
+public interface ColumnarSplitter {
+  /**
+   * Splits generated MDKey to multiple columns.
+   *
+   * @param key MDKey
+   * @return Multiple columns in 2 dimensional byte array
+   */
+  byte[][] splitKey(byte[] key);
+
+  /**
+   * It generates and splits key to multiple columns
+   *
+   * @param keys
+   * @return
+   * @throws KeyGenException
+   */
+  byte[][] generateAndSplitKey(long[] keys) throws KeyGenException;
+
+  /**
+   * It generates and splits key to multiple columns
+   *
+   * @param keys
+   * @return
+   * @throws KeyGenException
+   */
+  byte[][] generateAndSplitKey(int[] keys) throws KeyGenException;
+
+  /**
+   * Takes the split keys and generates the surrogate key array
+   *
+   * @param key
+   * @return
+   */
+  long[] getKeyArray(byte[][] key);
+
+  /**
+   * Takes the split keys and generates the surrogate key array in bytes
+   *
+   * @param key
+   * @return
+   */
+  byte[] getKeyByteArray(byte[][] key);
+
+  /**
+   * Takes the split keys and generates the surrogate key array in bytes
+   *
+   * @param key
+   * @param columnIndexes, takes columnIndexes to consider which columns are present in the key
+   * @return
+   */
+  byte[] getKeyByteArray(byte[][] key, int[] columnIndexes);
+
+  /**
+   * Takes the split keys and generates the surrogate key array
+   *
+   * @param key
+   * @param columnIndexes, takes columnIndexes to consider which columns are present in the key
+   * @return
+   */
+  long[] getKeyArray(byte[][] key, int[] columnIndexes);
+
+  /**
+   * Below method will be used to get the block size
+   *
+   * @return
+   */
+  int[] getBlockKeySize();
+
+  /**
+   * Below method will be used to get the total key Size of the particular block
+   *
+   * @param blockIndexes
+   * @return
+   */
+  int getKeySizeByBlock(int[] blockIndexes);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java
new file mode 100644
index 0000000..9276fe3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java
@@ -0,0 +1,244 @@
+/*
+ * 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.keygenerator.columnar.impl;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.keygenerator.KeyGenException;
+import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
+import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
+
+/**
+ * It is Equi Split implementation class of Columnar splitter. And uses var key length
+ * generator to generate keys.
+ * It splits depends on the @dimensionsToSplit parameter. This parameter decides how many
+ * dimensions should be present in each column.
+ */
+public class MultiDimKeyVarLengthEquiSplitGenerator extends MultiDimKeyVarLengthGenerator
+    implements ColumnarSplitter {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = -7767757692821917570L;
+
+  private byte dimensionsToSplit;
+
+  private int[][] splitDimArray;
+
+  private int[][] dimBlockArray;
+
+  private int[][][] byteRangesForDims;
+
+  private int[] blockKeySize;
+
+  public MultiDimKeyVarLengthEquiSplitGenerator(int[] lens, byte dimensionsToSplit) {
+    super(lens);
+    this.dimensionsToSplit = dimensionsToSplit;
+    intialize();
+  }
+
+  private void intialize() {
+    byte s = 0;
+    List<Set<Integer>> splitList =
+        new ArrayList<Set<Integer>>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    Set<Integer> split = new TreeSet<Integer>();
+    splitList.add(split);
+    for (int i = 0; i < byteRangesForKeys.length; i++) {
+      if (s == dimensionsToSplit) {
+        s = 0;
+        split = new TreeSet<Integer>();
+        splitList.add(split);
+      }
+      for (int j = 0; j < byteRangesForKeys[i].length; j++) {
+        for (int j2 = byteRangesForKeys[i][0]; j2 <= byteRangesForKeys[i][1]; j2++) {
+          split.add(j2);
+        }
+      }
+      s++;
+    }
+    List<Integer>[] splits = new List[splitList.size()];
+    int i = 0;
+    for (Set<Integer> splitLocal : splitList) {
+      List<Integer> range = new ArrayList<Integer>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      for (Integer index : splitLocal) {
+        range.add(index);
+      }
+      splits[i++] = range;
+    }
+    for (int j = 1; j < splits.length; j++) {
+      if (splits[j - 1].get(splits[j - 1].size() - 1) == splits[j].get(0)) {
+        splits[j].remove(0);
+      }
+    }
+    splitDimArray = new int[splits.length][];
+    for (int j = 0; j < splits.length; j++) {
+      int[] a = convertToArray(splits[j]);
+      splitDimArray[j] = a.length > 0 ? new int[] { a[0], a[a.length - 1] } : a;
+    }
+
+    dimBlockArray = new int[byteRangesForKeys.length][];
+    Set<Integer>[] dimBlockSet = new Set[dimBlockArray.length];
+    for (int k = 0; k < byteRangesForKeys.length; k++) {
+      int[] dimRange = byteRangesForKeys[k];
+      Set<Integer> dimBlockPosSet = new TreeSet<Integer>();
+      dimBlockSet[k] = dimBlockPosSet;
+      for (int j = 0; j < splitDimArray.length; j++) {
+        if (dimRange[0] >= splitDimArray[j][0] && dimRange[0] <= splitDimArray[j][1]) {
+          dimBlockPosSet.add(j);
+        }
+        if (dimRange[1] >= splitDimArray[j][0] && dimRange[1] <= splitDimArray[j][1]) {
+          dimBlockPosSet.add(j);
+        }
+      }
+
+    }
+
+    for (int j = 0; j < dimBlockSet.length; j++) {
+      dimBlockArray[j] = convertToArray(dimBlockSet[j]);
+    }
+
+    int[][] splitDimArrayLocalIndexes = new int[splitDimArray.length][];
+    for (int j = 0; j < splitDimArrayLocalIndexes.length; j++) {
+      splitDimArrayLocalIndexes[j] = splitDimArray[j].length > 0 ?
+          new int[] { 0, splitDimArray[j][1] - splitDimArray[j][0] } :
+          new int[0];
+    }
+
+    byteRangesForDims = new int[byteRangesForKeys.length][][];
+    for (int j = 0; j < byteRangesForKeys.length; j++) {
+      if (dimBlockArray[j].length > 1) {
+        int[] bArray1 = splitDimArrayLocalIndexes[dimBlockArray[j][0]];
+        byteRangesForDims[j] = new int[2][2];
+        byteRangesForDims[j][0] =
+            new int[] { bArray1[bArray1.length - 1], bArray1[bArray1.length - 1] };
+        byteRangesForDims[j][1] = new int[] { 0,
+            (byteRangesForKeys[j][byteRangesForKeys[j].length - 1] - byteRangesForKeys[j][0]) - 1 };
+      } else {
+        byteRangesForDims[j] = new int[1][1];
+        int[] bArray1 = splitDimArray[dimBlockArray[j][0]];
+        byteRangesForDims[j][0] = new int[] { byteRangesForKeys[j][0] - bArray1[0],
+            byteRangesForKeys[j][1] - bArray1[0] };
+      }
+    }
+    blockKeySize = new int[splitDimArray.length];
+
+    for (int j = 0; j < blockKeySize.length; j++) {
+      blockKeySize[j] =
+          splitDimArray[j].length > 0 ? splitDimArray[j][1] - splitDimArray[j][0] + 1 : 0;
+    }
+  }
+
+  private int[] convertToArray(List<Integer> list) {
+    int[] ints = new int[list.size()];
+    for (int i = 0; i < ints.length; i++) {
+      ints[i] = list.get(i);
+    }
+    return ints;
+  }
+
+  private int[] convertToArray(Set<Integer> set) {
+    int[] ints = new int[set.size()];
+    int i = 0;
+    for (Iterator iterator = set.iterator(); iterator.hasNext(); ) {
+      ints[i++] = (Integer) iterator.next();
+    }
+    return ints;
+  }
+
+  @Override public byte[][] splitKey(byte[] key) {
+    byte[][] split = new byte[blockKeySize.length][];
+    int copyIndex = 0;
+    for (int i = 0; i < split.length; i++) {
+      split[i] = new byte[blockKeySize[i]];
+      System.arraycopy(key, copyIndex, split[i], 0, split[i].length);
+      copyIndex += blockKeySize[i];
+    }
+    return split;
+  }
+
+  @Override public byte[][] generateAndSplitKey(long[] keys) throws KeyGenException {
+    return splitKey(generateKey(keys));
+  }
+
+  @Override public byte[][] generateAndSplitKey(int[] keys) throws KeyGenException {
+    return splitKey(generateKey(keys));
+  }
+
+  @Override public long[] getKeyArray(byte[][] key) {
+    byte[] fullKey = new byte[getKeySizeInBytes()];
+    int copyIndex = 0;
+    for (int i = 0; i < key.length; i++) {
+      System.arraycopy(key[i], 0, fullKey, copyIndex, key[i].length);
+      copyIndex += key[i].length;
+    }
+    return getKeyArray(fullKey);
+  }
+
+  @Override public byte[] getKeyByteArray(byte[][] key) {
+    byte[] fullKey = new byte[getKeySizeInBytes()];
+    int copyIndex = 0;
+    for (int i = 0; i < key.length; i++) {
+      System.arraycopy(key[i], 0, fullKey, copyIndex, key[i].length);
+      copyIndex += key[i].length;
+    }
+    return fullKey;
+  }
+
+  @Override public byte[] getKeyByteArray(byte[][] key, int[] columnIndexes) {
+    return null;
+  }
+
+  @Override public long[] getKeyArray(byte[][] key, int[] columnIndexes) {
+    return null;
+  }
+
+  public int[] getBlockKeySize() {
+    return blockKeySize;
+  }
+
+  @Override public int getKeySizeByBlock(int[] blockIndexes) {
+    int size = 0;
+
+    for (int i = 0; i < blockIndexes.length; i++) {
+      if (blockIndexes[i] < blockKeySize.length) {
+        size += blockKeySize[blockIndexes[i]];
+      }
+    }
+    return size;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof MultiDimKeyVarLengthEquiSplitGenerator)) {
+      return false;
+    }
+    MultiDimKeyVarLengthEquiSplitGenerator o = (MultiDimKeyVarLengthEquiSplitGenerator)obj;
+    return o.dimensionsToSplit == dimensionsToSplit && super.equals(obj);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode() + dimensionsToSplit;
+  }
+}