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:23 UTC

[38/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/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java
new file mode 100644
index 0000000..34515da
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java
@@ -0,0 +1,239 @@
+/*
+ * 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.Arrays;
+import java.util.HashSet;
+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;
+
+public class MultiDimKeyVarLengthVariableSplitGenerator extends MultiDimKeyVarLengthGenerator
+    implements ColumnarSplitter {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+
+  private int[] dimensionsToSplit;
+
+  private int[] blockKeySize;
+
+  public MultiDimKeyVarLengthVariableSplitGenerator(int[] lens, int[] dimSplit) {
+    super(lens);
+    this.dimensionsToSplit = dimSplit;
+    initialise();
+
+  }
+
+  private void initialise() {
+    int s = 0;
+    List<Set<Integer>> splitList =
+        new ArrayList<Set<Integer>>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    Set<Integer> split = new TreeSet<Integer>();
+    splitList.add(split);
+    int dimSplitIndx = 0;
+
+    for (int i = 0; i < byteRangesForKeys.length; i++) {
+      if (s == dimensionsToSplit[dimSplitIndx]) {
+        s = 0;
+        split = new TreeSet<Integer>();
+        splitList.add(split);
+        dimSplitIndx++;
+      }
+      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);
+      }
+    }
+    int[][] 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;
+    }
+
+    int[][] 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];
+    }
+
+    int[][][] 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) {
+    Set<Integer> selectedRanges = new HashSet<>();
+    for (int i = 0; i < blockIndexes.length; i++) {
+      int[] byteRange = byteRangesForKeys[blockIndexes[i]];
+      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
+        selectedRanges.add(j);
+      }
+    }
+    return selectedRanges.size();
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof MultiDimKeyVarLengthVariableSplitGenerator)) {
+      return false;
+    }
+    MultiDimKeyVarLengthVariableSplitGenerator o = (MultiDimKeyVarLengthVariableSplitGenerator)obj;
+    return Arrays.equals(o.dimensionsToSplit, dimensionsToSplit) && super.equals(obj);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode() + Arrays.hashCode(dimensionsToSplit);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
new file mode 100644
index 0000000..595760a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
@@ -0,0 +1,56 @@
+/*
+ * 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.directdictionary;
+
+/**
+ * The interface provides the method to generate dictionary key
+ * and getting the actual value from the dictionaryKey for direct dictionary column.
+ */
+public interface DirectDictionaryGenerator {
+
+  /**
+   * The method generate and returns the dictionary / surrogate key for direct dictionary column
+   *
+   * @param member The member string value
+   * @return returns dictionary/ surrogate value
+   */
+  int generateDirectSurrogateKey(String member);
+
+  /**
+   * The method returns the actual value of the requested dictionary / surrogate
+   *
+   * @param key
+   * @return dictionary actual member
+   */
+  Object getValueFromSurrogate(int key);
+
+  /**
+   * The method generate and returns the dictionary / surrogate key for direct dictionary column
+   * This Method is called while executing filter queries for getting direct surrogate members.
+   * Currently the query engine layer only supports yyyy-MM-dd HH:mm:ss date format no matter
+   * in which format the data is been stored, so while retrieving the direct surrogate value for
+   * filter member first it should be converted in date form as per above format and needs to
+   * retrieve time stamp.
+   *
+   * @param member The member string value
+   * @return returns dictionary/ surrogate value
+   */
+  int generateDirectSurrogateKey(String memberStr, String format);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
new file mode 100644
index 0000000..b038789
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.directdictionary;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampDirectDictionaryGenerator;
+
+/**
+ * Factory for DirectDictionary Key generator
+ */
+public final class DirectDictionaryKeyGeneratorFactory {
+  /**
+   * private constructor
+   */
+  private DirectDictionaryKeyGeneratorFactory() {
+
+  }
+
+  /**
+   * The method returns the DirectDictionaryGenerator based for direct dictionary
+   * column based on dataType
+   *
+   * @param dataType DataType
+   * @return the generator instance
+   */
+  public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType) {
+    DirectDictionaryGenerator directDictionaryGenerator = null;
+    switch (dataType) {
+      case TIMESTAMP:
+        directDictionaryGenerator = TimeStampDirectDictionaryGenerator.instance;
+        break;
+      default:
+
+    }
+    return directDictionaryGenerator;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
new file mode 100644
index 0000000..7b8e49f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
@@ -0,0 +1,215 @@
+/*
+ * 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.directdictionary.timestamp;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.util.CarbonProperties;
+
+import static org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants.TIME_GRAN_DAY;
+import static org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants.TIME_GRAN_HOUR;
+import static org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants.TIME_GRAN_MIN;
+import static org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants.TIME_GRAN_SEC;
+
+/**
+ * The class provides the method to generate dictionary key and getting the actual value from
+ * the dictionaryKey for direct dictionary column for TIMESTAMP type.
+ */
+public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
+
+  private TimeStampDirectDictionaryGenerator() {
+
+  }
+
+  public static TimeStampDirectDictionaryGenerator instance =
+      new TimeStampDirectDictionaryGenerator();
+
+  /**
+   * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
+   */
+  public static final long granularityFactor;
+  /**
+   * The date timestamp to be considered as start date for calculating the timestamp
+   * java counts the number of milliseconds from  start of "January 1, 1970", this property is
+   * customized the start of position. for example "January 1, 2000"
+   */
+  public static final long cutOffTimeStamp;
+  /**
+   * Logger instance
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
+
+  /**
+   * initialization block for granularityFactor and cutOffTimeStamp
+   */
+  static {
+    String cutOffTimeStampString = CarbonProperties.getInstance()
+        .getProperty(TimeStampGranularityConstants.CARBON_CUTOFF_TIMESTAMP);
+    String timeGranularity = CarbonProperties.getInstance()
+        .getProperty(TimeStampGranularityConstants.CARBON_TIME_GRANULARITY, TIME_GRAN_SEC);
+    long granularityFactorLocal = 1000;
+    switch (timeGranularity) {
+      case TIME_GRAN_SEC:
+        granularityFactorLocal = TimeStampGranularityTypeValue.MILLIS_SECONDS.getValue();
+        break;
+      case TIME_GRAN_MIN:
+        granularityFactorLocal = TimeStampGranularityTypeValue.MILLIS_MINUTE.getValue();
+        break;
+      case TIME_GRAN_HOUR:
+        granularityFactorLocal = TimeStampGranularityTypeValue.MILLIS_HOUR.getValue();
+        break;
+      case TIME_GRAN_DAY:
+        granularityFactorLocal = TimeStampGranularityTypeValue.MILLIS_DAY.getValue();
+        break;
+      default:
+        granularityFactorLocal = 1000;
+    }
+    long cutOffTimeStampLocal;
+    if (null == cutOffTimeStampString) {
+      cutOffTimeStampLocal = -1;
+    } else {
+      try {
+        SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+        timeParser.setLenient(false);
+        Date dateToStr = timeParser.parse(cutOffTimeStampString);
+        cutOffTimeStampLocal = dateToStr.getTime();
+      } catch (ParseException e) {
+        LOGGER.warn("Cannot convert" + cutOffTimeStampString
+            + " to Time/Long type value. Value considered for cutOffTimeStamp is -1." + e
+            .getMessage());
+        cutOffTimeStampLocal = -1;
+      }
+    }
+    granularityFactor = granularityFactorLocal;
+    cutOffTimeStamp = cutOffTimeStampLocal;
+  }
+
+  /**
+   * The method take member String as input and converts
+   * and returns the dictionary key
+   *
+   * @param memberStr date format string
+   * @return dictionary value
+   */
+  @Override public int generateDirectSurrogateKey(String memberStr) {
+    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+    timeParser.setLenient(false);
+    if (null == memberStr || memberStr.trim().isEmpty() || memberStr
+        .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+      return 1;
+    }
+    return getDirectSurrogateForMember(memberStr, timeParser);
+  }
+
+  /**
+   * The method take member String as input and converts
+   * and returns the dictionary key
+   *
+   * @param memberStr date format string
+   * @return dictionary value
+   */
+  public int generateDirectSurrogateKey(String memberStr, String format) {
+    if (null == format) {
+      return generateDirectSurrogateKeyForNonTimestampType(memberStr);
+    } else {
+      SimpleDateFormat timeParser = new SimpleDateFormat(format);
+      timeParser.setLenient(false);
+      if (null == memberStr || memberStr.trim().isEmpty() || memberStr
+          .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+        return 1;
+      }
+      return getDirectSurrogateForMember(memberStr, timeParser);
+    }
+  }
+
+  private int getDirectSurrogateForMember(String memberStr, SimpleDateFormat timeParser) {
+    Date dateToStr = null;
+    try {
+      dateToStr = timeParser.parse(memberStr);
+    } catch (ParseException e) {
+      LOGGER.debug(
+          "Cannot convert " + memberStr + " to Time/Long type value. Value considered as null." + e
+              .getMessage());
+      dateToStr = null;
+    }
+    //adding +2 to reserve the first cuttOffDiff value for null or empty date
+    if (null == dateToStr) {
+      return 1;
+    } else {
+      return generateKey(dateToStr.getTime());
+    }
+  }
+
+  /**
+   * The method take dictionary key as input and returns the
+   *
+   * @param key
+   * @return member value/actual value Date
+   */
+  @Override public Object getValueFromSurrogate(int key) {
+    if (key == 1) {
+      return null;
+    }
+    long timeStamp = 0;
+    if (cutOffTimeStamp >= 0) {
+      timeStamp = ((key - 2) * granularityFactor + cutOffTimeStamp);
+    } else {
+      timeStamp = (key - 2) * granularityFactor;
+    }
+    return timeStamp * 1000L;
+  }
+
+  private int generateDirectSurrogateKeyForNonTimestampType(String memberStr) {
+    long timeValue = -1;
+    try {
+      timeValue = Long.valueOf(memberStr) / 1000;
+    } catch (NumberFormatException e) {
+      LOGGER.debug(
+          "Cannot convert " + memberStr + " Long type value. Value considered as null." + e
+              .getMessage());
+    }
+    if (timeValue == -1) {
+      return 1;
+    } else {
+      return generateKey(timeValue);
+    }
+  }
+
+  private int generateKey(long timeValue) {
+    if (cutOffTimeStamp >= 0) {
+      int keyValue = (int) ((timeValue - cutOffTimeStamp) / granularityFactor);
+      return keyValue < 0 ? 1 : keyValue + 2;
+    } else {
+      int keyValue = (int) (timeValue / granularityFactor);
+      return keyValue < 0 ? 1 : keyValue + 2;
+    }
+  }
+
+}
\ 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/directdictionary/timestamp/TimeStampGranularityConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityConstants.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityConstants.java
new file mode 100644
index 0000000..e897843
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityConstants.java
@@ -0,0 +1,54 @@
+/*
+ * 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.directdictionary.timestamp;
+
+/**
+ * Constant related to timestamp conversion
+ */
+public interface TimeStampGranularityConstants {
+
+  /**
+   * The property to set the date to be considered as start date for calculating the timestamp
+   * java counts the number of milliseconds from  start of "January 1, 1970", this property is
+   * customized the start of position. for example "January 1, 2000"
+   */
+  public static final String CARBON_CUTOFF_TIMESTAMP = "carbon.cutOffTimestamp";
+  /**
+   * The property to set the timestamp (ie milis) conversion to the SECOND, MINUTE, HOUR
+   * or DAY level
+   */
+  public static final String CARBON_TIME_GRANULARITY = "carbon.timegranularity";
+
+  /**
+   * Second level key
+   */
+  String TIME_GRAN_SEC = "SECOND";
+  /**
+   * minute level key
+   */
+  String TIME_GRAN_MIN = "MINUTE";
+  /**
+   * hour level key
+   */
+  String TIME_GRAN_HOUR = "HOUR";
+  /**
+   * day level key
+   */
+  String TIME_GRAN_DAY = "DAY";
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityTypeValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityTypeValue.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityTypeValue.java
new file mode 100644
index 0000000..3b50a5c
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampGranularityTypeValue.java
@@ -0,0 +1,63 @@
+/*
+ * 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.directdictionary.timestamp;
+
+/**
+ * Enum constant having the milli second for second, minute, hour, day
+ */
+public enum TimeStampGranularityTypeValue {
+  /**
+   * 1 second value in ms
+   */
+  MILLIS_SECONDS(1000),
+  /**
+   * 1 minute value in ms
+   */
+  MILLIS_MINUTE(1000 * 60),
+  /**
+   * 1 hour value in ms
+   */
+  MILLIS_HOUR(1000 * 60 * 60),
+  /**
+   * 1 day value in ms
+   */
+  MILLIS_DAY(1000 * 60 * 60 * 24);
+
+  /**
+   * enum constant value
+   */
+  private final long value;
+
+  /**
+   * constructor of enum constant
+   *
+   * @param value
+   */
+  private TimeStampGranularityTypeValue(long value) {
+    this.value = value;
+  }
+
+  /**
+   * @return return the value of enum constant
+   */
+  public long getValue() {
+    return this.value;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java
new file mode 100644
index 0000000..ab3e7ac
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.factory;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.keygenerator.KeyGenerator;
+import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+public final class KeyGeneratorFactory {
+  private KeyGeneratorFactory() {
+
+  }
+
+  public static KeyGenerator getKeyGenerator(int[] dimesion) {
+    int[] incrementedCardinality;
+    boolean isFullyFilled =
+        Boolean.parseBoolean(CarbonCommonConstants.IS_FULLY_FILLED_BITS_DEFAULT_VALUE);
+    if (!isFullyFilled) {
+      incrementedCardinality = CarbonUtil.getIncrementedCardinality(dimesion);
+    } else {
+      incrementedCardinality = CarbonUtil.getIncrementedCardinalityFullyFilled(dimesion);
+    }
+    return new MultiDimKeyVarLengthGenerator(incrementedCardinality);
+  }
+
+  /**
+   *
+   * @param dimCardinality : dimension cardinality
+   * @param columnSplits : No of column in each block
+   * @return keygenerator
+   */
+  public static KeyGenerator getKeyGenerator(int[] dimCardinality, int[] columnSplits) {
+    int[] dimsBitLens = CarbonUtil.getDimensionBitLength(dimCardinality, columnSplits);
+
+    return new MultiDimKeyVarLengthGenerator(dimsBitLens);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/AbstractKeyGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/AbstractKeyGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/AbstractKeyGenerator.java
new file mode 100644
index 0000000..76128d4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/AbstractKeyGenerator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.mdkey;
+
+import org.apache.carbondata.core.keygenerator.KeyGenerator;
+
+public abstract class AbstractKeyGenerator implements KeyGenerator {
+
+  private static final long serialVersionUID = -6675293078575359769L;
+
+  @Override public int compare(byte[] byte1, byte[] byte2) {
+    // Short circuit equal case
+    if (byte1 == byte2) {
+      return 0;
+    }
+    // Bring WritableComparator code local
+    int i = 0;
+    int j = 0;
+    for (; i < byte1.length && j < byte2.length; i++, j++) {
+      int a = (byte1[i] & 0xff);
+      int b = (byte2[j] & 0xff);
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return 0;
+  }
+
+  public int compare(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2,
+      int length2) {
+    length1 += offset1;
+    length2 += offset2;
+    // Bring WritableComparator code local
+    for (; offset1 < length1 && offset2 < length2; offset1++, offset2++) {
+      int a = (buffer1[offset1] & 0xff);
+      int b = (buffer2[offset2] & 0xff);
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return 0;
+  }
+
+  @Override public void setProperty(Object key, Object value) {
+    /**
+     * No implementation required.
+     */
+  }
+
+  @Override public int getKeySizeInBytes() {
+    return 0;
+  }
+
+  @Override public int[] getKeyByteOffsets(int index) {
+    return null;
+  }
+
+  @Override public int getDimCount() {
+    return 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/Bits.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/Bits.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/Bits.java
new file mode 100644
index 0000000..2cdbc80
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/Bits.java
@@ -0,0 +1,327 @@
+/*
+ * 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.mdkey;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class Bits implements Serializable {
+
+  /**
+   * Bits MAX_LENGTH
+   */
+  private static final int MAX_LENGTH = 63;
+  private static final int LONG_LENGTH = 64;
+  /**
+   * serialVersionUID.
+   */
+  private static final long serialVersionUID = 1555114921503304849L;
+  /**
+   * LONG_MAX.
+   */
+  private static final long LONG_MAX = 0x7fffffffffffffffL;
+  /**
+   * length.
+   */
+  private int length = 100;
+  /**
+   * lens.
+   */
+  private int[] lens;
+  /**
+   * wsize.
+   */
+  private int wsize;
+  /**
+   * byteSize.
+   */
+  private int byteSize;
+
+  public Bits(int[] lens) {
+    this.lens = lens;
+    this.length = getTotalLength(lens);
+
+    wsize = length / LONG_LENGTH;
+    byteSize = length / 8;
+
+    if (length % LONG_LENGTH != 0) {
+      wsize++;
+    }
+
+    if (length % 8 != 0) {
+      byteSize++;
+    }
+  }
+
+  public int getByteSize() {
+    return byteSize;
+  }
+
+  private int getTotalLength(int[] lens) {
+    int tLen = 0;
+    for (int len : lens) {
+      tLen += len;
+    }
+    return tLen;
+  }
+
+  public int getDimCount() {
+    return lens.length;
+  }
+
+  /**
+   * Return the start and end Byte offsets of dimension in the MDKey. int []
+   * {start, end}
+   */
+  public int[] getKeyByteOffsets(int index) {
+    int prefixPaddingBits = length % 8 == 0 ? 0 : (8 - length % 8);
+
+    int priorLen = prefixPaddingBits;
+    int start = 0;
+    int end = 0;
+
+    // Calculate prior length for all previous keys
+    for (int i = 0; i < index; i++) {
+      priorLen += lens[i];
+    }
+
+    // Start
+    start = priorLen / 8;
+
+    int tillKeyLength = priorLen + lens[index];
+
+    // End key
+    end = (tillKeyLength) / 8;
+
+    // Consider if end is the last bit. No need to include the next byte.
+    if (tillKeyLength % 8 == 0) {
+      end--;
+    }
+
+    return new int[] { start, end };
+  }
+
+  protected long[] get(long[] keys) {
+    long[] words = new long[wsize];
+    int ll = 0;
+    for (int i = lens.length - 1; i >= 0; i--) {
+
+      long val = keys[i];
+
+      int idx = ll >> 6;// divide by 64 to get the new word index
+      int position = ll & 0x3f;// to ignore sign bit and consider the remaining
+      val = val & (LONG_MAX >> (MAX_LENGTH - lens[i]));// To control the
+      // logic so that
+      // any val do not
+      // exceed the
+      // cardinality
+      long mask = (val << position);
+      long word = words[idx];
+      words[idx] = (word | mask);
+      ll += lens[i];
+
+      int nextIndex = ll >> 6;// This is divide by 64
+
+      if (nextIndex != idx) {
+        int consideredBits = lens[i] - ll & 0x3f;
+        if (consideredBits < lens[i]) //Check for spill over only if all the bits are not considered
+        {
+          mask = (val >> (lens[i] - ll & 0x3f));//& (0x7fffffffffffffffL >> (0x3f-pos));
+          word = words[nextIndex];
+          words[nextIndex] = (word | mask);
+        }
+      }
+
+    }
+
+    return words;
+  }
+
+  protected long[] get(int[] keys) {
+    long[] words = new long[wsize];
+    int ll = 0;
+    for (int i = lens.length - 1; i >= 0; i--) {
+
+      long val = keys[i];
+
+      int index = ll >> 6;// divide by 64 to get the new word index
+      int pos = ll & 0x3f;// to ignore sign bit and consider the remaining
+      val = val & (LONG_MAX >> (MAX_LENGTH - lens[i]));// To control the
+      // logic so that
+      // any val do not
+      // exceed the
+      // cardinality
+      long mask = (val << pos);
+      long word = words[index];
+      words[index] = (word | mask);
+      ll += lens[i];
+
+      int nextIndex = ll >> 6;// This is divide by 64
+
+      if (nextIndex != index) {
+        int consideredBits = lens[i] - ll & 0x3f;
+        if (consideredBits < lens[i]) //Check for spill over only if all the bits are not considered
+        {
+          // Check for spill over
+          mask = (val >> (lens[i] - ll & 0x3f));
+          word = words[nextIndex];
+          words[nextIndex] = (word | mask);
+        }
+      }
+
+    }
+
+    return words;
+  }
+
+  private long[] getArray(long[] words) {
+    long[] vals = new long[lens.length];
+    int ll = 0;
+    for (int i = lens.length - 1; i >= 0; i--) {
+
+      int index = ll >> 6;
+      int pos = ll & 0x3f;
+      long val = words[index];
+      long mask = (LONG_MAX >>> (MAX_LENGTH - lens[i]));
+      mask = mask << pos;
+      vals[i] = (val & mask);
+      vals[i] >>>= pos;
+      ll += lens[i];
+
+      int nextIndex = ll >> 6;
+      if (nextIndex != index) {
+        pos = ll & 0x3f;
+        if (pos != 0) // Number of bits pending for current key is zero, no spill over
+        {
+          mask = (LONG_MAX >>> (MAX_LENGTH - pos));
+          val = words[nextIndex];
+          vals[i] = vals[i] | ((val & mask) << (lens[i] - pos));
+        }
+      }
+    }
+    return vals;
+  }
+
+  public byte[] getBytes(long[] keys) {
+
+    long[] words = get(keys);
+
+    return getBytesVal(words);
+  }
+
+  private byte[] getBytesVal(long[] words) {
+    int length = 8;
+    byte[] bytes = new byte[byteSize];
+
+    int l = byteSize - 1;
+    for (int i = 0; i < words.length; i++) {
+      long val = words[i];
+
+      for (int j = length - 1; j > 0 && l > 0; j--) {
+        bytes[l] = (byte) val;
+        val >>>= 8;
+        l--;
+      }
+      bytes[l] = (byte) val;
+      l--;
+    }
+    return bytes;
+  }
+
+  public byte[] getBytes(int[] keys) {
+
+    long[] words = get(keys);
+
+    return getBytesVal(words);
+  }
+
+  public long[] getKeyArray(byte[] key, int offset) {
+
+    int length = 8;
+    int ls = byteSize;
+    long[] words = new long[wsize];
+    for (int i = 0; i < words.length; i++) {
+      long l = 0;
+      ls -= 8;
+      int m = 0;
+      if (ls < 0) {
+        m = ls + length;
+        ls = 0;
+      } else {
+        m = ls + 8;
+      }
+      for (int j = ls; j < m; j++) {
+        l <<= 8;
+        l ^= key[j + offset] & 0xFF;
+      }
+      words[i] = l;
+    }
+
+    return getArray(words);
+
+  }
+
+  public long[] getKeyArray(byte[] key, int[] maskByteRanges) {
+
+    int length = 8;
+    int ls = byteSize;
+    long[] words = new long[wsize];
+    for (int i = 0; i < words.length; i++) {
+      long l = 0;
+      ls -= 8;
+      int m2 = 0;
+      if (ls < 0) {
+        m2 = ls + length;
+        ls = 0;
+      } else {
+        m2 = ls + 8;
+      }
+      if (maskByteRanges == null) {
+        for (int j = ls; j < m2; j++) {
+          l <<= 8;
+          l ^= key[j] & 0xFF;
+        }
+      } else {
+        for (int j = ls; j < m2; j++) {
+          l <<= 8;
+          if (maskByteRanges[j] != -1) {
+            l ^= key[maskByteRanges[j]] & 0xFF;
+          }
+        }
+      }
+      words[i] = l;
+    }
+
+    return getArray(words);
+
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (obj instanceof Bits) {
+      Bits other = (Bits) obj;
+      return Arrays.equals(lens, other.lens);
+    }
+    return false;
+  }
+
+  @Override public int hashCode() {
+    return Arrays.hashCode(lens);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
new file mode 100644
index 0000000..6f8dd7d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
@@ -0,0 +1,117 @@
+/*
+ * 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.mdkey;
+
+import org.apache.carbondata.core.keygenerator.KeyGenException;
+
+public class MultiDimKeyVarLengthGenerator extends AbstractKeyGenerator {
+
+  private static final long serialVersionUID = 9134778127271586515L;
+  /**
+   *
+   */
+  protected int[][] byteRangesForKeys;
+  private Bits bits;
+  private int startAndEndKeySizeWithPrimitives;
+
+  public MultiDimKeyVarLengthGenerator(int[] lens) {
+    bits = new Bits(lens);
+    byteRangesForKeys = new int[lens.length][];
+    int keys = lens.length;
+    for (int i = 0; i < keys; i++) {
+      byteRangesForKeys[i] = bits.getKeyByteOffsets(i);
+    }
+  }
+
+  @Override public byte[] generateKey(long[] keys) throws KeyGenException {
+
+    return bits.getBytes(keys);
+  }
+
+  @Override public byte[] generateKey(int[] keys) throws KeyGenException {
+
+    return bits.getBytes(keys);
+  }
+
+  @Override public long[] getKeyArray(byte[] key) {
+
+    return bits.getKeyArray(key, 0);
+  }
+
+  @Override public long[] getKeyArray(byte[] key, int offset) {
+
+    return bits.getKeyArray(key, offset);
+  }
+
+  @Override public long getKey(byte[] key, int index) {
+
+    return bits.getKeyArray(key, 0)[index];
+  }
+
+  public int getKeySizeInBytes() {
+    return bits.getByteSize();
+  }
+
+  @Override public long[] getSubKeyArray(byte[] key, int index, int size) {
+    if (index < 0 || size == 0) {
+      return null;
+    }
+    long[] keys = bits.getKeyArray(key, 0);
+    long[] rtn = new long[size];
+    System.arraycopy(keys, index, rtn, 0, size);
+    return rtn;
+  }
+
+  @Override public int[] getKeyByteOffsets(int index) {
+    return byteRangesForKeys[index];
+  }
+
+  @Override public int getDimCount() {
+
+    return bits.getDimCount();
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (obj instanceof MultiDimKeyVarLengthGenerator) {
+      MultiDimKeyVarLengthGenerator other = (MultiDimKeyVarLengthGenerator) obj;
+      return bits.equals(other.bits);
+    }
+
+    return false;
+  }
+
+  @Override public int hashCode() {
+    return bits.hashCode();
+  }
+
+  @Override public long[] getKeyArray(byte[] key, int[] maskedByteRanges) {
+    return bits.getKeyArray(key, maskedByteRanges);
+  }
+
+  @Override public int getStartAndEndKeySizeWithOnlyPrimitives() {
+    return startAndEndKeySizeWithPrimitives;
+  }
+
+  @Override
+  public void setStartAndEndKeySizeWithOnlyPrimitives(int startAndEndKeySizeWithPrimitives) {
+    this.startAndEndKeySizeWithPrimitives = startAndEndKeySizeWithPrimitives;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/NumberCompressor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/NumberCompressor.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/NumberCompressor.java
new file mode 100644
index 0000000..b83cf71
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/NumberCompressor.java
@@ -0,0 +1,220 @@
+/*
+ * 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.mdkey;
+
+/**
+ * It compresses the data as per max cardinality. It takes only the required bits for each key.
+ */
+public class NumberCompressor {
+
+  /**
+   * Bits MAX_LENGTH
+   */
+  private static final int MAX_LENGTH = 63;
+
+  private static final int LONG_LENGTH = 64;
+
+  private static final int BYTE_LENGTH = 8;
+
+  /**
+   * LONG_MAX.
+   */
+  private static final long LONG_MAX = 0x7fffffffffffffffL;
+
+  private byte bitsLength;
+
+  public NumberCompressor(int cardinaity) {
+    bitsLength = (byte) Long.toBinaryString(cardinaity).length();
+  }
+
+  public byte[] compress(int[] keys) {
+    int[] sizes = getWordsAndByteSize(keys.length);
+    long[] words = get(keys, sizes[0]);
+
+    return getByteValues(sizes, words);
+  }
+
+  private byte[] getByteValues(int[] sizes, long[] words) {
+    byte[] bytes = new byte[sizes[1]];
+
+    int l = sizes[1] - 1;
+    for (int i = 0; i < words.length; i++) {
+      long val = words[i];
+
+      for (int j = BYTE_LENGTH - 1; j > 0 && l > 0; j--) {
+        bytes[l] = (byte) val;
+        val >>>= 8;
+        l--;
+      }
+      bytes[l] = (byte) val;
+      l--;
+    }
+    return bytes;
+  }
+
+  protected long[] get(int[] keys, int wsize) {
+    long[] words = new long[wsize];
+    int ll = 0;
+    int index = 0;
+    int pos = 0;
+    int nextIndex = 0;
+    for (int i = keys.length - 1; i >= 0; i--) {
+
+      long val = keys[i];
+
+      index = ll >> 6;// divide by 64 to get the new word index
+      pos = ll & 0x3f;// to ignore sign bit and consider the remaining
+      //            val = val & controlBits;
+      long mask = (val << pos);
+      long word = words[index];
+      words[index] = (word | mask);
+      ll += bitsLength;
+
+      nextIndex = ll >> 6;// This is divide by 64
+
+      if (nextIndex != index) {
+        int consideredBits = bitsLength - ll & 0x3f;
+        if (consideredBits < bitsLength) // Check for spill over only if
+        // all the bits are not
+        // considered
+        {
+          // Check for spill over
+          mask = (val >> (bitsLength - ll & 0x3f));
+          words[nextIndex] |= mask;
+        }
+      }
+
+    }
+    return words;
+  }
+
+  protected long[] get(byte[] keys, int wsize) {
+    long[] words = new long[wsize];
+    int ll = 0;
+    long val = 0L;
+    for (int i = keys.length - 1; i >= 0; ) {
+
+      int size = i;
+      val = 0L;
+      for (int j = i + 1; j <= size; ) {
+        val <<= BYTE_LENGTH;
+        val ^= keys[j++] & 0xFF;
+        i--;
+      }
+      int index = ll >> 6;// divide by 64 to get the new word index
+      words[index] |= (val << (ll & 0x3f));
+      ll += bitsLength;
+
+      int nextIndex = ll >> 6;// This is divide by 64
+
+      if (nextIndex != index) {
+        int consideredBits = bitsLength - ll & 0x3f;
+        if (consideredBits < bitsLength) // Check for spill over only if
+        // all the bits are not
+        // considered
+        {
+          // Check for spill over
+          words[nextIndex] |= (val >> (bitsLength - ll & 0x3f));
+        }
+      }
+
+    }
+    return words;
+  }
+
+  public int[] unCompress(byte[] key) {
+    int ls = key.length;
+    int arrayLength = (ls * BYTE_LENGTH) / bitsLength;
+    long[] words = new long[getWordsSizeFromBytesSize(ls)];
+    unCompressVal(key, ls, words);
+    return getArray(words, arrayLength);
+  }
+
+  private void unCompressVal(byte[] key, int ls, long[] words) {
+    for (int i = 0; i < words.length; i++) {
+      long l = 0;
+      ls -= BYTE_LENGTH;
+      int m = 0;
+      if (ls < 0) {
+        m = ls + BYTE_LENGTH;
+        ls = 0;
+      } else {
+        m = ls + BYTE_LENGTH;
+      }
+      for (int j = ls; j < m; j++) {
+        l <<= BYTE_LENGTH;
+        l ^= key[j] & 0xFF;
+      }
+      words[i] = l;
+    }
+  }
+
+  private int[] getArray(long[] words, int arrayLength) {
+    int[] vals = new int[arrayLength];
+    int ll = 0;
+    long globalMask = LONG_MAX >>> (MAX_LENGTH - bitsLength);
+    for (int i = arrayLength - 1; i >= 0; i--) {
+
+      int index = ll >> 6;
+      int pos = ll & 0x3f;
+      long val = words[index];
+      long mask = globalMask << pos;
+      long value = (val & mask) >>> pos;
+      ll += bitsLength;
+
+      int nextIndex = ll >> 6;
+      if (nextIndex != index) {
+        pos = ll & 0x3f;
+        if (pos != 0) // Number of bits pending for current key is zero, no spill over
+        {
+          mask = (LONG_MAX >>> (MAX_LENGTH - pos));
+          val = words[nextIndex];
+          value = value | ((val & mask) << (bitsLength - pos));
+        }
+      }
+      vals[i] = (int) value;
+    }
+    return vals;
+  }
+
+  private int[] getWordsAndByteSize(int arrayLength) {
+    int length = arrayLength * bitsLength;
+    int wsize = length / LONG_LENGTH;
+    int byteSize = length / BYTE_LENGTH;
+
+    if (length % LONG_LENGTH != 0) {
+      wsize++;
+    }
+
+    if (length % BYTE_LENGTH != 0) {
+      byteSize++;
+    }
+    return new int[] { wsize, byteSize };
+  }
+
+  private int getWordsSizeFromBytesSize(int byteSize) {
+    int wsize = byteSize / BYTE_LENGTH;
+    if (byteSize % BYTE_LENGTH != 0) {
+      wsize++;
+    }
+    return wsize;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/load/BlockDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/load/BlockDetails.java b/core/src/main/java/org/apache/carbondata/core/load/BlockDetails.java
new file mode 100644
index 0000000..c3fd997
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/load/BlockDetails.java
@@ -0,0 +1,78 @@
+/*
+ * 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.load;
+
+import java.io.Serializable;
+
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
+
+/**
+ * blocks info
+ */
+public class BlockDetails implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 2293906691860002339L;
+  //block offset
+  private long blockOffset;
+  //block length
+  private long blockLength;
+  //file path which block belong to
+  private String filePath;
+  // locations where this block exists
+  private String[] locations;
+
+  public BlockDetails(String filePath, long blockOffset, long blockLength, String[] locations) {
+    this.filePath = filePath;
+    this.blockOffset = blockOffset;
+    this.blockLength = blockLength;
+    this.locations = locations;
+  }
+
+  public long getBlockOffset() {
+    return blockOffset;
+  }
+
+  public void setBlockOffset(long blockOffset) {
+    this.blockOffset = blockOffset;
+  }
+
+  public long getBlockLength() {
+    return blockLength;
+  }
+
+  public void setBlockLength(long blockLength) {
+    this.blockLength = blockLength;
+  }
+
+  public String getFilePath() {
+    return FileFactory.getUpdatedFilePath(filePath);
+  }
+
+  public void setFilePath(String filePath) {
+    this.filePath = filePath;
+  }
+
+  public String[] getLocations() {
+    return locations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/load/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/load/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/load/LoadMetadataDetails.java
new file mode 100644
index 0000000..7c58ae7
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/load/LoadMetadataDetails.java
@@ -0,0 +1,226 @@
+/*
+ * 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.load;
+
+import java.io.Serializable;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+public class LoadMetadataDetails implements Serializable {
+
+  private static final long serialVersionUID = 1106104914918491724L;
+  private String timestamp;
+  private String loadStatus;
+  private String loadName;
+  private String partitionCount;
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LoadMetadataDetails.class.getName());
+
+  private static final SimpleDateFormat parser =
+      new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
+  /**
+   * Segment modification or deletion time stamp
+   */
+  private String modificationOrdeletionTimesStamp;
+  private String loadStartTime;
+
+  private String mergedLoadName;
+  /**
+   * visibility is used to determine whether to the load is visible or not.
+   */
+  private String visibility = "true";
+
+  /**
+   * To know if the segment is a major compacted segment or not.
+   */
+  private String majorCompacted;
+
+  public String getPartitionCount() {
+    return partitionCount;
+  }
+
+  public void setPartitionCount(String partitionCount) {
+    this.partitionCount = partitionCount;
+  }
+
+  public String getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(String timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  public String getLoadStatus() {
+    return loadStatus;
+  }
+
+  public void setLoadStatus(String loadStatus) {
+    this.loadStatus = loadStatus;
+  }
+
+  public String getLoadName() {
+    return loadName;
+  }
+
+  public void setLoadName(String loadName) {
+    this.loadName = loadName;
+  }
+
+  /**
+   * @return the modificationOrdeletionTimesStamp
+   */
+  public String getModificationOrdeletionTimesStamp() {
+    return modificationOrdeletionTimesStamp;
+  }
+
+  /**
+   * @param modificationOrdeletionTimesStamp the modificationOrdeletionTimesStamp to set
+   */
+  public void setModificationOrdeletionTimesStamp(String modificationOrdeletionTimesStamp) {
+    this.modificationOrdeletionTimesStamp = modificationOrdeletionTimesStamp;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#hashCode()
+   */
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((loadName == null) ? 0 : loadName.hashCode());
+    return result;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#equals(java.lang.Object)
+   */
+  @Override public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+
+    }
+    if (!(obj instanceof LoadMetadataDetails)) {
+      return false;
+    }
+    LoadMetadataDetails other = (LoadMetadataDetails) obj;
+    if (loadName == null) {
+      if (other.loadName != null) {
+        return false;
+      }
+    } else if (!loadName.equals(other.loadName)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * @return the startLoadTime
+   */
+  public String getLoadStartTime() {
+    return loadStartTime;
+  }
+
+  /**
+   * return loadStartTime
+   * @return
+   */
+  public long getLoadStartTimeAsLong() {
+    return getTimeStamp(loadStartTime);
+  }
+
+  /**
+   * returns load start time as long value
+   * @param loadStartTime
+   * @return
+   */
+  private Long getTimeStamp(String loadStartTime) {
+    if (loadStartTime.isEmpty()) {
+      return null;
+    }
+
+    Date dateToStr = null;
+    try {
+      dateToStr = parser.parse(loadStartTime);
+      return dateToStr.getTime() * 1000;
+    } catch (ParseException e) {
+      LOGGER.error("Cannot convert" + loadStartTime + " to Time/Long type value" + e.getMessage());
+      return null;
+    }
+  }
+  /**
+   * @param loadStartTime
+   */
+  public void setLoadStartTime(String loadStartTime) {
+    this.loadStartTime = loadStartTime;
+  }
+
+  /**
+   * @return the mergedLoadName
+   */
+  public String getMergedLoadName() {
+    return mergedLoadName;
+  }
+
+  /**
+   * @param mergedLoadName the mergedLoadName to set
+   */
+  public void setMergedLoadName(String mergedLoadName) {
+    this.mergedLoadName = mergedLoadName;
+  }
+
+  /**
+   * @return the visibility
+   */
+  public String getVisibility() {
+    return visibility;
+  }
+
+  /**
+   * @param visibility the visibility to set
+   */
+  public void setVisibility(String visibility) {
+    this.visibility = visibility;
+  }
+
+  /**
+   * Return true if it is a major compacted segment.
+   * @return
+   */
+  public String isMajorCompacted() {
+    return majorCompacted;
+  }
+
+  /**
+   * Set true if it is a major compacted segment.
+   * @param majorCompacted
+   */
+  public void setMajorCompacted(String majorCompacted) {
+    this.majorCompacted = majorCompacted;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfo.java
new file mode 100644
index 0000000..5cfa11b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfo.java
@@ -0,0 +1,202 @@
+/*
+ * 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.metadata;
+
+public class BlockletInfo {
+  /**
+   * fileName.
+   */
+  private String fileName;
+
+  /**
+   * keyOffset.
+   */
+  private long keyOffset;
+
+  /**
+   * measureOffset.
+   */
+  private long[] measureOffset;
+
+  /**
+   * measureLength.
+   */
+  private int[] measureLength;
+
+  /**
+   * keyLength.
+   */
+  private int keyLength;
+
+  /**
+   * numberOfKeys.
+   */
+  private int numberOfKeys;
+
+  /**
+   * startKey.
+   */
+  private byte[] startKey;
+
+  /**
+   * endKey.
+   */
+  private byte[] endKey;
+
+  /**
+   * getFileName().
+   *
+   * @return String.
+   */
+  public String getFileName() {
+    return fileName;
+  }
+
+  /**
+   * setFileName.
+   */
+  public void setFileName(String fileName) {
+    this.fileName = fileName;
+  }
+
+  /**
+   * getKeyOffset.
+   *
+   * @return long.
+   */
+  public long getKeyOffset() {
+    return keyOffset;
+  }
+
+  /**
+   * setKeyOffset.
+   *
+   * @param keyOffset
+   */
+  public void setKeyOffset(long keyOffset) {
+    this.keyOffset = keyOffset;
+  }
+
+  /**
+   * getMeasureLength
+   *
+   * @return int[].
+   */
+  public int[] getMeasureLength() {
+    return measureLength;
+  }
+
+  /**
+   * setMeasureLength.
+   *
+   * @param measureLength
+   */
+  public void setMeasureLength(int[] measureLength) {
+    this.measureLength = measureLength;
+  }
+
+  /**
+   * getKeyLength.
+   *
+   * @return
+   */
+  public int getKeyLength() {
+    return keyLength;
+  }
+
+  /**
+   * setKeyLength.
+   */
+  public void setKeyLength(int keyLength) {
+    this.keyLength = keyLength;
+  }
+
+  /**
+   * getMeasureOffset.
+   *
+   * @return long[].
+   */
+  public long[] getMeasureOffset() {
+    return measureOffset;
+  }
+
+  /**
+   * setMeasureOffset.
+   *
+   * @param measureOffset
+   */
+  public void setMeasureOffset(long[] measureOffset) {
+    this.measureOffset = measureOffset;
+  }
+
+  /**
+   * getNumberOfKeys()
+   *
+   * @return int.
+   */
+  public int getNumberOfKeys() {
+    return numberOfKeys;
+  }
+
+  /**
+   * setNumberOfKeys.
+   *
+   * @param numberOfKeys
+   */
+  public void setNumberOfKeys(int numberOfKeys) {
+    this.numberOfKeys = numberOfKeys;
+  }
+
+  /**
+   * getStartKey().
+   *
+   * @return byte[].
+   */
+  public byte[] getStartKey() {
+    return startKey;
+  }
+
+  /**
+   * setStartKey.
+   *
+   * @param startKey
+   */
+  public void setStartKey(byte[] startKey) {
+    this.startKey = startKey;
+  }
+
+  /**
+   * getEndKey().
+   *
+   * @return byte[].
+   */
+  public byte[] getEndKey() {
+    return endKey;
+  }
+
+  /**
+   * setEndKey.
+   *
+   * @param endKey
+   */
+  public void setEndKey(byte[] endKey) {
+    this.endKey = endKey;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfoColumnar.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfoColumnar.java b/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfoColumnar.java
new file mode 100644
index 0000000..54e5361
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/BlockletInfoColumnar.java
@@ -0,0 +1,405 @@
+/*
+ * 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.metadata;
+
+import java.util.BitSet;
+
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
+
+public class BlockletInfoColumnar {
+  /**
+   * fileName.
+   */
+  private String fileName;
+
+  /**
+   * measureOffset.
+   */
+  private long[] measureOffset;
+
+  /**
+   * measureLength.
+   */
+  private int[] measureLength;
+
+  /**
+   * numberOfKeys.
+   */
+  private int numberOfKeys;
+
+  /**
+   * startKey.
+   */
+  private byte[] startKey;
+
+  /**
+   * endKey.
+   */
+  private byte[] endKey;
+
+  /**
+   * keyOffSets
+   */
+  private long[] keyOffSets;
+
+  /**
+   * keyLengths
+   */
+  private int[] keyLengths;
+
+  /**
+   * isSortedKeyColumn
+   */
+  private boolean[] isSortedKeyColumn;
+
+  /**
+   * keyBlockIndexOffSets
+   */
+  private long[] keyBlockIndexOffSets;
+
+  /**
+   * keyBlockIndexLength
+   */
+  private int[] keyBlockIndexLength;
+
+  /**
+   * dataIndexMap
+   */
+  private int[] dataIndexMapLength;
+
+  /**
+   * dataIndexMap
+   */
+  private long[] dataIndexMapOffsets;
+
+  private boolean[] aggKeyBlock;
+  /**
+   * blockletMetaSize
+   */
+  private int blockletMetaSize;
+
+  private NumberCompressor[] keyBlockUnCompressor;
+
+  private ValueCompressionModel compressionModel;
+
+  /**
+   * column min array
+   */
+  private byte[][] columnMaxData;
+
+  /**
+   * column max array
+   */
+  private byte[][] columnMinData;
+
+  /**
+   * true if given index is colgroup block
+   */
+  private boolean[] colGrpBlock;
+
+  /**
+   * bit set which will holds the measure
+   * indexes which are null
+   */
+  private BitSet[] measureNullValueIndex;
+
+  /**
+   * getFileName().
+   *
+   * @return String.
+   */
+  public String getFileName() {
+    return fileName;
+  }
+
+  /**
+   * setFileName.
+   */
+  public void setFileName(String fileName) {
+    this.fileName = fileName;
+  }
+
+  /**
+   * getMeasureLength
+   *
+   * @return int[].
+   */
+  public int[] getMeasureLength() {
+    return measureLength;
+  }
+
+  /**
+   * setMeasureLength.
+   *
+   * @param measureLength
+   */
+  public void setMeasureLength(int[] measureLength) {
+    this.measureLength = measureLength;
+  }
+
+  /**
+   * getMeasureOffset.
+   *
+   * @return long[].
+   */
+  public long[] getMeasureOffset() {
+    return measureOffset;
+  }
+
+  /**
+   * setMeasureOffset.
+   *
+   * @param measureOffset
+   */
+  public void setMeasureOffset(long[] measureOffset) {
+    this.measureOffset = measureOffset;
+  }
+
+  /**
+   * getStartKey().
+   *
+   * @return byte[].
+   */
+  public byte[] getStartKey() {
+    return startKey;
+  }
+
+  /**
+   * setStartKey.
+   *
+   * @param startKey
+   */
+  public void setStartKey(byte[] startKey) {
+    this.startKey = startKey;
+  }
+
+  /**
+   * getEndKey().
+   *
+   * @return byte[].
+   */
+  public byte[] getEndKey() {
+    return endKey;
+  }
+
+  /**
+   * setEndKey.
+   *
+   * @param endKey
+   */
+  public void setEndKey(byte[] endKey) {
+    this.endKey = endKey;
+  }
+
+  /**
+   * @return the keyOffSets
+   */
+  public long[] getKeyOffSets() {
+    return keyOffSets;
+  }
+
+  /**
+   * @param keyOffSets the keyOffSets to set
+   */
+  public void setKeyOffSets(long[] keyOffSets) {
+    this.keyOffSets = keyOffSets;
+  }
+
+  /**
+   * @return the keyLengths
+   */
+  public int[] getKeyLengths() {
+    return keyLengths;
+  }
+
+  //TODO SIMIAN
+
+  /**
+   * @param keyLengths the keyLengths to set
+   */
+  public void setKeyLengths(int[] keyLengths) {
+    this.keyLengths = keyLengths;
+  }
+
+  /**
+   * getNumberOfKeys()
+   *
+   * @return int.
+   */
+  public int getNumberOfKeys() {
+    return numberOfKeys;
+  }
+
+  /**
+   * setNumberOfKeys.
+   *
+   * @param numberOfKeys
+   */
+  public void setNumberOfKeys(int numberOfKeys) {
+    this.numberOfKeys = numberOfKeys;
+  }
+
+  /**
+   * @return the isSortedKeyColumn
+   */
+  public boolean[] getIsSortedKeyColumn() {
+    return isSortedKeyColumn;
+  }
+
+  /**
+   * @param isSortedKeyColumn the isSortedKeyColumn to set
+   */
+  public void setIsSortedKeyColumn(boolean[] isSortedKeyColumn) {
+    this.isSortedKeyColumn = isSortedKeyColumn;
+  }
+
+  /**
+   * @return the keyBlockIndexOffSets
+   */
+  public long[] getKeyBlockIndexOffSets() {
+    return keyBlockIndexOffSets;
+  }
+
+  /**
+   * @param keyBlockIndexOffSets the keyBlockIndexOffSets to set
+   */
+  public void setKeyBlockIndexOffSets(long[] keyBlockIndexOffSets) {
+    this.keyBlockIndexOffSets = keyBlockIndexOffSets;
+  }
+
+  /**
+   * @return the keyBlockIndexLength
+   */
+  public int[] getKeyBlockIndexLength() {
+    return keyBlockIndexLength;
+  }
+
+  /**
+   * @param keyBlockIndexLength the keyBlockIndexLength to set
+   */
+  public void setKeyBlockIndexLength(int[] keyBlockIndexLength) {
+    this.keyBlockIndexLength = keyBlockIndexLength;
+  }
+
+  /**
+   * @return the blockletMetaSize
+   */
+  public int getBlockletMetaSize() {
+    return blockletMetaSize;
+  }
+
+  /**
+   * @param blockletMetaSize the blockletMetaSize to set
+   */
+  public void setBlockletMetaSize(int blockletMetaSize) {
+    this.blockletMetaSize = blockletMetaSize;
+  }
+
+  /**
+   * @return the dataIndexMapLenght
+   */
+  public int[] getDataIndexMapLength() {
+    return dataIndexMapLength;
+  }
+
+  public void setDataIndexMapLength(int[] dataIndexMapLength) {
+    this.dataIndexMapLength = dataIndexMapLength;
+  }
+
+  /**
+   * @return the dataIndexMapOffsets
+   */
+  public long[] getDataIndexMapOffsets() {
+    return dataIndexMapOffsets;
+  }
+
+  public void setDataIndexMapOffsets(long[] dataIndexMapOffsets) {
+    this.dataIndexMapOffsets = dataIndexMapOffsets;
+  }
+
+  public boolean[] getAggKeyBlock() {
+    return aggKeyBlock;
+  }
+
+  public void setAggKeyBlock(boolean[] aggKeyBlock) {
+    this.aggKeyBlock = aggKeyBlock;
+  }
+
+  public NumberCompressor[] getKeyBlockUnCompressor() {
+    return keyBlockUnCompressor;
+  }
+
+  public void setKeyBlockUnCompressor(NumberCompressor[] keyBlockUnCompressor) {
+    this.keyBlockUnCompressor = keyBlockUnCompressor;
+  }
+
+  public byte[][] getColumnMaxData() {
+    return this.columnMaxData;
+  }
+
+  public void setColumnMaxData(byte[][] columnMaxData) {
+    this.columnMaxData = columnMaxData;
+  }
+
+  public byte[][] getColumnMinData() {
+    return this.columnMinData;
+  }
+
+  public void setColumnMinData(byte[][] columnMinData) {
+    this.columnMinData = columnMinData;
+  }
+
+  public ValueCompressionModel getCompressionModel() {
+    return compressionModel;
+  }
+
+  public void setCompressionModel(ValueCompressionModel compressionModel) {
+    this.compressionModel = compressionModel;
+  }
+
+  /**
+   * @return
+   */
+  public boolean[] getColGrpBlocks() {
+    return this.colGrpBlock;
+  }
+
+  /**
+   * @param colGrpBlock
+   */
+  public void setColGrpBlocks(boolean[] colGrpBlock) {
+    this.colGrpBlock = colGrpBlock;
+  }
+
+  /**
+   * @return the measureNullValueIndex
+   */
+  public BitSet[] getMeasureNullValueIndex() {
+    return measureNullValueIndex;
+  }
+
+  /**
+   * @param measureNullValueIndex the measureNullValueIndex to set
+   */
+  public void setMeasureNullValueIndex(BitSet[] measureNullValueIndex) {
+    this.measureNullValueIndex = measureNullValueIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/metadata/ValueEncoderMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/ValueEncoderMeta.java b/core/src/main/java/org/apache/carbondata/core/metadata/ValueEncoderMeta.java
new file mode 100644
index 0000000..471158d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/ValueEncoderMeta.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.metadata;
+
+import java.io.Serializable;
+
+/**
+ * It holds Value compression metadata for one data column
+ */
+public class ValueEncoderMeta implements Serializable {
+
+  /**
+   * maxValue
+   */
+  private Object maxValue;
+  /**
+   * minValue.
+   */
+  private Object minValue;
+
+  /**
+   * uniqueValue
+   */
+  private Object uniqueValue;
+  /**
+   * decimal.
+   */
+  private int decimal;
+
+  /**
+   * aggType
+   */
+  private char type;
+
+  /**
+   * dataTypeSelected
+   */
+  private byte dataTypeSelected;
+
+  public Object getMaxValue() {
+    return maxValue;
+  }
+
+  public void setMaxValue(Object maxValue) {
+    this.maxValue = maxValue;
+  }
+
+  public Object getMinValue() {
+    return minValue;
+  }
+
+  public void setMinValue(Object minValue) {
+    this.minValue = minValue;
+  }
+
+  public Object getUniqueValue() {
+    return uniqueValue;
+  }
+
+  public void setUniqueValue(Object uniqueValue) {
+    this.uniqueValue = uniqueValue;
+  }
+
+  public int getDecimal() {
+    return decimal;
+  }
+
+  public void setDecimal(int decimal) {
+    this.decimal = decimal;
+  }
+
+  public char getType() {
+    return type;
+  }
+
+  public void setType(char type) {
+    this.type = type;
+  }
+
+  public byte getDataTypeSelected() {
+    return dataTypeSelected;
+  }
+
+  public void setDataTypeSelected(byte dataTypeSelected) {
+    this.dataTypeSelected = dataTypeSelected;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryColumnMetaChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryColumnMetaChunk.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryColumnMetaChunk.java
new file mode 100644
index 0000000..3c4658e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryColumnMetaChunk.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.reader;
+
+/**
+ * A wrapper class for thrift class ColumnDictionaryChunkMeta which will
+ * contain data like min and max surrogate key, start and end offset, chunk count
+ */
+public class CarbonDictionaryColumnMetaChunk {
+
+  /**
+   * Minimum value surrogate key for a segment
+   */
+  private int min_surrogate_key;
+
+  /**
+   * Max value of surrogate key for a segment
+   */
+  private int max_surrogate_key;
+
+  /**
+   * start offset of dictionary chunk in dictionary file for a segment
+   */
+  private long start_offset;
+
+  /**
+   * end offset of dictionary chunk in dictionary file for a segment
+   */
+  private long end_offset;
+
+  /**
+   * count of dictionary chunks for a segment
+   */
+  private int chunk_count;
+
+  /**
+   * constructor
+   *
+   * @param min_surrogate_key Minimum value surrogate key for a segment
+   * @param max_surrogate_key Maximum value surrogate key for a segment
+   * @param start_offset      start offset of dictionary chunk in dictionary file for a segment
+   * @param end_offset        end offset of dictionary chunk in dictionary file for a segment
+   * @param chunk_count       count of dictionary chunks for a segment
+   */
+  public CarbonDictionaryColumnMetaChunk(int min_surrogate_key, int max_surrogate_key,
+      long start_offset, long end_offset, int chunk_count) {
+    this.min_surrogate_key = min_surrogate_key;
+    this.max_surrogate_key = max_surrogate_key;
+    this.start_offset = start_offset;
+    this.end_offset = end_offset;
+    this.chunk_count = chunk_count;
+  }
+
+  /**
+   * @return min surrogate key
+   */
+  public int getMin_surrogate_key() {
+    return min_surrogate_key;
+  }
+
+  /**
+   * @return max surrogate key
+   */
+  public int getMax_surrogate_key() {
+    return max_surrogate_key;
+  }
+
+  /**
+   * @return start offset
+   */
+  public long getStart_offset() {
+    return start_offset;
+  }
+
+  /**
+   * @return end offset
+   */
+  public long getEnd_offset() {
+    return end_offset;
+  }
+
+  /**
+   * @return chunk count
+   */
+  public int getChunk_count() {
+    return chunk_count;
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReader.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReader.java
new file mode 100644
index 0000000..09a85d3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReader.java
@@ -0,0 +1,58 @@
+/*
+ * 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.reader;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * dictionary metadata reader interface which declares methods to read dictionary metadata
+ */
+public interface CarbonDictionaryMetadataReader extends Closeable {
+
+  /**
+   * This method will be used to read complete metadata file.
+   * Applicable scenarios:
+   * 1. Query execution. Whenever a query is executed then to read the dictionary file
+   * and define the query scope first dictionary metadata has to be read first.
+   * 2. If dictionary file is read using start and end offset then using this meta list
+   * we can count the total number of dictionary chunks present between the 2 offsets
+   *
+   * @return list of all dictionary meta chunks which contains information for each segment
+   * @throws IOException if an I/O error occurs
+   */
+  List<CarbonDictionaryColumnMetaChunk> read() throws IOException;
+
+  /**
+   * This method will be used to read only the last entry of dictionary meta chunk.
+   * Applicable scenarios :
+   * 1. Global dictionary generation for incremental load. In this case only the
+   * last dictionary chunk meta entry has to be read to calculate min, max surrogate
+   * key and start and end offset for the new dictionary chunk.
+   * 2. Truncate operation. While writing dictionary file in case of incremental load
+   * dictionary file needs to be validated for any inconsistency. Here end offset of last
+   * dictionary chunk meta is validated with file size.
+   *
+   * @return last segment entry for dictionary chunk
+   * @throws IOException if an I/O error occurs
+   */
+  CarbonDictionaryColumnMetaChunk readLastEntryOfDictionaryMetaChunk() throws IOException;
+}