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

[01/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Repository: carbondata
Updated Branches:
  refs/heads/master 75e0bd419 -> 956833e55


http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 1addd03..ef89771 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -35,7 +35,7 @@ import org.apache.spark.sql.types._
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnIdentifier}
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.util.DataTypeUtil
@@ -165,34 +165,34 @@ case class CarbonDictionaryDecoder(
              """.stripMargin
 
             val caseCode = getDictionaryColumnIds(index)._3.getDataType match {
-              case DataType.INT =>
+              case CarbonDataTypes.INT =>
                 s"""
                    |int $value = Integer.parseInt(new String($valueIntern,
                    |org.apache.carbondata.core.constants.CarbonCommonConstants
                    |.DEFAULT_CHARSET_CLASS));
                  """.stripMargin
-              case DataType.SHORT =>
+              case CarbonDataTypes.SHORT =>
                 s"""
                    |short $value =
                    |Short.parseShort(new String($valueIntern,
                    |org.apache.carbondata.core.constants.CarbonCommonConstants
                    |.DEFAULT_CHARSET_CLASS));
                  """.stripMargin
-              case DataType.DOUBLE =>
+              case CarbonDataTypes.DOUBLE =>
                 s"""
                    |double $value =
                    |Double.parseDouble(new String($valueIntern,
                    |org.apache.carbondata.core.constants.CarbonCommonConstants
                    |.DEFAULT_CHARSET_CLASS));
                  """.stripMargin
-              case DataType.LONG =>
+              case CarbonDataTypes.LONG =>
                 s"""
                    |long $value =
                    |Long.parseLong(new String($valueIntern,
                    |org.apache.carbondata.core.constants.CarbonCommonConstants
                    |.DEFAULT_CHARSET_CLASS));
                  """.stripMargin
-              case DataType.DECIMAL =>
+              case CarbonDataTypes.DECIMAL =>
                 s"""
                    |org.apache.spark.sql.types.Decimal $value =
                    |Decimal.apply(new java.math.BigDecimal(
@@ -382,13 +382,13 @@ object CarbonDictionaryDecoder {
   def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
       relation: CarbonRelation): types.DataType = {
     carbonDimension.getDataType match {
-      case DataType.STRING => StringType
-      case DataType.SHORT => ShortType
-      case DataType.INT => IntegerType
-      case DataType.LONG => LongType
-      case DataType.DOUBLE => DoubleType
-      case DataType.BOOLEAN => BooleanType
-      case DataType.DECIMAL =>
+      case CarbonDataTypes.STRING => StringType
+      case CarbonDataTypes.SHORT => ShortType
+      case CarbonDataTypes.INT => IntegerType
+      case CarbonDataTypes.LONG => LongType
+      case CarbonDataTypes.DOUBLE => DoubleType
+      case CarbonDataTypes.BOOLEAN => BooleanType
+      case CarbonDataTypes.DECIMAL =>
         val scale: Int = carbonDimension.getColumnSchema.getScale
         val precision: Int = carbonDimension.getColumnSchema.getPrecision
         if (scale == 0 && precision == 0) {
@@ -396,12 +396,12 @@ object CarbonDictionaryDecoder {
         } else {
           DecimalType(precision, scale)
         }
-      case DataType.TIMESTAMP => TimestampType
-      case DataType.DATE => DateType
-      case DataType.STRUCT =>
+      case CarbonDataTypes.TIMESTAMP => TimestampType
+      case CarbonDataTypes.DATE => DateType
+      case CarbonDataTypes.STRUCT =>
         CarbonMetastoreTypes
           .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
-      case DataType.ARRAY =>
+      case CarbonDataTypes.ARRAY =>
         CarbonMetastoreTypes
           .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
     }
@@ -467,13 +467,13 @@ class CarbonDecoderRDD(
   def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
       relation: CarbonRelation): types.DataType = {
     carbonDimension.getDataType match {
-      case DataType.STRING => StringType
-      case DataType.SHORT => ShortType
-      case DataType.INT => IntegerType
-      case DataType.LONG => LongType
-      case DataType.DOUBLE => DoubleType
-      case DataType.BOOLEAN => BooleanType
-      case DataType.DECIMAL =>
+      case CarbonDataTypes.STRING => StringType
+      case CarbonDataTypes.SHORT => ShortType
+      case CarbonDataTypes.INT => IntegerType
+      case CarbonDataTypes.LONG => LongType
+      case CarbonDataTypes.DOUBLE => DoubleType
+      case CarbonDataTypes.BOOLEAN => BooleanType
+      case CarbonDataTypes.DECIMAL =>
         val scale: Int = carbonDimension.getColumnSchema.getScale
         val precision: Int = carbonDimension.getColumnSchema.getPrecision
         if (scale == 0 && precision == 0) {
@@ -481,12 +481,12 @@ class CarbonDecoderRDD(
         } else {
           DecimalType(precision, scale)
         }
-      case DataType.TIMESTAMP => TimestampType
-      case DataType.DATE => DateType
-      case DataType.STRUCT =>
+      case CarbonDataTypes.TIMESTAMP => TimestampType
+      case CarbonDataTypes.DATE => DateType
+      case CarbonDataTypes.STRUCT =>
         CarbonMetastoreTypes
           .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
-      case DataType.ARRAY =>
+      case CarbonDataTypes.ARRAY =>
         CarbonMetastoreTypes
           .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index f245df6..6bac0da 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -16,6 +16,7 @@
  */
 package org.apache.spark.sql.hive
 
+import java.util
 import java.util.LinkedHashSet
 
 import scala.Array.canBuildFrom
@@ -28,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Stati
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.datatype.DataType.DECIMAL
+import org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.path.CarbonStorePath
@@ -45,7 +46,7 @@ case class CarbonRelation(
   extends LeafNode with MultiInstanceRelation {
 
   def recursiveMethod(dimName: String, childDim: CarbonDimension): String = {
-    childDim.getDataType.toString.toLowerCase match {
+    childDim.getDataType.getName.toLowerCase match {
       case "array" => s"${
         childDim.getColName.substring(dimName.length + 1)
       }:array<${ getArrayChildren(childDim.getColName) }>"
@@ -58,7 +59,7 @@ case class CarbonRelation(
 
   def getArrayChildren(dimName: String): String = {
     metaData.carbonTable.getChildren(dimName).asScala.map(childDim => {
-      childDim.getDataType.toString.toLowerCase match {
+      childDim.getDataType.getName.toLowerCase match {
         case "array" => s"array<${ getArrayChildren(childDim.getColName) }>"
         case "struct" => s"struct<${ getStructChildren(childDim.getColName) }>"
         case dType => addDecimalScaleAndPrecision(childDim, dType)
@@ -68,7 +69,7 @@ case class CarbonRelation(
 
   def getStructChildren(dimName: String): String = {
     metaData.carbonTable.getChildren(dimName).asScala.map(childDim => {
-      childDim.getDataType.toString.toLowerCase match {
+      childDim.getDataType.getName.toLowerCase match {
         case "array" => s"${
           childDim.getColName.substring(dimName.length + 1)
         }:array<${ getArrayChildren(childDim.getColName) }>"
@@ -95,8 +96,7 @@ case class CarbonRelation(
     sett.asScala.toSeq.map(dim => {
       val dimval = metaData.carbonTable
         .getDimensionByName(metaData.carbonTable.getFactTableName, dim.getColName)
-      val output: DataType = dimval.getDataType
-        .toString.toLowerCase match {
+      val output: DataType = dimval.getDataType.getName.toLowerCase match {
         case "array" =>
           CarbonMetastoreTypes.toDataType(s"array<${ getArrayChildren(dim.getColName) }>")
         case "struct" =>
@@ -118,14 +118,17 @@ case class CarbonRelation(
     new LinkedHashSet(
       tableMeta.carbonTable.
         getMeasureByTableName(tableMeta.carbonTable.getFactTableName).
-        asScala.asJava).asScala.toSeq
-      .map(x => AttributeReference(x.getColName, CarbonMetastoreTypes.toDataType(
-        metaData.carbonTable.getMeasureByName(factTable, x.getColName).getDataType.toString
-          .toLowerCase match {
-          case "decimal" => "decimal(" + x.getPrecision + "," + x.getScale + ")"
-          case others => others
-        }),
-        nullable = true)())
+        asScala.asJava).asScala.toSeq.map { x =>
+      val metastoreType = metaData.carbonTable.getMeasureByName(factTable, x.getColName)
+        .getDataType.getName.toLowerCase match {
+        case "decimal" => "decimal(" + x.getPrecision + "," + x.getScale + ")"
+        case others => others
+      }
+      AttributeReference(
+        x.getColName,
+        CarbonMetastoreTypes.toDataType(metastoreType),
+        nullable = true)()
+    }
   }
 
   override val output = {
@@ -134,7 +137,7 @@ case class CarbonRelation(
     // convert each column to Attribute
     columns.filter(!_.isInvisible).map { column =>
       if (column.isDimension()) {
-        val output: DataType = column.getDataType.toString.toLowerCase match {
+        val output: DataType = column.getDataType.getName.toLowerCase match {
           case "array" =>
             CarbonMetastoreTypes.toDataType(s"array<${getArrayChildren(column.getColName)}>")
           case "struct" =>
@@ -147,8 +150,7 @@ case class CarbonRelation(
           qualifier = Option(tableName + "." + column.getColName))
       } else {
         val output = CarbonMetastoreTypes.toDataType {
-          column.getDataType.toString
-            .toLowerCase match {
+          column.getDataType.getName.toLowerCase match {
             case "decimal" => "decimal(" + column.getColumnSchema.getPrecision + "," + column
               .getColumnSchema.getScale + ")"
             case others => others

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index 73c9760..dbc807d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.types._
 import org.apache.spark.sql.CarbonContainsWith
 import org.apache.spark.sql.CarbonEndsWith
 
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
@@ -134,9 +134,9 @@ object CarbonFilters {
     def getCarbonLiteralExpression(name: String, value: Any): CarbonExpression = {
       val dataTypeOfAttribute = CarbonScalaUtil.convertSparkToCarbonDataType(dataTypeOf(name))
       val dataType = if (Option(value).isDefined
-                         && dataTypeOfAttribute == DataType.STRING
+                         && dataTypeOfAttribute == CarbonDataTypes.STRING
                          && value.isInstanceOf[Double]) {
-        DataType.DOUBLE
+        CarbonDataTypes.DOUBLE
       } else {
         dataTypeOfAttribute
       }
@@ -396,24 +396,6 @@ object CarbonFilters {
     }
   }
 
-  private def getActualCarbonDataType(column: String, carbonTable: CarbonTable) = {
-    var carbonColumn: CarbonColumn =
-      carbonTable.getDimensionByName(carbonTable.getFactTableName, column)
-    val dataType = if (carbonColumn != null) {
-      carbonColumn.getDataType
-    } else {
-      carbonColumn = carbonTable.getMeasureByName(carbonTable.getFactTableName, column)
-      carbonColumn.getDataType match {
-        case DataType.INT => DataType.INT
-        case DataType.SHORT => DataType.SHORT
-        case DataType.LONG => DataType.LONG
-        case DataType.DECIMAL => DataType.DECIMAL
-        case _ => DataType.DOUBLE
-      }
-    }
-    CarbonScalaUtil.convertCarbonToSparkDataType(dataType)
-  }
-
   // Convert scala list to java list, Cannot use scalaList.asJava as while deserializing it is
   // not able find the classes inside scala list and gives ClassNotFoundException.
   private def convertToJavaList(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
index e90fd4a..a340ab1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
@@ -176,8 +176,8 @@ public class StructDataType implements GenericDataType<StructObject> {
   }
 
   /*
-       * parse bytearray and bit pack
-       */
+   * parse bytearray and bit pack
+   */
   @Override
   public void parseAndBitPack(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream,
       KeyGenerator[] generator) throws IOException, KeyGenException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index 2efbe26..193d3e6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -25,6 +25,8 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.dictionary.client.DictionaryClient;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -108,35 +110,37 @@ public class FieldEncoderFactory {
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
       String storePath, Map<Object, Integer> localCache) {
-    switch (carbonColumn.getDataType()) {
-      case ARRAY:
-        List<CarbonDimension> listOfChildDimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create array parser with complex delimiter
-        ArrayDataType arrayDataType =
-            new ArrayDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
-        for (CarbonDimension dimension : listOfChildDimensions) {
-          arrayDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
-              carbonTableIdentifier, client, useOnePass, storePath, localCache));
-        }
-        return arrayDataType;
-      case STRUCT:
-        List<CarbonDimension> dimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create struct parser with complex delimiter
-        StructDataType structDataType =
-            new StructDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
-        for (CarbonDimension dimension : dimensions) {
-          structDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
-              carbonTableIdentifier, client, useOnePass, storePath, localCache));
-        }
-        return structDataType;
-      case MAP:
-        throw new UnsupportedOperationException("Complex type Map is not supported yet");
-      default:
-        return new PrimitiveDataType(carbonColumn.getColName(), parentName,
-            carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache,
-            carbonTableIdentifier, client, useOnePass, storePath, localCache);
+    DataType dataType = carbonColumn.getDataType();
+    if (dataType == DataTypes.ARRAY) {
+      List<CarbonDimension> listOfChildDimensions =
+          ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+      // Create array parser with complex delimiter
+      ArrayDataType arrayDataType =
+          new ArrayDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
+      for (CarbonDimension dimension : listOfChildDimensions) {
+        arrayDataType.addChildren(
+            createComplexType(dimension, carbonColumn.getColName(), cache, carbonTableIdentifier,
+                client, useOnePass, storePath, localCache));
+      }
+      return arrayDataType;
+    } else if (dataType == DataTypes.STRUCT) {
+      List<CarbonDimension> dimensions =
+          ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+      // Create struct parser with complex delimiter
+      StructDataType structDataType =
+          new StructDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
+      for (CarbonDimension dimension : dimensions) {
+        structDataType.addChildren(
+            createComplexType(dimension, carbonColumn.getColName(), cache, carbonTableIdentifier,
+                client, useOnePass, storePath, localCache));
+      }
+      return structDataType;
+    } else if (dataType == DataTypes.MAP) {
+      throw new UnsupportedOperationException("Complex type Map is not supported yet");
+    } else {
+      return new PrimitiveDataType(carbonColumn.getColName(), parentName,
+          carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache, carbonTableIdentifier,
+          client, useOnePass, storePath, localCache);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
index 8170680..eac9d69 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.processing.loading.converter.impl;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.loading.DataField;
@@ -52,7 +53,7 @@ public class NonDictionaryFieldConverterImpl implements FieldConverter {
 
   @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
     String dimensionValue = row.getString(index);
-    if (null == dimensionValue && column.getDataType() != DataType.STRING) {
+    if (null == dimensionValue && column.getDataType() != DataTypes.STRING) {
       logHolder.setReason(
           CarbonDataProcessorUtil.prepareFailureReason(column.getColName(), column.getDataType()));
       updateWithNullValue(row);
@@ -81,7 +82,7 @@ public class NonDictionaryFieldConverterImpl implements FieldConverter {
   }
 
   private void updateWithNullValue(CarbonRow row) {
-    if (dataType == DataType.STRING) {
+    if (dataType == DataTypes.STRING) {
       row.update(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, index);
     } else {
       row.update(CarbonCommonConstants.EMPTY_BYTE_ARRAY, index);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
index 0ee1d90..608d0a9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
@@ -18,6 +18,8 @@ package org.apache.carbondata.processing.loading.parser;
 
 import java.util.List;
 
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.processing.loading.parser.impl.ArrayParserImpl;
@@ -51,30 +53,29 @@ public final class CarbonParserFactory {
    */
   private static GenericParser createParser(CarbonColumn carbonColumn, String[] complexDelimiters,
       String nullFormat, int depth) {
-    switch (carbonColumn.getDataType()) {
-      case ARRAY:
-        List<CarbonDimension> listOfChildDimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create array parser with complex delimiter
-        ArrayParserImpl arrayParser = new ArrayParserImpl(complexDelimiters[depth], nullFormat);
-        for (CarbonDimension dimension : listOfChildDimensions) {
-          arrayParser
-              .addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
-        }
-        return arrayParser;
-      case STRUCT:
-        List<CarbonDimension> dimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create struct parser with complex delimiter
-        StructParserImpl parser = new StructParserImpl(complexDelimiters[depth], nullFormat);
-        for (CarbonDimension dimension : dimensions) {
-          parser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
-        }
-        return parser;
-      case MAP:
-        throw new UnsupportedOperationException("Complex type Map is not supported yet");
-      default:
-        return new PrimitiveParserImpl();
+    DataType dataType = carbonColumn.getDataType();
+    if (dataType == DataTypes.ARRAY) {
+      List<CarbonDimension> listOfChildDimensions =
+          ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+      // Create array parser with complex delimiter
+      ArrayParserImpl arrayParser = new ArrayParserImpl(complexDelimiters[depth], nullFormat);
+      for (CarbonDimension dimension : listOfChildDimensions) {
+        arrayParser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
+      }
+      return arrayParser;
+    } else if (dataType == DataTypes.STRUCT) {
+      List<CarbonDimension> dimensions =
+          ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+      // Create struct parser with complex delimiter
+      StructParserImpl parser = new StructParserImpl(complexDelimiters[depth], nullFormat);
+      for (CarbonDimension dimension : dimensions) {
+        parser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
+      }
+      return parser;
+    } else if (dataType == DataTypes.MAP) {
+      throw new UnsupportedOperationException("Complex type Map is not supported yet");
+    } else {
+      return new PrimitiveParserImpl();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
index 06bd716..a41f734 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.processing.loading.partition.impl;
 
 import java.util.List;
 
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.processing.loading.partition.Partitioner;
 
@@ -36,19 +38,14 @@ public class HashPartitionerImpl implements Partitioner<Object[]> {
     this.numberOfBuckets = numberOfBuckets;
     hashes = new Hash[indexes.size()];
     for (int i = 0; i < indexes.size(); i++) {
-      switch (columnSchemas.get(i).getDataType()) {
-        case SHORT:
-        case INT:
-        case LONG:
-          hashes[i] = new IntegralHash(indexes.get(i));
-          break;
-        case DOUBLE:
-        case FLOAT:
-        case DECIMAL:
-          hashes[i] = new DecimalHash(indexes.get(i));
-          break;
-        default:
-          hashes[i] = new StringHash(indexes.get(i));
+      DataType dataType = columnSchemas.get(i).getDataType();
+      if (dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) {
+        hashes[i] = new IntegralHash(indexes.get(i));
+      } else if (dataType == DataTypes.DOUBLE || dataType == DataTypes.FLOAT ||
+          dataType == DataTypes.DECIMAL) {
+        hashes[i] = new DecimalHash(indexes.get(i));
+      } else {
+        hashes[i] = new StringHash(indexes.get(i));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
index 14ab838..ff42e2d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 /**
@@ -125,41 +126,35 @@ public class UnsafeCarbonRowPage {
     for (int mesCount = 0; mesCount < measureSize; mesCount++) {
       Object value = row[mesCount + dimensionSize];
       if (null != value) {
-        switch (measureDataType[mesCount]) {
-          case SHORT:
-            Short sval = (Short) value;
-            CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
-            size += 2;
-            break;
-          case INT:
-            Integer ival = (Integer) value;
-            CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
-            size += 4;
-            break;
-          case LONG:
-            Long val = (Long) value;
-            CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
-            size += 8;
-            break;
-          case DOUBLE:
-            Double doubleVal = (Double) value;
-            CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
-            size += 8;
-            break;
-          case DECIMAL:
-            BigDecimal decimalVal = (BigDecimal) value;
-            byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(decimalVal);
-            CarbonUnsafe.getUnsafe().putShort(baseObject, address + size,
-                (short) bigDecimalInBytes.length);
-            size += 2;
-            CarbonUnsafe.getUnsafe()
-                .copyMemory(bigDecimalInBytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-                    address + size, bigDecimalInBytes.length);
-            size += bigDecimalInBytes.length;
-            break;
-          default:
-            throw  new IllegalArgumentException("unsupported data type:" +
-                measureDataType[mesCount]);
+        DataType dataType = measureDataType[mesCount];
+        if (dataType == DataTypes.SHORT) {
+          Short sval = (Short) value;
+          CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
+          size += 2;
+        } else if (dataType == DataTypes.INT) {
+          Integer ival = (Integer) value;
+          CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
+          size += 4;
+        } else if (dataType == DataTypes.LONG) {
+          Long val = (Long) value;
+          CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
+          size += 8;
+        } else if (dataType == DataTypes.DOUBLE) {
+          Double doubleVal = (Double) value;
+          CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
+          size += 8;
+        } else if (dataType == DataTypes.DECIMAL) {
+          BigDecimal decimalVal = (BigDecimal) value;
+          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(decimalVal);
+          CarbonUnsafe.getUnsafe()
+              .putShort(baseObject, address + size, (short) bigDecimalInBytes.length);
+          size += 2;
+          CarbonUnsafe.getUnsafe()
+              .copyMemory(bigDecimalInBytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
+                  address + size, bigDecimalInBytes.length);
+          size += bigDecimalInBytes.length;
+        } else {
+          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
         }
         set(nullSetWords, mesCount);
       } else {
@@ -213,39 +208,33 @@ public class UnsafeCarbonRowPage {
 
     for (int mesCount = 0; mesCount < measureSize; mesCount++) {
       if (isSet(nullSetWords, mesCount)) {
-        switch (measureDataType[mesCount]) {
-          case SHORT:
-            Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-            size += 2;
-            rowToFill[dimensionSize + mesCount] = sval;
-            break;
-          case INT:
-            Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-            size += 4;
-            rowToFill[dimensionSize + mesCount] = ival;
-            break;
-          case LONG:
-            Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-            size += 8;
-            rowToFill[dimensionSize + mesCount] = val;
-            break;
-          case DOUBLE:
-            Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-            size += 8;
-            rowToFill[dimensionSize + mesCount] = doubleVal;
-            break;
-          case DECIMAL:
-            short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-            byte[] bigDecimalInBytes = new byte[aShort];
-            size += 2;
-            CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
-                CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
-            size += bigDecimalInBytes.length;
-            rowToFill[dimensionSize + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-            break;
-          default:
-            throw new IllegalArgumentException("unsupported data type:" +
-                measureDataType[mesCount]);
+        DataType dataType = measureDataType[mesCount];
+        if (dataType == DataTypes.SHORT) {
+          Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          size += 2;
+          rowToFill[dimensionSize + mesCount] = sval;
+        } else if (dataType == DataTypes.INT) {
+          Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+          size += 4;
+          rowToFill[dimensionSize + mesCount] = ival;
+        } else if (dataType == DataTypes.LONG) {
+          Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+          size += 8;
+          rowToFill[dimensionSize + mesCount] = val;
+        } else if (dataType == DataTypes.DOUBLE) {
+          Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
+          size += 8;
+          rowToFill[dimensionSize + mesCount] = doubleVal;
+        } else if (dataType == DataTypes.DECIMAL) {
+          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          byte[] bigDecimalInBytes = new byte[aShort];
+          size += 2;
+          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
+              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
+          size += bigDecimalInBytes.length;
+          rowToFill[dimensionSize + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
+        } else {
+          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
         }
       } else {
         rowToFill[dimensionSize + mesCount] = null;
@@ -301,40 +290,34 @@ public class UnsafeCarbonRowPage {
 
     for (int mesCount = 0; mesCount < measureSize; mesCount++) {
       if (isSet(nullSetWords, mesCount)) {
-        switch (measureDataType[mesCount]) {
-          case SHORT:
-            short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-            size += 2;
-            stream.writeShort(sval);
-            break;
-          case INT:
-            int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-            size += 4;
-            stream.writeInt(ival);
-            break;
-          case LONG:
-            long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-            size += 8;
-            stream.writeLong(val);
-            break;
-          case DOUBLE:
-            double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-            size += 8;
-            stream.writeDouble(doubleVal);
-            break;
-          case DECIMAL:
-            short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-            byte[] bigDecimalInBytes = new byte[aShort];
-            size += 2;
-            CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
-                CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
-            size += bigDecimalInBytes.length;
-            stream.writeShort(aShort);
-            stream.write(bigDecimalInBytes);
-            break;
-          default:
-            throw new IllegalArgumentException("unsupported data type:" +
-                measureDataType[mesCount]);
+        DataType dataType = measureDataType[mesCount];
+        if (dataType == DataTypes.SHORT) {
+          short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          size += 2;
+          stream.writeShort(sval);
+        } else if (dataType == DataTypes.INT) {
+          int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+          size += 4;
+          stream.writeInt(ival);
+        } else if (dataType == DataTypes.LONG) {
+          long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+          size += 8;
+          stream.writeLong(val);
+        } else if (dataType == DataTypes.DOUBLE) {
+          double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
+          size += 8;
+          stream.writeDouble(doubleVal);
+        } else if (dataType == DataTypes.DECIMAL) {
+          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          byte[] bigDecimalInBytes = new byte[aShort];
+          size += 2;
+          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
+              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
+          size += bigDecimalInBytes.length;
+          stream.writeShort(aShort);
+          stream.write(bigDecimalInBytes);
+        } else {
+          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 5fed2ea..404a521 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -33,6 +33,7 @@ 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.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
@@ -325,28 +326,23 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
       for (int mesCount = 0; mesCount < measureCount; mesCount++) {
         if (UnsafeCarbonRowPage.isSet(words, mesCount)) {
-          switch (measureDataType[mesCount]) {
-            case SHORT:
-              row[dimensionCount + mesCount] = stream.readShort();
-              break;
-            case INT:
-              row[dimensionCount + mesCount] = stream.readInt();
-              break;
-            case LONG:
-              row[dimensionCount + mesCount] = stream.readLong();
-              break;
-            case DOUBLE:
-              row[dimensionCount + mesCount] = stream.readDouble();
-              break;
-            case DECIMAL:
-              short aShort = stream.readShort();
-              byte[] bigDecimalInBytes = new byte[aShort];
-              stream.readFully(bigDecimalInBytes);
-              row[dimensionCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-              break;
-            default:
-              throw new IllegalArgumentException("unsupported data type:" +
-                  measureDataType[mesCount]);
+          DataType dataType = measureDataType[mesCount];
+          if (dataType == DataTypes.SHORT) {
+            row[dimensionCount + mesCount] = stream.readShort();
+          } else if (dataType == DataTypes.INT) {
+            row[dimensionCount + mesCount] = stream.readInt();
+          } else if (dataType == DataTypes.LONG) {
+            row[dimensionCount + mesCount] = stream.readLong();
+          } else if (dataType == DataTypes.DOUBLE) {
+            row[dimensionCount + mesCount] = stream.readDouble();
+          } else if (dataType == DataTypes.DECIMAL) {
+            short aShort = stream.readShort();
+            byte[] bigDecimalInBytes = new byte[aShort];
+            stream.readFully(bigDecimalInBytes);
+            row[dimensionCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
+          } else {
+            throw new IllegalArgumentException(
+                "unsupported data type:" + measureDataType[mesCount]);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
index 4303ec8..9f7d6c3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
@@ -32,6 +32,7 @@ import java.util.concurrent.Callable;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
@@ -315,31 +316,26 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
     for (int mesCount = 0; mesCount < measureSize; mesCount++) {
       Object value = row[mesCount + dimensionSize];
       if (null != value) {
-        switch (type[mesCount]) {
-          case SHORT:
-            rowData.putShort(size, (Short) value);
-            size += 2;
-            break;
-          case INT:
-            rowData.putInt(size, (Integer) value);
-            size += 4;
-            break;
-          case LONG:
-            rowData.putLong(size, (Long) value);
-            size += 8;
-            break;
-          case DOUBLE:
-            rowData.putDouble(size, (Double) value);
-            size += 8;
-            break;
-          case DECIMAL:
-            byte[] bigDecimalInBytes = (byte[]) value;
-            rowData.putShort(size, (short)bigDecimalInBytes.length);
-            size += 2;
-            for (int i = 0; i < bigDecimalInBytes.length; i++) {
-              rowData.put(size++, bigDecimalInBytes[i]);
-            }
-            break;
+        DataType dataType = type[mesCount];
+        if (dataType == DataTypes.SHORT) {
+          rowData.putShort(size, (Short) value);
+          size += 2;
+        } else if (dataType == DataTypes.INT) {
+          rowData.putInt(size, (Integer) value);
+          size += 4;
+        } else if (dataType == DataTypes.LONG) {
+          rowData.putLong(size, (Long) value);
+          size += 8;
+        } else if (dataType == DataTypes.DOUBLE) {
+          rowData.putDouble(size, (Double) value);
+          size += 8;
+        } else if (dataType == DataTypes.DECIMAL) {
+          byte[] bigDecimalInBytes = (byte[]) value;
+          rowData.putShort(size, (short) bigDecimalInBytes.length);
+          size += 2;
+          for (int i = 0; i < bigDecimalInBytes.length; i++) {
+            rowData.put(size++, bigDecimalInBytes[i]);
+          }
         }
         UnsafeCarbonRowPage.set(nullSetWords, mesCount);
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index edffae9..0e20ef5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -256,14 +257,13 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
    * @return
    */
   private Object getConvertedMeasureValue(Object value, DataType type) {
-    switch (type) {
-      case DECIMAL:
-        if (value != null) {
-          value = ((Decimal) value).toJavaBigDecimal();
-        }
-        return value;
-      default:
-        return value;
+    if (type == DataTypes.DECIMAL) {
+      if (value != null) {
+        value = ((Decimal) value).toJavaBigDecimal();
+      }
+      return value;
+    } else {
+      return value;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
index d4a8dd6..3671316 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -30,6 +30,7 @@ import java.util.concurrent.Callable;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.NonDictionaryUtil;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
@@ -345,26 +346,21 @@ public class IntermediateFileMerger implements Callable<Void> {
       for (int counter = 0; counter < mergerParameters.getMeasureColCount(); counter++) {
         if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
           stream.write((byte) 1);
-          switch (aggType[counter]) {
-            case SHORT:
-              stream.writeShort((short)NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case INT:
-              stream.writeInt((int)NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case LONG:
-              stream.writeLong((long)NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case DOUBLE:
-              stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case DECIMAL:
-              byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
-              stream.writeInt(bigDecimalInBytes.length);
-              stream.write(bigDecimalInBytes);
-              break;
-            default:
-              throw new IllegalArgumentException("unsupported data type:" + aggType[counter]);
+          DataType dataType = aggType[counter];
+          if (dataType == DataTypes.SHORT) {
+            stream.writeShort((short) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.INT) {
+            stream.writeInt((int) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.LONG) {
+            stream.writeLong((long) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.DOUBLE) {
+            stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.DECIMAL) {
+            byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
+            stream.writeInt(bigDecimalInBytes.length);
+            stream.write(bigDecimalInBytes);
+          } else {
+            throw new IllegalArgumentException("unsupported data type:" + aggType[counter]);
           }
         } else {
           stream.write((byte) 0);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
index 11df276..5cc96c5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -34,6 +34,7 @@ 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.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -293,27 +294,22 @@ public class SortDataRows {
           Object value = row[mesCount + dimColCount];
           if (null != value) {
             stream.write((byte) 1);
-            switch (type[mesCount]) {
-              case SHORT:
-                stream.writeShort((Short) value);
-                break;
-              case INT:
-                stream.writeInt((Integer) value);
-                break;
-              case LONG:
-                stream.writeLong((Long) value);
-                break;
-              case DOUBLE:
-                stream.writeDouble((Double) value);
-                break;
-              case DECIMAL:
-                BigDecimal val = (BigDecimal) value;
-                byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
-                stream.writeInt(bigDecimalInBytes.length);
-                stream.write(bigDecimalInBytes);
-                break;
-              default:
-                throw new IllegalArgumentException("unsupported data type:" + type[mesCount]);
+            DataType dataType = type[mesCount];
+            if (dataType == DataTypes.SHORT) {
+              stream.writeShort((Short) value);
+            } else if (dataType == DataTypes.INT) {
+              stream.writeInt((Integer) value);
+            } else if (dataType == DataTypes.LONG) {
+              stream.writeLong((Long) value);
+            } else if (dataType == DataTypes.DOUBLE) {
+              stream.writeDouble((Double) value);
+            } else if (dataType == DataTypes.DECIMAL) {
+              BigDecimal val = (BigDecimal) value;
+              byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
+              stream.writeInt(bigDecimalInBytes.length);
+              stream.write(bigDecimalInBytes);
+            } else {
+              throw new IllegalArgumentException("unsupported data type:" + type[mesCount]);
             }
           } else {
             stream.write((byte) 0);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
index 3e56605..5d339c7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
@@ -32,6 +32,7 @@ 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.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
@@ -343,27 +344,22 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
       // read measure values
       for (int i = 0; i < this.measureCount; i++) {
         if (stream.readByte() == 1) {
-          switch (aggType[i]) {
-            case SHORT:
-              measures[index++] = stream.readShort();
-              break;
-            case INT:
-              measures[index++] = stream.readInt();
-              break;
-            case LONG:
-              measures[index++] = stream.readLong();
-              break;
-            case DOUBLE:
-              measures[index++] = stream.readDouble();
-              break;
-            case DECIMAL:
-              int len = stream.readInt();
-              byte[] buff = new byte[len];
-              stream.readFully(buff);
-              measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
-              break;
-            default:
-              throw new IllegalArgumentException("unsupported data type:" + aggType[i]);
+          DataType dataType = aggType[i];
+          if (dataType == DataTypes.SHORT) {
+            measures[index++] = stream.readShort();
+          } else if (dataType == DataTypes.INT) {
+            measures[index++] = stream.readInt();
+          } else if (dataType == DataTypes.LONG) {
+            measures[index++] = stream.readLong();
+          } else if (dataType == DataTypes.DOUBLE) {
+            measures[index++] = stream.readDouble();
+          } else if (dataType == DataTypes.DECIMAL) {
+            int len = stream.readInt();
+            byte[] buff = new byte[len];
+            stream.readFully(buff);
+            measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
+          } else {
+            throw new IllegalArgumentException("unsupported data type:" + aggType[i]);
           }
         } else {
           measures[index++] = null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index 78f1637..fdf44cf 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -45,6 +45,7 @@ import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -503,7 +504,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
         new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     DataType[] type = model.getMeasureDataType();
     for (int j = 0; j < type.length; j++) {
-      if (type[j] != DataType.BYTE && type[j] != DataType.DECIMAL) {
+      if (type[j] != DataTypes.BYTE && type[j] != DataTypes.DECIMAL) {
         otherMeasureIndexList.add(j);
       } else {
         customMeasureIndexList.add(j);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
index 287de0a..d2cf1c4 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
@@ -46,6 +46,7 @@ import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
 
 import org.apache.spark.sql.types.Decimal;
@@ -91,14 +92,14 @@ public class TablePage {
     dictDimensionPages = new ColumnPage[numDictDimension];
     for (int i = 0; i < dictDimensionPages.length; i++) {
       TableSpec.DimensionSpec spec = tableSpec.getDimensionSpec(i);
-      ColumnPage page = ColumnPage.newPage(spec, DataType.BYTE_ARRAY, pageSize);
-      page.setStatsCollector(KeyPageStatsCollector.newInstance(DataType.BYTE_ARRAY));
+      ColumnPage page = ColumnPage.newPage(spec, DataTypes.BYTE_ARRAY, pageSize);
+      page.setStatsCollector(KeyPageStatsCollector.newInstance(DataTypes.BYTE_ARRAY));
       dictDimensionPages[i] = page;
     }
     noDictDimensionPages = new ColumnPage[model.getNoDictionaryCount()];
     for (int i = 0; i < noDictDimensionPages.length; i++) {
       TableSpec.DimensionSpec spec = tableSpec.getDimensionSpec(i + numDictDimension);
-      ColumnPage page = ColumnPage.newPage(spec, DataType.STRING, pageSize);
+      ColumnPage page = ColumnPage.newPage(spec, DataTypes.STRING, pageSize);
       page.setStatsCollector(LVStringStatsCollector.newInstance());
       noDictDimensionPages[i] = page;
     }
@@ -113,7 +114,7 @@ public class TablePage {
     for (int i = 0; i < measurePages.length; i++) {
       TableSpec.MeasureSpec spec = model.getTableSpec().getMeasureSpec(i);
       ColumnPage page;
-      if (spec.getSchemaDataType() == DataType.DECIMAL) {
+      if (spec.getSchemaDataType() == DataTypes.DECIMAL) {
         page = ColumnPage.newDecimalPage(spec, dataTypes[i], pageSize);
       } else {
         page = ColumnPage.newPage(spec, dataTypes[i], pageSize);
@@ -182,7 +183,7 @@ public class TablePage {
 
       // in compaction flow the measure with decimal type will come as Spark decimal.
       // need to convert it to byte array.
-      if (measurePages[i].getDataType() == DataType.DECIMAL &&
+      if (measurePages[i].getDataType() == DataTypes.DECIMAL &&
           model.isCompactionFlow() &&
           value != null) {
         value = ((Decimal) value).toJavaBigDecimal();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index 79e49ef..1c7f9e7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -41,6 +41,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory.FileType;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -278,8 +279,8 @@ public final class CarbonDataProcessorUtil {
     StringBuilder dimString = new StringBuilder();
     for (int i = 0; i < dataFields.length; i++) {
       DataField dataField = dataFields[i];
-      if (dataField.getColumn().getDataType().equals(DataType.ARRAY) || dataField.getColumn()
-          .getDataType().equals(DataType.STRUCT)) {
+      if (dataField.getColumn().getDataType() == DataTypes.ARRAY ||
+          dataField.getColumn().getDataType() == DataTypes.STRUCT) {
         addAllComplexTypeChildren((CarbonDimension) dataField.getColumn(), dimString, "");
         dimString.append(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
       }
@@ -321,22 +322,19 @@ public final class CarbonDataProcessorUtil {
     for (int i = 0; i < hierarchies.length; i++) {
       String[] levels = hierarchies[i].split(CarbonCommonConstants.HASH_SPC_CHARACTER);
       String[] levelInfo = levels[0].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      GenericDataType g = levelInfo[1].equals(CarbonCommonConstants.ARRAY) ?
+      GenericDataType g = levelInfo[1].toLowerCase().contains(CarbonCommonConstants.ARRAY) ?
           new ArrayDataType(levelInfo[0], "", levelInfo[3]) :
           new StructDataType(levelInfo[0], "", levelInfo[3]);
       complexTypesMap.put(levelInfo[0], g);
       for (int j = 1; j < levels.length; j++) {
         levelInfo = levels[j].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-        switch (levelInfo[1]) {
-          case CarbonCommonConstants.ARRAY:
-            g.addChildren(new ArrayDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
-            break;
-          case CarbonCommonConstants.STRUCT:
-            g.addChildren(new StructDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
-            break;
-          default:
-            g.addChildren(new PrimitiveDataType(levelInfo[0], levelInfo[2], levelInfo[3],
-                Integer.parseInt(levelInfo[4])));
+        if (levelInfo[1].toLowerCase().contains(CarbonCommonConstants.ARRAY)) {
+          g.addChildren(new ArrayDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
+        } else if (levelInfo[1].toLowerCase().contains(CarbonCommonConstants.STRUCT)) {
+          g.addChildren(new StructDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
+        } else {
+          g.addChildren(new PrimitiveDataType(levelInfo[0], levelInfo[2], levelInfo[3],
+              Integer.parseInt(levelInfo[4])));
         }
       }
     }
@@ -396,7 +394,7 @@ public final class CarbonDataProcessorUtil {
       String tableName) {
     DataType[] type = new DataType[measureCount];
     for (int i = 0; i < type.length; i++) {
-      type[i] = DataType.DOUBLE;
+      type[i] = DataTypes.DOUBLE;
     }
     CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
         databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
@@ -458,7 +456,7 @@ public final class CarbonDataProcessorUtil {
       int measureCount) {
     DataType[] type = new DataType[measureCount];
     for (int i = 0; i < type.length; i++) {
-      type[i] = DataType.DOUBLE;
+      type[i] = DataTypes.DOUBLE;
     }
     List<CarbonMeasure> measures = carbonTable.getMeasureByTableName(tableName);
     for (int i = 0; i < measureCount; i++) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
index 61771ea..37331c9 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -50,6 +50,7 @@ import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
@@ -183,7 +184,7 @@ public class StoreCreator {
     ColumnSchema id = new ColumnSchema();
     id.setColumnName("ID");
     id.setColumnar(true);
-    id.setDataType(DataType.INT);
+    id.setDataType(DataTypes.INT);
     id.setEncodingList(encodings);
     id.setColumnUniqueId(UUID.randomUUID().toString());
     id.setDimensionColumn(true);
@@ -193,7 +194,7 @@ public class StoreCreator {
     ColumnSchema date = new ColumnSchema();
     date.setColumnName("date");
     date.setColumnar(true);
-    date.setDataType(DataType.STRING);
+    date.setDataType(DataTypes.STRING);
     date.setEncodingList(encodings);
     date.setColumnUniqueId(UUID.randomUUID().toString());
     date.setDimensionColumn(true);
@@ -203,7 +204,7 @@ public class StoreCreator {
     ColumnSchema country = new ColumnSchema();
     country.setColumnName("country");
     country.setColumnar(true);
-    country.setDataType(DataType.STRING);
+    country.setDataType(DataTypes.STRING);
     country.setEncodingList(encodings);
     country.setColumnUniqueId(UUID.randomUUID().toString());
     country.setDimensionColumn(true);
@@ -213,7 +214,7 @@ public class StoreCreator {
     ColumnSchema name = new ColumnSchema();
     name.setColumnName("name");
     name.setColumnar(true);
-    name.setDataType(DataType.STRING);
+    name.setDataType(DataTypes.STRING);
     name.setEncodingList(encodings);
     name.setColumnUniqueId(UUID.randomUUID().toString());
     name.setDimensionColumn(true);
@@ -223,7 +224,7 @@ public class StoreCreator {
     ColumnSchema phonetype = new ColumnSchema();
     phonetype.setColumnName("phonetype");
     phonetype.setColumnar(true);
-    phonetype.setDataType(DataType.STRING);
+    phonetype.setDataType(DataTypes.STRING);
     phonetype.setEncodingList(encodings);
     phonetype.setColumnUniqueId(UUID.randomUUID().toString());
     phonetype.setDimensionColumn(true);
@@ -233,7 +234,7 @@ public class StoreCreator {
     ColumnSchema serialname = new ColumnSchema();
     serialname.setColumnName("serialname");
     serialname.setColumnar(true);
-    serialname.setDataType(DataType.STRING);
+    serialname.setDataType(DataTypes.STRING);
     serialname.setEncodingList(encodings);
     serialname.setColumnUniqueId(UUID.randomUUID().toString());
     serialname.setDimensionColumn(true);
@@ -243,7 +244,7 @@ public class StoreCreator {
     ColumnSchema salary = new ColumnSchema();
     salary.setColumnName("salary");
     salary.setColumnar(true);
-    salary.setDataType(DataType.INT);
+    salary.setDataType(DataTypes.INT);
     salary.setEncodingList(new ArrayList<Encoding>());
     salary.setColumnUniqueId(UUID.randomUUID().toString());
     salary.setDimensionColumn(false);


[06/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index e8488f9..cfe2cc8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -54,6 +54,7 @@ import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -329,7 +330,7 @@ public final class FilterUtil {
     if (expression.getFilterExpressionType() == ExpressionType.LITERAL
         && expression instanceof LiteralExpression) {
       DataType dataType = ((LiteralExpression) expression).getLiteralExpDataType();
-      if (!(dataType == DataType.TIMESTAMP || dataType == DataType.DATE)) {
+      if (!(dataType == DataTypes.TIMESTAMP || dataType == DataTypes.DATE)) {
         return true;
       }
     }
@@ -420,7 +421,7 @@ public final class FilterUtil {
       for (int i = 0; i < length; i++) {
         result = evaluateResultListFinal.get(i);
         if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(result)) {
-          if (dataType == DataType.STRING) {
+          if (dataType == DataTypes.STRING) {
             filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
           } else {
             filterValuesList.add(CarbonCommonConstants.EMPTY_BYTE_ARRAY);
@@ -1295,39 +1296,33 @@ public final class FilterUtil {
   public static int compareFilterKeyBasedOnDataType(String dictionaryVal, String memberVal,
       DataType dataType) {
     try {
-      switch (dataType) {
-        case SHORT:
-          return Short.compare((Short.parseShort(dictionaryVal)), (Short.parseShort(memberVal)));
-        case INT:
-          return Integer.compare((Integer.parseInt(dictionaryVal)), (Integer.parseInt(memberVal)));
-        case DOUBLE:
-          return Double
-              .compare((Double.parseDouble(dictionaryVal)), (Double.parseDouble(memberVal)));
-        case LONG:
-          return Long.compare((Long.parseLong(dictionaryVal)), (Long.parseLong(memberVal)));
-        case BOOLEAN:
-          return Boolean
-              .compare((Boolean.parseBoolean(dictionaryVal)), (Boolean.parseBoolean(memberVal)));
-        case DATE:
-        case TIMESTAMP:
-          String format = CarbonUtil.getFormatFromProperty(dataType);
-          SimpleDateFormat parser = new SimpleDateFormat(format);
-          Date dateToStr;
-          Date dictionaryDate;
-          dateToStr = parser.parse(memberVal);
-          dictionaryDate = parser.parse(dictionaryVal);
-          return dictionaryDate.compareTo(dateToStr);
-
-        case DECIMAL:
-          java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
-          java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
-          return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
-        default:
-          return -1;
+      if (dataType == DataTypes.SHORT) {
+        return Short.compare((Short.parseShort(dictionaryVal)), (Short.parseShort(memberVal)));
+      } else if (dataType == DataTypes.INT) {
+        return Integer.compare((Integer.parseInt(dictionaryVal)), (Integer.parseInt(memberVal)));
+      } else if (dataType == DataTypes.DOUBLE) {
+        return Double.compare((Double.parseDouble(dictionaryVal)), (Double.parseDouble(memberVal)));
+      } else if (dataType == DataTypes.LONG) {
+        return Long.compare((Long.parseLong(dictionaryVal)), (Long.parseLong(memberVal)));
+      } else if (dataType == DataTypes.BOOLEAN) {
+        return Boolean.compare(
+            (Boolean.parseBoolean(dictionaryVal)), (Boolean.parseBoolean(memberVal)));
+      } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+        String format = CarbonUtil.getFormatFromProperty(dataType);
+        SimpleDateFormat parser = new SimpleDateFormat(format);
+        Date dateToStr;
+        Date dictionaryDate;
+        dateToStr = parser.parse(memberVal);
+        dictionaryDate = parser.parse(dictionaryVal);
+        return dictionaryDate.compareTo(dateToStr);
+      } else if (dataType == DataTypes.DECIMAL) {
+        java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
+        java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
+        return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
+      } else {
+        return -1;
       }
-    } catch (ParseException e) {
-      return -1;
-    } catch (NumberFormatException e) {
+    } catch (ParseException | NumberFormatException e) {
       return -1;
     }
   }
@@ -1422,53 +1417,47 @@ public final class FilterUtil {
   private static int compareFilterMembersBasedOnActualDataType(String filterMember1,
       String filterMember2, DataType dataType) {
     try {
-      switch (dataType) {
-        case SHORT:
-        case INT:
-        case LONG:
-        case DOUBLE:
-
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
-            return 1;
-          }
-          Double d1 = Double.parseDouble(filterMember1);
-          Double d2 = Double.parseDouble(filterMember2);
-          return d1.compareTo(d2);
-        case DECIMAL:
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
-            return 1;
-          }
-          java.math.BigDecimal val1 = new BigDecimal(filterMember1);
-          java.math.BigDecimal val2 = new BigDecimal(filterMember2);
-          return val1.compareTo(val2);
-        case DATE:
-        case TIMESTAMP:
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
-            return 1;
-          }
-          String format = null;
-          if (dataType == DataType.DATE) {
-            format = CarbonProperties.getInstance()
-                .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-                    CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
-          } else {
-            format = CarbonProperties.getInstance()
-                .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                    CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-          }
-          SimpleDateFormat parser = new SimpleDateFormat(format);
-          Date date1 = null;
-          Date date2 = null;
-          date1 = parser.parse(filterMember1);
-          date2 = parser.parse(filterMember2);
-          return date1.compareTo(date2);
-        case STRING:
-        default:
-          return filterMember1.compareTo(filterMember2);
+      if (dataType == DataTypes.SHORT ||
+          dataType == DataTypes.INT ||
+          dataType == DataTypes.LONG ||
+          dataType == DataTypes.DOUBLE) {
+        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
+          return 1;
+        }
+        Double d1 = Double.parseDouble(filterMember1);
+        Double d2 = Double.parseDouble(filterMember2);
+        return d1.compareTo(d2);
+      } else if (dataType == DataTypes.DECIMAL) {
+        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
+          return 1;
+        }
+        java.math.BigDecimal val1 = new BigDecimal(filterMember1);
+        java.math.BigDecimal val2 = new BigDecimal(filterMember2);
+        return val1.compareTo(val2);
+      } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
+          return 1;
+        }
+        String format = null;
+        if (dataType == DataTypes.DATE) {
+          format = CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+                  CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+        } else {
+          format = CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+        }
+        SimpleDateFormat parser = new SimpleDateFormat(format);
+        Date date1 = null;
+        Date date2 = null;
+        date1 = parser.parse(filterMember1);
+        date2 = parser.parse(filterMember2);
+        return date1.compareTo(date2);
+      } else {
+        return filterMember1.compareTo(filterMember2);
       }
-    } catch (ParseException e) {
-      return -1;
-    } catch (NumberFormatException e) {
+    } catch (ParseException | NumberFormatException e) {
       return -1;
     }
   }
@@ -1562,7 +1551,7 @@ public final class FilterUtil {
    */
   public static boolean isExpressionNeedsToResolved(Expression rightExp, boolean isIncludeFilter) {
     if (!isIncludeFilter && rightExp instanceof LiteralExpression && (
-        DataType.NULL == ((LiteralExpression) rightExp)
+        DataTypes.NULL == ((LiteralExpression) rightExp)
             .getLiteralExpDataType())) {
       return true;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
index 841ecdc..4013578 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -25,6 +25,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
@@ -123,17 +124,16 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   }
 
   private DataType getMeasureDataType(MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo) {
-    switch (msrColumnEvaluatorInfo.getType()) {
-      case SHORT:
-        return DataType.SHORT;
-      case INT:
-        return DataType.INT;
-      case LONG:
-        return DataType.LONG;
-      case DECIMAL:
-        return DataType.DECIMAL;
-      default:
-        return DataType.DOUBLE;
+    if (msrColumnEvaluatorInfo.getType() == DataTypes.SHORT) {
+      return DataTypes.SHORT;
+    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.INT) {
+      return DataTypes.INT;
+    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.LONG) {
+      return DataTypes.LONG;
+    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.DECIMAL) {
+      return DataTypes.DECIMAL;
+    } else {
+      return DataTypes.DOUBLE;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
index 7d6fd4f..04fce4a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -25,6 +25,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
@@ -147,17 +148,16 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
   }
 
   private DataType getMeasureDataType(MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo) {
-    switch (msrColumnEvaluatorInfo.getType()) {
-      case SHORT:
-        return DataType.SHORT;
-      case INT:
-        return DataType.INT;
-      case LONG:
-        return DataType.LONG;
-      case DECIMAL:
-        return DataType.DECIMAL;
-      default:
-        return DataType.DOUBLE;
+    if (msrColumnEvaluatorInfo.getType() == DataTypes.SHORT) {
+      return DataTypes.SHORT;
+    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.INT) {
+      return DataTypes.INT;
+    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.LONG) {
+      return DataTypes.LONG;
+    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.DECIMAL) {
+      return DataTypes.DECIMAL;
+    } else {
+      return DataTypes.DOUBLE;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
index acce53e..0cfa198 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -564,7 +564,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
           defaultValue = ByteUtil.toBytes(key);
         }
       } else {
-        if (dimColEvaluatorInfo.getDimension().getDataType() == DataType.STRING) {
+        if (dimColEvaluatorInfo.getDimension().getDataType() == DataTypes.STRING) {
           defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
         } else {
           defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java
index c570ed2..cf3c11e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java
@@ -21,7 +21,7 @@ import java.nio.charset.Charset;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -54,7 +54,7 @@ public abstract class RestructureEvaluatorImpl implements FilterExecuter {
     if (!dimension.hasEncoding(Encoding.DICTIONARY)) {
       // for no dictionary cases
       // 3 cases: is NUll, is Not Null and filter on default value of newly added column
-      if (null == defaultValue && dimension.getDataType() == DataType.STRING) {
+      if (null == defaultValue && dimension.getDataType() == DataTypes.STRING) {
         // default value for case where user gives is Null condition
         defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL
             .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 08ce565..c243368 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -42,6 +42,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -143,9 +144,9 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     this.tableIdentifier = tableIdentifier;
     this.complexDimensionInfoMap = complexDimensionInfoMap;
     this.dateDictionaryGenerator =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.DATE);
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.DATE);
     this.timestampDictionaryGenerator =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.TIMESTAMP);
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
     initDimensionBlockIndexes();
     initMeasureBlockIndexes();
   }
@@ -287,8 +288,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
             getDimensionDefaultValue(dimColumnEvaluatorInfo);
         continue;
       }
-      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.ARRAY
-          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.STRUCT) {
+      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.ARRAY
+          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.STRUCT) {
         if (!dimColumnEvaluatorInfo.isDimensionExistsInCurrentSilce()) {
           record[dimColumnEvaluatorInfo.getRowIndex()] =
               dimColumnEvaluatorInfo.getDimension().getDefaultValue();
@@ -311,8 +312,6 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
             record[dimColumnEvaluatorInfo.getRowIndex()] = DataTypeUtil
                 .getDataBasedOnDataTypeForNoDictionaryColumn(memberBytes,
                     dimColumnEvaluatorInfo.getDimension().getDataType());
-          } else {
-            continue;
           }
         } else {
           int dictionaryValue = readSurrogatesFromColumnBlock(blockChunkHolder, index, pageIndex,
@@ -351,21 +350,17 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     DataType msrType;
     for (int i = 0; i < msrColEvalutorInfoList.size(); i++) {
       MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = msrColEvalutorInfoList.get(i);
-      switch (msrColumnEvalutorInfo.getType()) {
-        case SHORT:
-          msrType = DataType.SHORT;
-          break;
-        case INT:
-          msrType = DataType.INT;
-          break;
-        case LONG:
-          msrType = DataType.LONG;
-          break;
-        case DECIMAL:
-          msrType = DataType.DECIMAL;
-          break;
-        default:
-          msrType = DataType.DOUBLE;
+      DataType dataType = msrColumnEvalutorInfo.getType();
+      if (dataType == DataTypes.SHORT) {
+        msrType = DataTypes.SHORT;
+      } else if (dataType == DataTypes.INT) {
+        msrType = DataTypes.INT;
+      } else if (dataType == DataTypes.LONG) {
+        msrType = DataTypes.LONG;
+      } else if (dataType == DataTypes.DECIMAL) {
+        msrType = DataTypes.DECIMAL;
+      } else {
+        msrType = DataTypes.DOUBLE;
       }
       // add default value for the measure in case filter measure is not present
       // in the current block measure list
@@ -381,30 +376,24 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       ColumnPage columnPage =
           blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]]
               .convertToColumnPage(pageIndex);
-      switch (msrType) {
-        case SHORT:
-          msrValue = (short) columnPage.getLong(index);
-          break;
-        case INT:
-          msrValue = (int) columnPage.getLong(index);
-          break;
-        case LONG:
-          msrValue = columnPage.getLong(index);
-          break;
-        case DECIMAL:
-          BigDecimal bigDecimalValue = columnPage.getDecimal(index);
-          if (null != bigDecimalValue &&
-              msrColumnEvalutorInfo.getCarbonColumn().getColumnSchema().getScale() >
-                  bigDecimalValue.scale()) {
-            bigDecimalValue =
-                bigDecimalValue.setScale(
-                    msrColumnEvalutorInfo.getCarbonColumn().getColumnSchema().getScale(),
-                    RoundingMode.HALF_UP);
-          }
-          msrValue = bigDecimalValue;
-          break;
-        default:
-          msrValue = columnPage.getDouble(index);
+      if (msrType == DataTypes.SHORT) {
+        msrValue = (short) columnPage.getLong(index);
+      } else if (msrType == DataTypes.INT) {
+        msrValue = (int) columnPage.getLong(index);
+      } else if (msrType == DataTypes.LONG) {
+        msrValue = columnPage.getLong(index);
+      } else if (msrType == DataTypes.DECIMAL) {
+        BigDecimal bigDecimalValue = columnPage.getDecimal(index);
+        if (null != bigDecimalValue
+            && msrColumnEvalutorInfo.getCarbonColumn().getColumnSchema().getScale()
+            > bigDecimalValue.scale()) {
+          bigDecimalValue = bigDecimalValue
+              .setScale(msrColumnEvalutorInfo.getCarbonColumn().getColumnSchema().getScale(),
+                  RoundingMode.HALF_UP);
+        }
+        msrValue = bigDecimalValue;
+      } else {
+        msrValue = columnPage.getDouble(index);
       }
       record[msrColumnEvalutorInfo.getRowIndex()] =
           columnPage.getNullBits().get(index) ? null : msrValue;
@@ -444,13 +433,12 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    */
   private Object getFilterActualValueFromDirectDictionaryValue(
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int dictionaryValue) {
-    switch (dimColumnEvaluatorInfo.getDimension().getDataType()) {
-      case DATE:
-        return dateDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
-      case TIMESTAMP:
-        return timestampDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
-      default:
-        throw new RuntimeException("Invalid data type for dierct dictionary");
+    if (dimColumnEvaluatorInfo.getDimension().getDataType() == DataTypes.DATE) {
+      return dateDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
+    } else if (dimColumnEvaluatorInfo.getDimension().getDataType() == DataTypes.TIMESTAMP) {
+      return timestampDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
+    } else {
+      throw new RuntimeException("Invalid data type for dierct dictionary");
     }
   }
 
@@ -533,8 +521,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
-      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.ARRAY
-          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.STRUCT) {
+      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.ARRAY
+          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.STRUCT) {
         if (null == blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]) {
           blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]] =
               blockChunkHolder.getDataBlock()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index 67fe848..5e0bb41 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -275,7 +276,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       } else {
         defaultValue = ByteUtil.toBytes(key);
       }
-    } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataType.STRING) {
+    } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataTypes.STRING) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
     BitSet bitSet = null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index 8db37c4..74162f6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -277,7 +278,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       } else {
         defaultValue = ByteUtil.toBytes(key);
       }
-    } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataType.STRING) {
+    } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataTypes.STRING) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
     BitSet bitSet = null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
index 6dd59fd..bd78e08 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@ -22,7 +22,7 @@ import java.util.SortedMap;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -122,8 +122,8 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
         metadata.setExpression(leftExp);
         metadata.setIncludeFilter(isIncludeFilter);
         metadata.setTableProvider(tableProvider);
-        if (columnExpression.getDataType().equals(DataType.TIMESTAMP) ||
-            columnExpression.getDataType().equals(DataType.DATE)) {
+        if (columnExpression.getDataType().equals(DataTypes.TIMESTAMP) ||
+            columnExpression.getDataType().equals(DataTypes.DATE)) {
           isExpressionResolve = true;
         } else {
           // if imei=imei comes in filter condition then we need to
@@ -169,9 +169,9 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
       } else if ((null != columnList.get(0).getDimension()) && (
           columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) && !(
               columnList.get(0).getDimension().getDataType()
-                  == org.apache.carbondata.core.metadata.datatype.DataType.STRUCT
+                  == org.apache.carbondata.core.metadata.datatype.DataTypes.STRUCT
                   || columnList.get(0).getDimension().getDataType()
-                  == org.apache.carbondata.core.metadata.datatype.DataType.ARRAY))) {
+                  == org.apache.carbondata.core.metadata.datatype.DataTypes.ARRAY))) {
         dimColResolvedFilterInfo.setFilterValues(FilterUtil
             .getFilterListForAllValues(absoluteTableIdentifier, exp, columnList.get(0),
                 isIncludeFilter, tableProvider));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
index c4df001..425c725 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -167,7 +167,7 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
     for (ExpressionResult result : listOfExpressionResults) {
       try {
         if (result.getString() == null) {
-          if (result.getDataType() == DataType.STRING) {
+          if (result.getDataType() == DataTypes.STRING) {
             filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
           } else {
             filterValuesList.add(CarbonCommonConstants.EMPTY_BYTE_ARRAY);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
index 221d243..496cf6a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
@@ -96,7 +97,7 @@ public class CustomTypeDictionaryVisitor implements ResolvedFilterInfoVisitorInt
       List<Integer> surrogates, DirectDictionaryGenerator directDictionaryGenerator,
       DataType dataType) {
     String timeFormat = null;
-    if (dataType == DataType.DATE) {
+    if (dataType == DataTypes.DATE) {
       timeFormat = CarbonProperties.getInstance()
           .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
               CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
index 5921326..85e6cef 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
@@ -22,6 +22,7 @@ import java.text.DateFormat;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 
 import org.apache.commons.lang.StringUtils;
@@ -47,29 +48,28 @@ public class PartitionUtil {
     if (data == null) {
       return null;
     }
-    if (actualDataType != DataType.STRING && StringUtils.isEmpty(data)) {
+    if (actualDataType != DataTypes.STRING && StringUtils.isEmpty(data)) {
       return null;
     }
     try {
-      switch (actualDataType) {
-        case STRING:
-          return data;
-        case INT:
-          return Integer.parseInt(data);
-        case SHORT:
-          return Short.parseShort(data);
-        case DOUBLE:
-          return Double.parseDouble(data);
-        case LONG:
-          return Long.parseLong(data);
-        case DATE:
-          return dateFormatter.parse(data).getTime();
-        case TIMESTAMP:
-          return timestampFormatter.parse(data).getTime();
-        case DECIMAL:
-          return new BigDecimal(data);
-        default:
-          return data;
+      if (actualDataType == DataTypes.STRING) {
+        return data;
+      } else if (actualDataType == DataTypes.INT) {
+        return Integer.parseInt(data);
+      } else if (actualDataType == DataTypes.SHORT) {
+        return Short.parseShort(data);
+      } else if (actualDataType == DataTypes.DOUBLE) {
+        return Double.parseDouble(data);
+      } else if (actualDataType == DataTypes.LONG) {
+        return Long.parseLong(data);
+      } else if (actualDataType == DataTypes.DATE) {
+        return dateFormatter.parse(data).getTime();
+      } else if (actualDataType == DataTypes.TIMESTAMP) {
+        return timestampFormatter.parse(data).getTime();
+      } else if (actualDataType == DataTypes.DECIMAL) {
+        return new BigDecimal(data);
+      } else {
+        return data;
       }
     } catch (Exception ex) {
       return null;
@@ -86,28 +86,26 @@ public class PartitionUtil {
     if (data == null) {
       return null;
     }
-    if (actualDataType != DataType.STRING && StringUtils.isEmpty(data)) {
+    if (actualDataType != DataTypes.STRING && StringUtils.isEmpty(data)) {
       return null;
     }
     try {
-      switch (actualDataType) {
-        case STRING:
-          return data;
-        case INT:
-          return Integer.parseInt(data);
-        case SHORT:
-          return Short.parseShort(data);
-        case DOUBLE:
-          return Double.parseDouble(data);
-        case LONG:
-          return Long.parseLong(data);
-        case DATE:
-        case TIMESTAMP:
-          return Long.parseLong(data) / 1000;
-        case DECIMAL:
-          return new BigDecimal(data);
-        default:
-          return data;
+      if (actualDataType == DataTypes.STRING) {
+        return data;
+      } else if (actualDataType == DataTypes.INT) {
+        return Integer.parseInt(data);
+      } else if (actualDataType == DataTypes.SHORT) {
+        return Short.parseShort(data);
+      } else if (actualDataType == DataTypes.DOUBLE) {
+        return Double.parseDouble(data);
+      } else if (actualDataType == DataTypes.LONG) {
+        return Long.parseLong(data);
+      } else if (actualDataType == DataTypes.DATE || actualDataType == DataTypes.TIMESTAMP) {
+        return Long.parseLong(data) / 1000;
+      } else if (actualDataType == DataTypes.DECIMAL) {
+        return new BigDecimal(data);
+      } else {
+        return data;
       }
     } catch (Exception ex) {
       return null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
index 50f609a..e61aefd 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -50,7 +51,7 @@ public class RangePartitioner implements Partitioner {
     DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
     numPartitions = values.size();
     bounds = new Object[numPartitions];
-    if (partitionColumnDataType == DataType.STRING) {
+    if (partitionColumnDataType == DataTypes.STRING) {
       for (int i = 0; i < numPartitions; i++) {
         bounds[i] = ByteUtil.toBytes(values.get(i));
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
index c145055..9b01e1f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
@@ -21,6 +21,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.apache.spark.sql.types.Decimal;
 
@@ -306,17 +307,16 @@ public class MeasureDataVectorProcessor {
   public static class MeasureVectorFillerFactory {
 
     public static MeasureVectorFiller getMeasureVectorFiller(DataType dataType) {
-      switch (dataType) {
-        case SHORT:
-          return new ShortMeasureVectorFiller();
-        case INT:
-          return new IntegralMeasureVectorFiller();
-        case LONG:
-          return new LongMeasureVectorFiller();
-        case DECIMAL:
-          return new DecimalMeasureVectorFiller();
-        default:
-          return new DefaultMeasureVectorFiller();
+      if (dataType == DataTypes.SHORT) {
+        return new ShortMeasureVectorFiller();
+      } else if (dataType == DataTypes.INT) {
+        return new IntegralMeasureVectorFiller();
+      } else if (dataType == DataTypes.LONG) {
+        return new LongMeasureVectorFiller();
+      } else if (dataType == DataTypes.DECIMAL) {
+        return new DecimalMeasureVectorFiller();
+      } else {
+        return new DefaultMeasureVectorFiller();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index c1cf8e8..1020e5b 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.metadata.blocklet.index.BlockletBTreeIndex;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.reader.CarbonIndexFileReader;
@@ -352,27 +353,27 @@ public abstract class AbstractDataFileFooterConverter {
       org.apache.carbondata.format.DataType dataTypeThrift) {
     switch (dataTypeThrift) {
       case STRING:
-        return DataType.STRING;
+        return DataTypes.STRING;
       case SHORT:
-        return DataType.SHORT;
+        return DataTypes.SHORT;
       case INT:
-        return DataType.INT;
+        return DataTypes.INT;
       case LONG:
-        return DataType.LONG;
+        return DataTypes.LONG;
       case DOUBLE:
-        return DataType.DOUBLE;
+        return DataTypes.DOUBLE;
       case DECIMAL:
-        return DataType.DECIMAL;
+        return DataTypes.DECIMAL;
       case DATE:
-        return DataType.DATE;
+        return DataTypes.DATE;
       case TIMESTAMP:
-        return DataType.TIMESTAMP;
+        return DataTypes.TIMESTAMP;
       case ARRAY:
-        return DataType.ARRAY;
+        return DataTypes.ARRAY;
       case STRUCT:
-        return DataType.STRUCT;
+        return DataTypes.STRUCT;
       default:
-        return DataType.STRING;
+        return DataTypes.STRING;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
index ece97fe..2c9ef66 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.page.EncodedTablePage;
 import org.apache.carbondata.core.datastore.page.statistics.TablePageStatistics;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.index.BlockIndexInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.format.BlockIndex;
@@ -340,33 +341,30 @@ public class CarbonMetadataUtil {
     return CarbonMetadataUtil.getDataChunk3(dataChunksList);
   }
 
-  public static int compareMeasureData(byte[] first, byte[] second, DataType dataType) {
+  private static int compareMeasureData(byte[] first, byte[] second, DataType dataType) {
     ByteBuffer firstBuffer = null;
     ByteBuffer secondBuffer = null;
-    switch (dataType) {
-      case DOUBLE:
-        firstBuffer = ByteBuffer.allocate(8);
-        firstBuffer.put(first);
-        secondBuffer = ByteBuffer.allocate(8);
-        secondBuffer.put(second);
-        firstBuffer.flip();
-        secondBuffer.flip();
-        return (int) (firstBuffer.getDouble() - secondBuffer.getDouble());
-      case LONG:
-      case INT:
-      case SHORT:
-        firstBuffer = ByteBuffer.allocate(8);
-        firstBuffer.put(first);
-        secondBuffer = ByteBuffer.allocate(8);
-        secondBuffer.put(second);
-        firstBuffer.flip();
-        secondBuffer.flip();
-        return (int) (firstBuffer.getLong() - secondBuffer.getLong());
-      case DECIMAL:
-        return DataTypeUtil.byteToBigDecimal(first)
-            .compareTo(DataTypeUtil.byteToBigDecimal(second));
-      default:
-        throw new IllegalArgumentException("Invalid data type");
+    if (dataType == DataTypes.DOUBLE) {
+      firstBuffer = ByteBuffer.allocate(8);
+      firstBuffer.put(first);
+      secondBuffer = ByteBuffer.allocate(8);
+      secondBuffer.put(second);
+      firstBuffer.flip();
+      secondBuffer.flip();
+      return (int) (firstBuffer.getDouble() - secondBuffer.getDouble());
+    } else if (dataType == DataTypes.LONG || dataType == DataTypes.INT
+        || dataType == DataTypes.SHORT) {
+      firstBuffer = ByteBuffer.allocate(8);
+      firstBuffer.put(first);
+      secondBuffer = ByteBuffer.allocate(8);
+      secondBuffer.put(second);
+      firstBuffer.flip();
+      secondBuffer.flip();
+      return (int) (firstBuffer.getLong() - secondBuffer.getLong());
+    } else if (dataType == DataTypes.DECIMAL) {
+      return DataTypeUtil.byteToBigDecimal(first).compareTo(DataTypeUtil.byteToBigDecimal(second));
+    } else {
+      throw new IllegalArgumentException("Invalid data type");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index b5b3e63..2924c09 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -62,6 +62,7 @@ import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -853,23 +854,6 @@ public final class CarbonUtil {
     return false;
   }
 
-  /**
-   * below method is to check whether it is complex data type
-   *
-   * @param dataType data type to be searched
-   * @return if data type is present
-   */
-  public static boolean hasComplexDataType(DataType dataType) {
-    switch (dataType) {
-      case ARRAY:
-      case STRUCT:
-      case MAP:
-        return true;
-      default:
-        return false;
-    }
-  }
-
   public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
@@ -900,7 +884,7 @@ public final class CarbonUtil {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
-          CarbonUtil.hasComplexDataType(queryDimensions[i].getDimension().getDataType());
+          queryDimensions[i].getDimension().getDataType().isComplexType();
     }
     return dictionaryEncodingArray;
   }
@@ -1619,16 +1603,15 @@ public final class CarbonUtil {
    * @return format
    */
   public static String getFormatFromProperty(DataType dataType) {
-    switch (dataType) {
-      case DATE:
-        return CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-            CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
-      case TIMESTAMP:
-        return CarbonProperties.getInstance()
-            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-      default:
-        return null;
+    if (dataType.equals(DataTypes.DATE)) {
+      return CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+    } else if (dataType.equals(DataTypes.TIMESTAMP)) {
+      return CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+              CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+    } else {
+      return null;
     }
   }
 
@@ -1955,42 +1938,40 @@ public final class CarbonUtil {
    */
   public static byte[] getValueAsBytes(DataType dataType, Object value) {
     ByteBuffer b;
-    switch (dataType) {
-      case BYTE:
-        b = ByteBuffer.allocate(8);
-        b.putLong((byte) value);
-        b.flip();
-        return b.array();
-      case SHORT:
-        b = ByteBuffer.allocate(8);
-        b.putLong((short) value);
-        b.flip();
-        return b.array();
-      case INT:
-        b = ByteBuffer.allocate(8);
-        b.putLong((int) value);
-        b.flip();
-        return b.array();
-      case LONG:
-        b = ByteBuffer.allocate(8);
-        b.putLong((long) value);
-        b.flip();
-        return b.array();
-      case DOUBLE:
-        b = ByteBuffer.allocate(8);
-        b.putDouble((double) value);
-        b.flip();
-        return b.array();
-      case DECIMAL:
-        return DataTypeUtil.bigDecimalToByte((BigDecimal)value);
-      case BYTE_ARRAY:
-        return (byte[]) value;
-      case STRING:
-      case TIMESTAMP:
-      case DATE:
-        return (byte[]) value;
-      default:
-        throw new IllegalArgumentException("Invalid data type: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((byte) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.SHORT) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((short) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.INT) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((int) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.LONG) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((long) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.DOUBLE) {
+      b = ByteBuffer.allocate(8);
+      b.putDouble((double) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.DECIMAL) {
+      return DataTypeUtil.bigDecimalToByte((BigDecimal) value);
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
+      return (byte[]) value;
+    } else if (dataType == DataTypes.STRING || dataType == DataTypes.TIMESTAMP ||
+        dataType == DataTypes.DATE) {
+      return (byte[]) value;
+    } else {
+      throw new IllegalArgumentException("Invalid data type: " + dataType);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
index 2e65983..b8cd59d 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
@@ -37,13 +37,12 @@ import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 
-
-
 public final class DataTypeUtil {
 
   /**
@@ -73,19 +72,19 @@ public final class DataTypeUtil {
 
   static {
     dataTypeDisplayNames = new HashMap<String, String>(16);
-    dataTypeDisplayNames.put(DataType.DATE.toString(), DataType.DATE.getName());
-    dataTypeDisplayNames.put(DataType.LONG.toString(), DataType.LONG.getName());
-    dataTypeDisplayNames.put(DataType.INT.toString(), DataType.INT.getName());
-    dataTypeDisplayNames.put(DataType.FLOAT.toString(), DataType.FLOAT.getName());
-    dataTypeDisplayNames.put(DataType.BOOLEAN.toString(), DataType.BOOLEAN.getName());
-    dataTypeDisplayNames.put(DataType.NULL.toString(), DataType.NULL.getName());
-    dataTypeDisplayNames.put(DataType.DECIMAL.toString(), DataType.DECIMAL.getName());
-    dataTypeDisplayNames.put(DataType.ARRAY.toString(), DataType.ARRAY.getName());
-    dataTypeDisplayNames.put(DataType.STRUCT.toString(), DataType.STRUCT.getName());
-    dataTypeDisplayNames.put(DataType.TIMESTAMP.toString(), DataType.TIMESTAMP.getName());
-    dataTypeDisplayNames.put(DataType.DATE.toString(), DataType.DATE.getName());
-    dataTypeDisplayNames.put(DataType.SHORT.toString(), DataType.SHORT.getName());
-    dataTypeDisplayNames.put(DataType.STRING.toString(), DataType.STRING.getName());
+    dataTypeDisplayNames.put(DataTypes.DATE.toString(), DataTypes.DATE.getName());
+    dataTypeDisplayNames.put(DataTypes.LONG.toString(), DataTypes.LONG.getName());
+    dataTypeDisplayNames.put(DataTypes.INT.toString(), DataTypes.INT.getName());
+    dataTypeDisplayNames.put(DataTypes.FLOAT.toString(), DataTypes.FLOAT.getName());
+    dataTypeDisplayNames.put(DataTypes.BOOLEAN.toString(), DataTypes.BOOLEAN.getName());
+    dataTypeDisplayNames.put(DataTypes.NULL.toString(), DataTypes.NULL.getName());
+    dataTypeDisplayNames.put(DataTypes.DECIMAL.toString(), DataTypes.DECIMAL.getName());
+    dataTypeDisplayNames.put(DataTypes.ARRAY.toString(), DataTypes.ARRAY.getName());
+    dataTypeDisplayNames.put(DataTypes.STRUCT.toString(), DataTypes.STRUCT.getName());
+    dataTypeDisplayNames.put(DataTypes.TIMESTAMP.toString(), DataTypes.TIMESTAMP.getName());
+    dataTypeDisplayNames.put(DataTypes.DATE.toString(), DataTypes.DATE.getName());
+    dataTypeDisplayNames.put(DataTypes.SHORT.toString(), DataTypes.SHORT.getName());
+    dataTypeDisplayNames.put(DataTypes.STRING.toString(), DataTypes.STRING.getName());
   }
 
   /**
@@ -103,23 +102,22 @@ public final class DataTypeUtil {
    */
   public static Object getMeasureValueBasedOnDataType(String msrValue, DataType dataType,
       CarbonMeasure carbonMeasure) {
-    switch (dataType) {
-      case DECIMAL:
-        BigDecimal bigDecimal =
-            new BigDecimal(msrValue).setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
-        return normalizeDecimalValue(bigDecimal, carbonMeasure.getPrecision());
-      case SHORT:
-        return Short.parseShort(msrValue);
-      case INT:
-        return Integer.parseInt(msrValue);
-      case LONG:
-        return Long.valueOf(msrValue);
-      default:
-        Double parsedValue = Double.valueOf(msrValue);
-        if (Double.isInfinite(parsedValue) || Double.isNaN(parsedValue)) {
-          return null;
-        }
-        return parsedValue;
+    if (dataType == DataTypes.DECIMAL) {
+      BigDecimal bigDecimal =
+          new BigDecimal(msrValue).setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
+      return normalizeDecimalValue(bigDecimal, carbonMeasure.getPrecision());
+    } else if (dataType == DataTypes.SHORT) {
+      return Short.parseShort(msrValue);
+    } else if (dataType == DataTypes.INT) {
+      return Integer.parseInt(msrValue);
+    } else if (dataType == DataTypes.LONG) {
+      return Long.valueOf(msrValue);
+    } else {
+      Double parsedValue = Double.valueOf(msrValue);
+      if (Double.isInfinite(parsedValue) || Double.isNaN(parsedValue)) {
+        return null;
+      }
+      return parsedValue;
     }
   }
 
@@ -128,42 +126,40 @@ public final class DataTypeUtil {
       return null;
     }
     ByteBuffer bb = ByteBuffer.wrap(data);
-    switch (dataType) {
-      case SHORT:
-        return (short)bb.getLong();
-      case INT:
-        return (int)bb.getLong();
-      case LONG:
-        return bb.getLong();
-      case DECIMAL:
-        return byteToBigDecimal(data);
-      default:
-        return bb.getDouble();
+    if (dataType == DataTypes.SHORT) {
+      return (short) bb.getLong();
+    } else if (dataType == DataTypes.INT) {
+      return (int) bb.getLong();
+    } else if (dataType == DataTypes.LONG) {
+      return bb.getLong();
+    } else if (dataType == DataTypes.DECIMAL) {
+      return byteToBigDecimal(data);
+    } else {
+      return bb.getDouble();
     }
   }
 
   public static Object getMeasureObjectBasedOnDataType(ColumnPage measurePage, int index,
       DataType dataType, CarbonMeasure carbonMeasure) {
-    switch (dataType) {
-      case SHORT:
-        return (short)measurePage.getLong(index);
-      case INT:
-        return (int)measurePage.getLong(index);
-      case LONG:
-        return measurePage.getLong(index);
-      case DECIMAL:
-        BigDecimal bigDecimalMsrValue = measurePage.getDecimal(index);
-        if (null != bigDecimalMsrValue && carbonMeasure.getScale() > bigDecimalMsrValue.scale()) {
-          bigDecimalMsrValue =
-              bigDecimalMsrValue.setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
-        }
-        if (null != bigDecimalMsrValue) {
-          return normalizeDecimalValue(bigDecimalMsrValue, carbonMeasure.getPrecision());
-        } else {
-          return bigDecimalMsrValue;
-        }
-      default:
-        return measurePage.getDouble(index);
+    if (dataType == DataTypes.SHORT) {
+      return (short) measurePage.getLong(index);
+    } else if (dataType == DataTypes.INT) {
+      return (int) measurePage.getLong(index);
+    } else if (dataType == DataTypes.LONG) {
+      return measurePage.getLong(index);
+    } else if (dataType == DataTypes.DECIMAL) {
+      BigDecimal bigDecimalMsrValue = measurePage.getDecimal(index);
+      if (null != bigDecimalMsrValue && carbonMeasure.getScale() > bigDecimalMsrValue.scale()) {
+        bigDecimalMsrValue =
+            bigDecimalMsrValue.setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
+      }
+      if (null != bigDecimalMsrValue) {
+        return normalizeDecimalValue(bigDecimalMsrValue, carbonMeasure.getPrecision());
+      } else {
+        return bigDecimalMsrValue;
+      }
+    } else {
+      return measurePage.getDouble(index);
     }
   }
 
@@ -245,38 +241,38 @@ public final class DataTypeUtil {
     DataType dataType = null;
     switch (dataTypeStr) {
       case "DATE":
-        dataType = DataType.DATE;
+        dataType = DataTypes.DATE;
         break;
       case "TIMESTAMP":
-        dataType = DataType.TIMESTAMP;
+        dataType = DataTypes.TIMESTAMP;
         break;
       case "STRING":
-        dataType = DataType.STRING;
+        dataType = DataTypes.STRING;
         break;
       case "INT":
-        dataType = DataType.INT;
+        dataType = DataTypes.INT;
         break;
       case "SMALLINT":
-        dataType = DataType.SHORT;
+        dataType = DataTypes.SHORT;
         break;
       case "LONG":
-        dataType = DataType.LONG;
+        dataType = DataTypes.LONG;
         break;
       case "DOUBLE":
-        dataType = DataType.DOUBLE;
+        dataType = DataTypes.DOUBLE;
         break;
       case "DECIMAL":
-        dataType = DataType.DECIMAL;
+        dataType = DataTypes.DECIMAL;
         break;
       case "ARRAY":
-        dataType = DataType.ARRAY;
+        dataType = DataTypes.ARRAY;
         break;
       case "STRUCT":
-        dataType = DataType.STRUCT;
+        dataType = DataTypes.STRUCT;
         break;
       case "MAP":
       default:
-        dataType = DataType.STRING;
+        dataType = DataTypes.STRING;
     }
     return dataType;
   }
@@ -307,61 +303,60 @@ public final class DataTypeUtil {
       return null;
     }
     try {
-      switch (actualDataType) {
-        case INT:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Integer.parseInt(data);
-        case SHORT:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Short.parseShort(data);
-        case FLOAT:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Float.parseFloat(data);
-        case DOUBLE:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Double.parseDouble(data);
-        case LONG:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Long.parseLong(data);
-        case DATE:
-          if (data.isEmpty()) {
-            return null;
-          }
-          try {
-            Date dateToStr = dateformatter.get().parse(data);
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
-            return null;
-          }
-        case TIMESTAMP:
-          if (data.isEmpty()) {
-            return null;
-          }
-          try {
-            Date dateToStr = timeStampformatter.get().parse(data);
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
-            return null;
-          }
-        case DECIMAL:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return converter.convertToDecimal(data);
-        default:
-          return converter.convertFromStringToUTF8String(data);
+      if (actualDataType == DataTypes.INT) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        return Integer.parseInt(data);
+      } else if (actualDataType == DataTypes.SHORT) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        return Short.parseShort(data);
+      } else if (actualDataType == DataTypes.FLOAT) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        return Float.parseFloat(data);
+      } else if (actualDataType == DataTypes.DOUBLE) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        return Double.parseDouble(data);
+      } else if (actualDataType == DataTypes.LONG) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        return Long.parseLong(data);
+      } else if (actualDataType == DataTypes.DATE) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        try {
+          Date dateToStr = dateformatter.get().parse(data);
+          return dateToStr.getTime() * 1000;
+        } catch (ParseException e) {
+          LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
+          return null;
+        }
+      } else if (actualDataType == DataTypes.TIMESTAMP) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        try {
+          Date dateToStr = timeStampformatter.get().parse(data);
+          return dateToStr.getTime() * 1000;
+        } catch (ParseException e) {
+          LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
+          return null;
+        }
+      } else if (actualDataType == DataTypes.DECIMAL) {
+        if (data.isEmpty()) {
+          return null;
+        }
+        return converter.convertToDecimal(data);
+      } else {
+        return converter.convertFromStringToUTF8String(data);
       }
     } catch (NumberFormatException ex) {
       LOGGER.error("Problem while converting data type" + data);
@@ -371,33 +366,32 @@ public final class DataTypeUtil {
 
   public static byte[] getBytesBasedOnDataTypeForNoDictionaryColumn(String dimensionValue,
       DataType actualDataType, String dateFormat) {
-    switch (actualDataType) {
-      case STRING:
-        return ByteUtil.toBytes(dimensionValue);
-      case BOOLEAN:
-        return ByteUtil.toBytes(Boolean.parseBoolean(dimensionValue));
-      case SHORT:
-        return ByteUtil.toBytes(Short.parseShort(dimensionValue));
-      case INT:
-        return ByteUtil.toBytes(Integer.parseInt(dimensionValue));
-      case LONG:
-        return ByteUtil.toBytes(Long.parseLong(dimensionValue));
-      case TIMESTAMP:
-        Date dateToStr = null;
-        DateFormat dateFormatter = null;
-        try {
-          if (null != dateFormat) {
-            dateFormatter = new SimpleDateFormat(dateFormat);
-          } else {
-            dateFormatter = timeStampformatter.get();
-          }
-          dateToStr = dateFormatter.parse(dimensionValue);
-          return ByteUtil.toBytes(dateToStr.getTime());
-        } catch (ParseException e) {
-          throw new NumberFormatException(e.getMessage());
+    if (actualDataType == DataTypes.STRING) {
+      return ByteUtil.toBytes(dimensionValue);
+    } else if (actualDataType == DataTypes.BOOLEAN) {
+      return ByteUtil.toBytes(Boolean.parseBoolean(dimensionValue));
+    } else if (actualDataType == DataTypes.SHORT) {
+      return ByteUtil.toBytes(Short.parseShort(dimensionValue));
+    } else if (actualDataType == DataTypes.INT) {
+      return ByteUtil.toBytes(Integer.parseInt(dimensionValue));
+    } else if (actualDataType == DataTypes.LONG) {
+      return ByteUtil.toBytes(Long.parseLong(dimensionValue));
+    } else if (actualDataType == DataTypes.TIMESTAMP) {
+      Date dateToStr = null;
+      DateFormat dateFormatter = null;
+      try {
+        if (null != dateFormat) {
+          dateFormatter = new SimpleDateFormat(dateFormat);
+        } else {
+          dateFormatter = timeStampformatter.get();
         }
-      default:
-        return ByteUtil.toBytes(dimensionValue);
+        dateToStr = dateFormatter.parse(dimensionValue);
+        return ByteUtil.toBytes(dateToStr.getTime());
+      } catch (ParseException e) {
+        throw new NumberFormatException(e.getMessage());
+      }
+    } else {
+      return ByteUtil.toBytes(dimensionValue);
     }
   }
 
@@ -417,21 +411,20 @@ public final class DataTypeUtil {
       return null;
     }
     try {
-      switch (actualDataType) {
-        case STRING:
-          return getDataTypeConverter().convertFromByteToUTF8String(dataInBytes);
-        case BOOLEAN:
-          return ByteUtil.toBoolean(dataInBytes);
-        case SHORT:
-          return ByteUtil.toShort(dataInBytes, 0, dataInBytes.length);
-        case INT:
-          return ByteUtil.toInt(dataInBytes, 0, dataInBytes.length);
-        case LONG:
-          return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length);
-        case TIMESTAMP:
-          return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length) * 1000L;
-        default:
-          return ByteUtil.toString(dataInBytes, 0, dataInBytes.length);
+      if (actualDataType == DataTypes.STRING) {
+        return getDataTypeConverter().convertFromByteToUTF8String(dataInBytes);
+      } else if (actualDataType == DataTypes.BOOLEAN) {
+        return ByteUtil.toBoolean(dataInBytes);
+      } else if (actualDataType == DataTypes.SHORT) {
+        return ByteUtil.toShort(dataInBytes, 0, dataInBytes.length);
+      } else if (actualDataType == DataTypes.INT) {
+        return ByteUtil.toInt(dataInBytes, 0, dataInBytes.length);
+      } else if (actualDataType == DataTypes.LONG) {
+        return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length);
+      } else if (actualDataType == DataTypes.TIMESTAMP) {
+        return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length) * 1000L;
+      } else {
+        return ByteUtil.toString(dataInBytes, 0, dataInBytes.length);
       }
     } catch (Throwable ex) {
       String data = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
@@ -457,68 +450,67 @@ public final class DataTypeUtil {
       return null;
     }
     try {
-      switch (dimension.getDataType()) {
-        case INT:
-          String data1 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data1.isEmpty()) {
-            return null;
-          }
-          return Integer.parseInt(data1);
-        case SHORT:
-          String data2 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data2.isEmpty()) {
-            return null;
-          }
-          return Short.parseShort(data2);
-        case DOUBLE:
-          String data3 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data3.isEmpty()) {
-            return null;
-          }
-          return Double.parseDouble(data3);
-        case LONG:
-          String data4 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data4.isEmpty()) {
-            return null;
-          }
-          return Long.parseLong(data4);
-        case DATE:
-          String data5 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data5.isEmpty()) {
-            return null;
-          }
-          try {
-            Date dateToStr = dateformatter.get().parse(data5);
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
-            return null;
-          }
-
-        case TIMESTAMP:
-          String data6 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data6.isEmpty()) {
-            return null;
-          }
-          try {
-            Date dateToStr = timeStampformatter.get().parse(data6);
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
-            return null;
-          }
-        case DECIMAL:
-          String data7 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
-          if (data7.isEmpty()) {
-            return null;
-          }
-          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data7);
-          if (dimension.getColumnSchema().getScale() > javaDecVal.scale()) {
-            javaDecVal = javaDecVal.setScale(dimension.getColumnSchema().getScale());
-          }
-          return getDataTypeConverter().convertToDecimal(javaDecVal);
-        default:
-          return getDataTypeConverter().convertFromByteToUTF8String(dataInBytes);
+      DataType dataType = dimension.getDataType();
+      if (dataType == DataTypes.INT) {
+        String data1 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data1.isEmpty()) {
+          return null;
+        }
+        return Integer.parseInt(data1);
+      } else if (dataType == DataTypes.SHORT) {
+        String data2 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data2.isEmpty()) {
+          return null;
+        }
+        return Short.parseShort(data2);
+      } else if (dataType == DataTypes.DOUBLE) {
+        String data3 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data3.isEmpty()) {
+          return null;
+        }
+        return Double.parseDouble(data3);
+      } else if (dataType == DataTypes.LONG) {
+        String data4 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data4.isEmpty()) {
+          return null;
+        }
+        return Long.parseLong(data4);
+      } else if (dataType == DataTypes.DATE) {
+        String data5 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data5.isEmpty()) {
+          return null;
+        }
+        try {
+          Date dateToStr = dateformatter.get().parse(data5);
+          return dateToStr.getTime() * 1000;
+        } catch (ParseException e) {
+          LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
+          return null;
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        String data6 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data6.isEmpty()) {
+          return null;
+        }
+        try {
+          Date dateToStr = timeStampformatter.get().parse(data6);
+          return dateToStr.getTime() * 1000;
+        } catch (ParseException e) {
+          LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
+          return null;
+        }
+      } else if (dataType == DataTypes.DECIMAL) {
+        String data7 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
+        if (data7.isEmpty()) {
+          return null;
+        }
+        java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data7);
+        if (dimension.getColumnSchema().getScale() > javaDecVal.scale()) {
+          javaDecVal = javaDecVal.setScale(dimension.getColumnSchema().getScale());
+        }
+        return getDataTypeConverter().convertToDecimal(javaDecVal);
+      } else {
+        return getDataTypeConverter().convertFromByteToUTF8String(dataInBytes);
       }
     } catch (NumberFormatException ex) {
       String data = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
@@ -542,18 +534,14 @@ public final class DataTypeUtil {
     }
     try {
       Object parsedValue = null;
-      switch (actualDataType) {
-        case SHORT:
-          parsedValue = Short.parseShort(data);
-          break;
-        case INT:
-          parsedValue = Integer.parseInt(data);
-          break;
-        case LONG:
-          parsedValue = Long.parseLong(data);
-          break;
-        default:
-          return data;
+      if (actualDataType == DataTypes.SHORT) {
+        parsedValue = Short.parseShort(data);
+      } else if (actualDataType == DataTypes.INT) {
+        parsedValue = Integer.parseInt(data);
+      } else if (actualDataType == DataTypes.LONG) {
+        parsedValue = Long.parseLong(data);
+      } else {
+        return data;
       }
       if (null != parsedValue) {
         return data;
@@ -577,19 +565,16 @@ public final class DataTypeUtil {
       Object parsedValue = null;
       // validation will not be done for timestamp datatype as for timestamp direct dictionary
       // is generated. No dictionary file is created for timestamp datatype column
-      switch (dimension.getDataType()) {
-        case DECIMAL:
-          return parseStringToBigDecimal(value, dimension);
-        case SHORT:
-        case INT:
-        case LONG:
-          parsedValue = normalizeIntAndLongValues(value, dimension.getDataType());
-          break;
-        case DOUBLE:
-          parsedValue = Double.parseDouble(value);
-          break;
-        default:
-          return value;
+      DataType dataType = dimension.getDataType();
+      if (dataType == DataTypes.DECIMAL) {
+        return parseStringToBigDecimal(value, dimension);
+      } else if (dataType == DataTypes.SHORT || dataType == DataTypes.INT ||
+          dataType == DataTypes.LONG) {
+        parsedValue = normalizeIntAndLongValues(value, dimension.getDataType());
+      } else if (dataType == DataTypes.DOUBLE) {
+        parsedValue = Double.parseDouble(value);
+      } else {
+        return value;
       }
       if (null != parsedValue) {
         return value;
@@ -613,23 +598,19 @@ public final class DataTypeUtil {
       return null;
     }
     try {
-      switch (dimension.getDataType()) {
-        case DECIMAL:
-          return parseStringToBigDecimal(value, dimension);
-        case INT:
-          Integer.parseInt(value);
-          break;
-        case DOUBLE:
-          Double.parseDouble(value);
-          break;
-        case LONG:
-          Long.parseLong(value);
-          break;
-        case FLOAT:
-          Float.parseFloat(value);
-          break;
-        default:
-          // do nothing
+      DataType dataType = dimension.getDataType();
+      if (dataType == DataTypes.DECIMAL) {
+        return parseStringToBigDecimal(value, dimension);
+      } else if (dataType == DataTypes.INT) {
+        Integer.parseInt(value);
+      } else if (dataType == DataTypes.DOUBLE) {
+        Double.parseDouble(value);
+      } else if (dataType == DataTypes.LONG) {
+        Long.parseLong(value);
+      } else if (dataType == DataTypes.FLOAT) {
+        Float.parseFloat(value);
+      } else {
+        // do nothing
       }
     } catch (NumberFormatException e) {
       return null;
@@ -682,55 +663,54 @@ public final class DataTypeUtil {
     }
     try {
       long parsedIntVal = 0;
-      switch (columnSchema.getDataType()) {
-        case INT:
-          parsedIntVal = (long) Integer.parseInt(data);
-          return String.valueOf(parsedIntVal)
-              .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        case SHORT:
-          parsedIntVal = (long) Short.parseShort(data);
-          return String.valueOf(parsedIntVal)
-              .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        case DOUBLE:
-          return String.valueOf(Double.parseDouble(data))
-              .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        case LONG:
-          return String.valueOf(Long.parseLong(data))
-              .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        case DATE:
-          DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+      DataType dataType = columnSchema.getDataType();
+      if (dataType == DataTypes.INT) {
+        parsedIntVal = (long) Integer.parseInt(data);
+        return String.valueOf(parsedIntVal)
+            .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      } else if (dataType == DataTypes.SHORT) {
+        parsedIntVal = (long) Short.parseShort(data);
+        return String.valueOf(parsedIntVal)
+            .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      } else if (dataType == DataTypes.DOUBLE) {
+        return String.valueOf(Double.parseDouble(data))
+            .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      } else if (dataType == DataTypes.LONG) {
+        return String.valueOf(Long.parseLong(data))
+            .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      } else if (dataType == DataTypes.DATE) {
+        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(columnSchema.getDataType());
+        int value = directDictionaryGenerator.generateDirectSurrogateKey(data);
+        return String.valueOf(value)
+            .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          DirectDictionaryGenerator directDictionaryGenerator1 = DirectDictionaryKeyGeneratorFactory
               .getDirectDictionaryGenerator(columnSchema.getDataType());
-          int value = directDictionaryGenerator.generateDirectSurrogateKey(data);
-          return String.valueOf(value)
+          int value1 = directDictionaryGenerator1.generateDirectSurrogateKey(data);
+          return String.valueOf(value1)
               .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        case TIMESTAMP:
-          if (columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-            DirectDictionaryGenerator directDictionaryGenerator1 =
-                DirectDictionaryKeyGeneratorFactory
-                    .getDirectDictionaryGenerator(columnSchema.getDataType());
-            int value1 = directDictionaryGenerator1.generateDirectSurrogateKey(data);
-            return String.valueOf(value1)
-                .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          } else {
-            try {
-              Date dateToStr = timeStampformatter.get().parse(data);
-              return ByteUtil.toBytes(dateToStr.getTime());
-            } catch (ParseException e) {
-              LOGGER.error(
-                  "Cannot convert value to Time/Long type value. Value is considered as null" + e
-                      .getMessage());
-              return null;
-            }
-          }
-        case DECIMAL:
-          String parsedValue = parseStringToBigDecimal(data, columnSchema);
-          if (null == parsedValue) {
+        } else {
+          try {
+            Date dateToStr = timeStampformatter.get().parse(data);
+            return ByteUtil.toBytes(dateToStr.getTime());
+          } catch (ParseException e) {
+            LOGGER.error(
+                "Cannot convert value to Time/Long type value. Value is considered as null" + e
+                    .getMessage());
             return null;
           }
-          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(parsedValue);
-          return bigDecimalToByte(javaDecVal);
-        default:
-          return getDataTypeConverter().convertFromStringToByte(data);
+        }
+      } else if (dataType == DataTypes.DECIMAL) {
+        String parsedValue = parseStringToBigDecimal(data, columnSchema);
+        if (null == parsedValue) {
+          return null;
+        }
+        java.math.BigDecimal javaDecVal = new java.math.BigDecimal(parsedValue);
+        return bigDecimalToByte(javaDecVal);
+      } else {
+        return getDataTypeConverter().convertFromStringToByte(data);
       }
     } catch (NumberFormatException ex) {
       LOGGER.error("Problem while converting data type" + data);
@@ -749,19 +729,16 @@ public final class DataTypeUtil {
   public static String normalizeColumnValueForItsDataType(String value, ColumnSchema columnSchema) {
     try {
       Object parsedValue = null;
-      switch (columnSchema.getDataType()) {
-        case DECIMAL:
-          return parseStringToBigDecimal(value, columnSchema);
-        case SHORT:
-        case INT:
-        case LONG:
-          parsedValue = normalizeIntAndLongValues(value, columnSchema.getDataType());
-          break;
-        case DOUBLE:
-          parsedValue = Double.parseDouble(value);
-          break;
-        default:
-          return value;
+      DataType dataType = columnSchema.getDataType();
+      if (dataType == DataTypes.DECIMAL) {
+        return parseStringToBigDecimal(value, columnSchema);
+      } else if (dataType == DataTypes.SHORT || dataType == DataTypes.INT ||
+          dataType == DataTypes.LONG) {
+        parsedValue = normalizeIntAndLongValues(value, columnSchema.getDataType());
+      } else if (dataType == DataTypes.DOUBLE) {
+        parsedValue = Double.parseDouble(value);
+      } else {
+        return value;
       }
       if (null != parsedValue) {
         return value;


[04/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
index 2cdb46e..cdc7fb9 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
 
 import org.junit.BeforeClass;
@@ -43,7 +44,7 @@ public class ExpressionResultTest {
   private static ExpressionResult expressionResult;
 
   @BeforeClass public static void setUp() {
-    expressionResult = new ExpressionResult(DataType.INT, null);
+    expressionResult = new ExpressionResult(DataTypes.INT, null);
   }
 
   @Test public void testGetIntForNull() throws FilterIllegalMemberException {
@@ -51,7 +52,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetIntForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "5");
     int actualValue = expressionResultForString.getInt();
     int expectedValue = 5;
     assertEquals(expectedValue, actualValue);
@@ -59,19 +60,19 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetIntForNumberFormatException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "FOOBAR");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "FOOBAR");
     expressionResultForString.getInt();
   }
 
   @Test public void testGetIntForDouble() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.DOUBLE, 5.0);
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.DOUBLE, 5.0);
     int actualValue = expressionResultForDouble.getInt();
     int expectedValue = 5;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetIntForInt() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.INT, 5);
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.INT, 5);
     int actualValue = expressionResultForInt.getInt();
     int expectedValue = 5;
     assertEquals(expectedValue, actualValue);
@@ -79,13 +80,13 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetIntForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForInt.getInt();
   }
 
   @Test(expected = FilterIllegalMemberException.class) public void testGetIntForClassCastException()
       throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.INT, "STRING");
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.INT, "STRING");
     expressionResultForInt.getInt();
   }
 
@@ -94,7 +95,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetShortForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "5");
     short actualValue = expressionResultForString.getShort();
     short expectedValue = 5;
     assertEquals(expectedValue, actualValue);
@@ -102,19 +103,19 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetShortForNumberFormatException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "FOOBAR");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "FOOBAR");
     expressionResultForString.getShort();
   }
 
   @Test public void testGetShortForDouble() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.DOUBLE, 5.0);
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.DOUBLE, 5.0);
     short actualValue = expressionResultForDouble.getShort();
     short expectedValue = 5;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetShortForInt() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.INT, 5);
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.INT, 5);
     short actualValue = expressionResultForInt.getShort();
     short expectedValue = 5;
     assertEquals(expectedValue, actualValue);
@@ -122,13 +123,13 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetShortForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForInt.getShort();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetShortForClassCastException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.INT, "STRING");
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.INT, "STRING");
     expressionResultForInt.getShort();
   }
 
@@ -137,14 +138,14 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetStringForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "5");
     String actualValue = expressionResultForString.getString();
     String expectedValue = "5";
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetStringForException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.INT, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.INT, "5");
     String actualValue = expressionResultForString.getString();
     String expectedValue = "5";
     assertEquals(expectedValue, actualValue);
@@ -155,7 +156,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetDoubleForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "5");
     double actualValue = expressionResultForString.getDouble();
     double expectedValue = 5;
     assertEquals(expectedValue, actualValue, DELTA);
@@ -163,19 +164,19 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetDoubleForNumberFormatException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "FOOBAR");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "FOOBAR");
     expressionResultForString.getDouble();
   }
 
   @Test public void testGetDoubleForInt() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.INT, 5);
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.INT, 5);
     double actualValue = expressionResultForDouble.getDouble();
     double expectedValue = 5.0;
     assertEquals(expectedValue, actualValue, DELTA);
   }
 
   @Test public void testGetDoubleForDouble() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.DOUBLE, 5.0);
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.DOUBLE, 5.0);
     double actualValue = expressionResultForDouble.getDouble();
     double expectedValue = 5;
     assertEquals(expectedValue, actualValue, DELTA);
@@ -183,13 +184,13 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetDoubleForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForDouble.getDouble();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetDoubleForClassCastException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.DOUBLE, "STRING");
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.DOUBLE, "STRING");
     expressionResultForDouble.getDouble();
   }
 
@@ -198,7 +199,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetLongForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "5");
     long actualValue = expressionResultForString.getLong();
     long expectedValue = 5;
     assertEquals(expectedValue, actualValue);
@@ -206,19 +207,19 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetLongForNumberFormatException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "FOOBAR");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "FOOBAR");
     expressionResultForString.getLong();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetLongForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForLong = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForLong = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForLong.getLong();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetLongForClassCastException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForLong = new ExpressionResult(DataType.LONG, "STRING");
+    ExpressionResult expressionResultForLong = new ExpressionResult(DataTypes.LONG, "STRING");
     expressionResultForLong.getLong();
   }
 
@@ -227,7 +228,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetDecimalForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "5");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "5");
     BigDecimal actualValue = expressionResultForString.getDecimal();
     BigDecimal expectedValue = new BigDecimal(5.00);
     assertEquals(expectedValue, actualValue);
@@ -235,26 +236,26 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetDecimalForNumberFormatException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "FOOBAR");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "FOOBAR");
     expressionResultForString.getDecimal();
   }
 
   @Test public void testGetDecimalForInt() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForInt = new ExpressionResult(DataType.INT, 5);
+    ExpressionResult expressionResultForInt = new ExpressionResult(DataTypes.INT, 5);
     BigDecimal actualValue = expressionResultForInt.getDecimal();
     BigDecimal expectedValue = new BigDecimal(5);
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetDecimalForDouble() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDouble = new ExpressionResult(DataType.DOUBLE, 5);
+    ExpressionResult expressionResultForDouble = new ExpressionResult(DataTypes.DOUBLE, 5);
     BigDecimal actualValue = expressionResultForDouble.getDecimal();
     BigDecimal expectedValue = new BigDecimal(5);
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetDecimalForDecimal() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDecimal = new ExpressionResult(DataType.DECIMAL, 5);
+    ExpressionResult expressionResultForDecimal = new ExpressionResult(DataTypes.DECIMAL, 5);
     BigDecimal actualValue = expressionResultForDecimal.getDecimal();
     BigDecimal expectedValue = new BigDecimal(5);
     assertEquals(expectedValue, actualValue);
@@ -262,13 +263,13 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetDecimalForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForFloat = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForFloat = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForFloat.getDecimal();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetDecimalForClassCastException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForException = new ExpressionResult(DataType.LONG, "STRING");
+    ExpressionResult expressionResultForException = new ExpressionResult(DataTypes.LONG, "STRING");
     expressionResultForException.getDecimal();
   }
 
@@ -278,7 +279,7 @@ public class ExpressionResultTest {
 
   @Test public void testGetTimeForString() throws FilterIllegalMemberException {
     ExpressionResult expressionResultForString =
-        new ExpressionResult(DataType.STRING, "2016-11-07 10:15:09");
+        new ExpressionResult(DataTypes.STRING, "2016-11-07 10:15:09");
     long actualValue = expressionResultForString.getTime();
     long expectedValue = getTime("2016-11-07 10:15:09");
     assertEquals(expectedValue, actualValue);
@@ -302,19 +303,19 @@ public class ExpressionResultTest {
   @Test(expected = FilterIllegalMemberException.class) public void testGetTimeForParseException()
       throws FilterIllegalMemberException {
     ExpressionResult expressionResultForParseException =
-        new ExpressionResult(DataType.STRING, "FOOBAR");
+        new ExpressionResult(DataTypes.STRING, "FOOBAR");
     expressionResultForParseException.getTime();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetTimeForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForFloat = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForFloat = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForFloat.getTime();
   }
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetTimeForClassCastException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForException = new ExpressionResult(DataType.LONG, "STRING");
+    ExpressionResult expressionResultForException = new ExpressionResult(DataTypes.LONG, "STRING");
     expressionResultForException.getTime();
   }
 
@@ -323,14 +324,14 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetBooleanForString() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.STRING, "true");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.STRING, "true");
     boolean actualValue = expressionResultForString.getBoolean();
     boolean expectedValue = true;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetBooleanForBoolean() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForString = new ExpressionResult(DataType.BOOLEAN, "true");
+    ExpressionResult expressionResultForString = new ExpressionResult(DataTypes.BOOLEAN, "true");
     boolean actualValue = expressionResultForString.getBoolean();
     boolean expectedValue = true;
     assertEquals(expectedValue, actualValue);
@@ -338,12 +339,12 @@ public class ExpressionResultTest {
 
   @Test(expected = FilterIllegalMemberException.class)
   public void testGetBooleanForIllegalMemberException() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForFloat = new ExpressionResult(DataType.FLOAT, 25.36);
+    ExpressionResult expressionResultForFloat = new ExpressionResult(DataTypes.FLOAT, 25.36);
     expressionResultForFloat.getBoolean();
   }
 
   @Test public void testGetList() {
-    ExpressionResult obj = new ExpressionResult(DataType.STRING, "test");
+    ExpressionResult obj = new ExpressionResult(DataTypes.STRING, "test");
     List<ExpressionResult> actualValue = obj.getList();
     List<ExpressionResult> expected = new ArrayList<>();
     expected.add(0, obj);
@@ -351,7 +352,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testHashCode() {
-    ExpressionResult obj = new ExpressionResult(DataType.INT, 1);
+    ExpressionResult obj = new ExpressionResult(DataTypes.INT, 1);
     int actualValue = obj.hashCode();
     int expectedValue = 80;
     assertEquals(expectedValue, actualValue);
@@ -365,14 +366,14 @@ public class ExpressionResultTest {
   }
 
   @Test public void testHashCodeForNul() {
-    ExpressionResult obj = new ExpressionResult(DataType.DOUBLE, null);
+    ExpressionResult obj = new ExpressionResult(DataTypes.DOUBLE, null);
     int actualValue = obj.hashCode();
     int expectedValue = 31;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testEqualsForObjNOtInstanceOfExpressionResult() {
-    ExpressionResult expressionResult = new ExpressionResult(DataType.INT, 5);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.INT, 5);
     List obj = new ArrayList();
     obj.add(0, 0);
     obj.add(1, 1);
@@ -381,60 +382,60 @@ public class ExpressionResultTest {
   }
 
   @Test public void testEqualsForString() {
-    ExpressionResult expressionResult = new ExpressionResult(DataType.STRING, "5");
-    ExpressionResult objToCompare = new ExpressionResult(DataType.STRING, "6");
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.STRING, "5");
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.STRING, "6");
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
 
   @Test public void testEqualsForShort() {
-    ExpressionResult expressionResult = new ExpressionResult(DataType.SHORT, 5);
-    ExpressionResult objToCompare = new ExpressionResult(DataType.SHORT, 6);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.SHORT, 5);
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.SHORT, 6);
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
 
   @Test public void testEqualsForInt() {
-    ExpressionResult expressionResult = new ExpressionResult(DataType.INT, 5);
-    ExpressionResult objToCompare = new ExpressionResult(DataType.INT, 6);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.INT, 5);
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.INT, 6);
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
 
   @Test public void testEqualsForDecimal() {
-    ExpressionResult expressionResult = new ExpressionResult(DataType.DECIMAL, 5);
-    ExpressionResult objToCompare = new ExpressionResult(DataType.DECIMAL, 6);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DECIMAL, 5);
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.DECIMAL, 6);
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
 
   @Test public void testEqualsForDouble() {
-    ExpressionResult expressionResult = new ExpressionResult(DataType.DOUBLE, 5.89);
-    ExpressionResult objToCompare = new ExpressionResult(DataType.DOUBLE, 6.90);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DOUBLE, 5.89);
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.DOUBLE, 6.90);
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
 
   @Test public void testEqualsForException() {
     ExpressionResult expressionResult =
-        new ExpressionResult(DataType.TIMESTAMP, "2016-11-07 10:15:09");
-    ExpressionResult objToCompare = new ExpressionResult(DataType.DOUBLE, "2016-11-06 10:15:09");
+        new ExpressionResult(DataTypes.TIMESTAMP, "2016-11-07 10:15:09");
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.DOUBLE, "2016-11-06 10:15:09");
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
 
   @Test public void testEqualsForLongAndTimeStamp() {
     ExpressionResult expressionResult =
-        new ExpressionResult(DataType.TIMESTAMP, new Long(11111111111111111L));
+        new ExpressionResult(DataTypes.TIMESTAMP, new Long(11111111111111111L));
     ExpressionResult objToCompare =
-        new ExpressionResult(DataType.LONG, new Long(11111111111111111L));
+        new ExpressionResult(DataTypes.LONG, new Long(11111111111111111L));
     boolean result = expressionResult.equals(objToCompare);
     assertTrue(result);
   }
 
   @Test public void compareToForInt() {
-    ExpressionResult obj = new ExpressionResult(DataType.INT, 5);
-    ExpressionResult expressionResult = new ExpressionResult(DataType.INT, 6);
+    ExpressionResult obj = new ExpressionResult(DataTypes.INT, 5);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.INT, 6);
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = 1;
     assertEquals(expectedValue, actualValue);
@@ -442,32 +443,32 @@ public class ExpressionResultTest {
   }
 
   @Test public void compareToForDecimal() {
-    ExpressionResult obj = new ExpressionResult(DataType.DECIMAL, 5);
-    ExpressionResult expressionResult = new ExpressionResult(DataType.DECIMAL, 6);
+    ExpressionResult obj = new ExpressionResult(DataTypes.DECIMAL, 5);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DECIMAL, 6);
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = 1;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void compareToForException() {
-    ExpressionResult obj = new ExpressionResult(DataType.INT, 5);
-    ExpressionResult expressionResult = new ExpressionResult(DataType.DECIMAL, 6);
+    ExpressionResult obj = new ExpressionResult(DataTypes.INT, 5);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DECIMAL, 6);
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = -1;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void compareToForTimestamp() {
-    ExpressionResult obj = new ExpressionResult(DataType.TIMESTAMP, "2016-11-07 10:15:09");
+    ExpressionResult obj = new ExpressionResult(DataTypes.TIMESTAMP, "2016-11-07 10:15:09");
     ExpressionResult expressionResult =
-        new ExpressionResult(DataType.TIMESTAMP, "2016-11-07 10:15:09");
+        new ExpressionResult(DataTypes.TIMESTAMP, "2016-11-07 10:15:09");
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = 0;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testGetListAsString() throws FilterIllegalMemberException {
-    ExpressionResult obj = new ExpressionResult(DataType.INT, 6);
+    ExpressionResult obj = new ExpressionResult(DataTypes.INT, 6);
     List<String> actualValue = obj.getListAsString();
     List<String> expectedValue = new ArrayList<>();
     expectedValue.add("6");
@@ -475,21 +476,21 @@ public class ExpressionResultTest {
   }
 
   @Test public void compareToForString() {
-    ExpressionResult obj = new ExpressionResult(DataType.STRING, "2016");
-    ExpressionResult expressionResult = new ExpressionResult(DataType.STRING, "2016");
+    ExpressionResult obj = new ExpressionResult(DataTypes.STRING, "2016");
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.STRING, "2016");
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = 0;
     assertEquals(expectedValue, actualValue);
   }
 
   @Test public void testIsNullForNull() {
-    ExpressionResult obj = new ExpressionResult(DataType.STRING, null);
+    ExpressionResult obj = new ExpressionResult(DataTypes.STRING, null);
     boolean actualValue = obj.isNull();
     assertTrue(actualValue);
   }
 
   @Test public void testIsNullForNotNull() {
-    ExpressionResult obj = new ExpressionResult(DataType.STRING, "test");
+    ExpressionResult obj = new ExpressionResult(DataTypes.STRING, "test");
     boolean actualValue = obj.isNull();
     assertFalse(actualValue);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/LiteralExpressionTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/LiteralExpressionTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/LiteralExpressionTest.java
index b99c7f9..d913968 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/LiteralExpressionTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/LiteralExpressionTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.scan.expression;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 
 import org.junit.Before;
@@ -31,13 +31,13 @@ public class LiteralExpressionTest {
 
   @Before public void setUp() {
     String value = "testing";
-    literalExpression = new LiteralExpression(value, DataType.STRING);
+    literalExpression = new LiteralExpression(value, DataTypes.STRING);
   }
 
   @Test public void testEvaluate() {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new String[] { "testing" });
-    ExpressionResult expectedResult = new ExpressionResult(DataType.STRING, "testing");
+    ExpressionResult expectedResult = new ExpressionResult(DataTypes.STRING, "testing");
     assertEquals(expectedResult, literalExpression.evaluate(rowImpl));
   }
 
@@ -45,7 +45,7 @@ public class LiteralExpressionTest {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new String[] { "testing" });
     literalExpression.evaluate(rowImpl);
-    ExpressionResult expectedResult = new ExpressionResult(DataType.STRING, "testing");
+    ExpressionResult expectedResult = new ExpressionResult(DataTypes.STRING, "testing");
     assertEquals(expectedResult, literalExpression.evaluate(rowImpl));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
index 7bc4287..0abfb18 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -44,7 +45,7 @@ public class EqualToExpressionUnitTest {
   static EqualToExpression equalToExpression;
 
   @Test public void testForEqualToExpressionWithGetString() throws Exception {
-    ColumnExpression right = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("name", DataTypes.STRING);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     String expected_result = "EqualTo(ColumnExpression(name),ColumnExpression(name))";
@@ -54,7 +55,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -75,7 +76,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithStringDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("name", DataTypes.STRING);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -96,7 +97,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -116,7 +117,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.DOUBLE);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -136,7 +137,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.LONG);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -157,7 +158,7 @@ public class EqualToExpressionUnitTest {
   @Test public void testEvaluateForEqualToExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression right = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(0);
       equalToExpression = new EqualToExpression(right, right);
       RowImpl value = new RowImpl();
@@ -183,7 +184,7 @@ public class EqualToExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForEqualToExpressionForDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -195,7 +196,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithBooleanParameter()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right, true);
     RowImpl value = new RowImpl();
@@ -216,9 +217,9 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -239,7 +240,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithIsNullReturnFalse()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -266,7 +267,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithNullWhileCreatingObject()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right, true);
     RowImpl value = new RowImpl();
@@ -287,13 +288,13 @@ public class EqualToExpressionUnitTest {
     };
 
     ExpressionResult result = equalToExpression.evaluate(value);
-    assertEquals(DataType.BOOLEAN, result.getDataType());
+    assertEquals(DataTypes.BOOLEAN, result.getDataType());
 
   }
 
   @Test public void testEvaluateForEqualToExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
index 6481dd5..35d8886 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -45,9 +46,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithBothStringISSame()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -67,9 +68,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("id", DataTypes.SHORT);
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -91,9 +92,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_number", DataTypes.INT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_number", DataTypes.INT);
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -114,9 +115,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -137,9 +138,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.LONG);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.LONG);
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -161,9 +162,9 @@ public class GreaterThanEqualToExpressionUnitTest {
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
       greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
       RowImpl value = new RowImpl();
@@ -190,9 +191,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForGreaterThanEqualToExpressionWithDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.BOOLEAN);
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -204,9 +205,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -226,9 +227,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithIsNullReturnTrue()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("id", DataTypes.SHORT);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -255,9 +256,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("name", DataTypes.STRING);
     left.setColIndex(1);
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(0);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -277,9 +278,9 @@ public class GreaterThanEqualToExpressionUnitTest {
   }
 
   @Test public void testForGreaterThanEqualToExpressionWithGetString() throws Exception {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     String expected_result = "GreaterThanEqualTo(ColumnExpression(left_name),ColumnExpression(right_name))";

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
index 42cd791..7d6a168 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -44,9 +45,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithStringDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -75,9 +76,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("id", DataTypes.SHORT);
     left.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -109,9 +110,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -142,9 +143,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_number", DataTypes.INT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_number", DataTypes.INT);
     left.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -176,9 +177,9 @@ public class GreaterThanExpressionUnitTest {
   @Test public void testEvaluateForGreaterThanExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
 
       greaterThanExpression = new GreaterThanExpression(left, right);
@@ -220,9 +221,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.LONG);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.LONG);
     left.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -250,9 +251,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
     left.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -280,7 +281,7 @@ public class GreaterThanExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForGreaterThanExpressionWithDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(0);
     greaterThanExpression = new GreaterThanExpression(right, right);
     RowImpl value = new RowImpl();
@@ -292,7 +293,7 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithIsNullReturnTrue()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     greaterThanExpression = new GreaterThanExpression(right, right);
     RowImpl value = new RowImpl();
@@ -319,9 +320,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -350,9 +351,9 @@ public class GreaterThanExpressionUnitTest {
   }
 
   @Test public void testForGreaterThanExpressionWithGetString() throws Exception {
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
     greaterThanExpression = new GreaterThanExpression(left, right);
     String expected_result = "GreaterThan(ColumnExpression(left_name),ColumnExpression(right_name))";

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
index 904f03b..2fd6585 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -45,9 +46,9 @@ public class InExpressionUnitTest {
 
   @Test public void testEvaluateForInExpressionWithString()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -58,7 +59,7 @@ public class InExpressionUnitTest {
     new MockUp<ExpressionResult>() {
 
       @Mock public DataType getDataType() {
-        return DataType.STRING;
+        return DataTypes.STRING;
       }
 
       @Mock public String getString() {
@@ -74,9 +75,9 @@ public class InExpressionUnitTest {
   @Test public void testEvaluateForInExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
 
-    ColumnExpression left = new ColumnExpression("left_id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("left_id", DataTypes.SHORT);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("right_id", DataTypes.SHORT);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -92,16 +93,16 @@ public class InExpressionUnitTest {
     };
 
     ExpressionResult result = inExpression.evaluate(value);
-    assertEquals(result.getDataType(), DataType.BOOLEAN);
+    assertEquals(result.getDataType(), DataTypes.BOOLEAN);
 
   }
 
   @Test public void testEvaluateForInExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
 
-    ColumnExpression left = new ColumnExpression("left_id", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_id", DataTypes.INT);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_id", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_id", DataTypes.INT);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -117,15 +118,15 @@ public class InExpressionUnitTest {
     };
 
     ExpressionResult result = inExpression.evaluate(value);
-    assertEquals(result.getDataType(), DataType.BOOLEAN);
+    assertEquals(result.getDataType(), DataTypes.BOOLEAN);
 
   }
 
   @Test public void testEvaluateForInExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -146,9 +147,9 @@ public class InExpressionUnitTest {
 
   @Test public void testEvaluateForInExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.LONG);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.LONG);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -170,9 +171,9 @@ public class InExpressionUnitTest {
   @Test public void testEvaluateForInExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("left_timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("left_timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("right_timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("right_timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
       inExpression = new InExpression(left, right);
 
@@ -202,9 +203,9 @@ public class InExpressionUnitTest {
 
   @Test public void testEvaluateForInExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DECIMAL);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DECIMAL);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -225,9 +226,9 @@ public class InExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForInExpressionWithDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.BOOLEAN);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();
@@ -239,9 +240,9 @@ public class InExpressionUnitTest {
   }
 
   @Test public void testForInExpressionWithGetString() throws Exception {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     String expected_result = "IN(ColumnExpression(left_name),ColumnExpression(right_name))";
@@ -251,9 +252,9 @@ public class InExpressionUnitTest {
 
   @Test public void testEvaluateForInExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("name", DataTypes.STRING);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("number", DataTypes.INT);
     left.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
index 32cc836..7000ceb 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -45,9 +46,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithBothStringISSame()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, left);
     RowImpl value = new RowImpl();
@@ -78,9 +79,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("id", DataTypes.SHORT);
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -112,9 +113,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -145,9 +146,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_number", DataTypes.INT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_number", DataTypes.INT);
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -179,9 +180,9 @@ public class LessThanEqualToExpressionUnitTest {
   @Test public void testEvaluateForLessThanEqualToExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
 
       lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
@@ -223,9 +224,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.LONG);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.LONG);
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -253,9 +254,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DECIMAL);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DECIMAL);
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -283,7 +284,7 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForLessThanEqualToExpressionWithDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(0);
     lessThanEqualToExpression = new LessThanEqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -295,7 +296,7 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithIsNullReturnTrue()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     lessThanEqualToExpression = new LessThanEqualToExpression(right, right);
     RowImpl value = new RowImpl();
@@ -322,9 +323,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();
@@ -353,9 +354,9 @@ public class LessThanEqualToExpressionUnitTest {
   }
 
   @Test public void testForLessThanEqualToExpressionWithGetString() throws Exception {
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     String expected_result =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
index 90684ac..889ebbd 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -44,9 +45,9 @@ public class LessThanExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanExpressionWithStringDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -75,9 +76,9 @@ public class LessThanExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("id", DataTypes.SHORT);
     left.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -109,9 +110,9 @@ public class LessThanExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -142,9 +143,9 @@ public class LessThanExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_number", DataTypes.INT);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_number", DataTypes.INT);
     left.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -176,9 +177,9 @@ public class LessThanExpressionUnitTest {
  @Test public void testEvaluateForLessThanExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
 
       lessThanExpression = new LessThanExpression(left, right);
@@ -220,9 +221,9 @@ public class LessThanExpressionUnitTest {
 
  @Test public void testEvaluateForLessThanExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.LONG);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.LONG);
     left.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -250,9 +251,9 @@ public class LessThanExpressionUnitTest {
 
  @Test public void testEvaluateForLessThanExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
     left.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -280,7 +281,7 @@ public class LessThanExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForLessThanExpressionWithDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(0);
     lessThanExpression = new LessThanExpression(right, right);
     RowImpl value = new RowImpl();
@@ -292,7 +293,7 @@ public class LessThanExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanExpressionWithIsNullReturnTrue()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     lessThanExpression = new LessThanExpression(right, right);
     RowImpl value = new RowImpl();
@@ -319,9 +320,9 @@ public class LessThanExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();
@@ -350,9 +351,9 @@ public class LessThanExpressionUnitTest {
   }
 
  @Test public void testForLessThanExpressionWithGetString() throws Exception {
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
     lessThanExpression = new LessThanExpression(left, right);
     String expected_result = "LessThan(ColumnExpression(left_name),ColumnExpression(right_name))";

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/ListExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/ListExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/ListExpressionUnitTest.java
index 6f8a375..22fdfd2 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/ListExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/ListExpressionUnitTest.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.scan.expression.conditional;
 import java.util.List;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
@@ -40,9 +41,9 @@ public class ListExpressionUnitTest {
   static ListExpression listExpression;
 
   @Test public void test() throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
 
     List<Expression> children = new ArrayList<>();


[02/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index 4b04116..ea90bbf 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -50,6 +50,7 @@ import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
@@ -187,7 +188,7 @@ public class StoreCreator {
     ColumnSchema id = new ColumnSchema();
     id.setColumnName("ID");
     id.setColumnar(true);
-    id.setDataType(DataType.INT);
+    id.setDataType(DataTypes.INT);
     id.setEncodingList(encodings);
     id.setColumnUniqueId(UUID.randomUUID().toString());
     id.setDimensionColumn(true);
@@ -197,7 +198,7 @@ public class StoreCreator {
     ColumnSchema date = new ColumnSchema();
     date.setColumnName("date");
     date.setColumnar(true);
-    date.setDataType(DataType.STRING);
+    date.setDataType(DataTypes.STRING);
     date.setEncodingList(encodings);
     date.setColumnUniqueId(UUID.randomUUID().toString());
     date.setDimensionColumn(true);
@@ -208,7 +209,7 @@ public class StoreCreator {
     ColumnSchema country = new ColumnSchema();
     country.setColumnName("country");
     country.setColumnar(true);
-    country.setDataType(DataType.STRING);
+    country.setDataType(DataTypes.STRING);
     country.setEncodingList(encodings);
     country.setColumnUniqueId(UUID.randomUUID().toString());
     country.setDimensionColumn(true);
@@ -219,7 +220,7 @@ public class StoreCreator {
     ColumnSchema name = new ColumnSchema();
     name.setColumnName("name");
     name.setColumnar(true);
-    name.setDataType(DataType.STRING);
+    name.setDataType(DataTypes.STRING);
     name.setEncodingList(encodings);
     name.setColumnUniqueId(UUID.randomUUID().toString());
     name.setDimensionColumn(true);
@@ -230,7 +231,7 @@ public class StoreCreator {
     ColumnSchema phonetype = new ColumnSchema();
     phonetype.setColumnName("phonetype");
     phonetype.setColumnar(true);
-    phonetype.setDataType(DataType.STRING);
+    phonetype.setDataType(DataTypes.STRING);
     phonetype.setEncodingList(encodings);
     phonetype.setColumnUniqueId(UUID.randomUUID().toString());
     phonetype.setDimensionColumn(true);
@@ -241,7 +242,7 @@ public class StoreCreator {
     ColumnSchema serialname = new ColumnSchema();
     serialname.setColumnName("serialname");
     serialname.setColumnar(true);
-    serialname.setDataType(DataType.STRING);
+    serialname.setDataType(DataTypes.STRING);
     serialname.setEncodingList(encodings);
     serialname.setColumnUniqueId(UUID.randomUUID().toString());
     serialname.setDimensionColumn(true);
@@ -252,7 +253,7 @@ public class StoreCreator {
     ColumnSchema salary = new ColumnSchema();
     salary.setColumnName("salary");
     salary.setColumnar(true);
-    salary.setDataType(DataType.INT);
+    salary.setDataType(DataTypes.INT);
     salary.setEncodingList(new ArrayList<Encoding>());
     salary.setColumnUniqueId(UUID.randomUUID().toString());
     salary.setDimensionColumn(false);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
index 34a1936..36ae65c 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -136,13 +137,12 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
    */
   private Writable createWritableObject(Object obj, CarbonColumn carbonColumn) throws IOException {
     DataType dataType = carbonColumn.getDataType();
-    switch (dataType) {
-      case STRUCT:
-        return createStruct(obj, carbonColumn);
-      case ARRAY:
-        return createArray(obj, carbonColumn);
-      default:
-        return createWritablePrimitive(obj, carbonColumn);
+    if (dataType == DataTypes.STRUCT) {
+      return createStruct(obj, carbonColumn);
+    } else if (dataType == DataTypes.ARRAY) {
+      return createArray(obj, carbonColumn);
+    } else {
+      return createWritablePrimitive(obj, carbonColumn);
     }
   }
 
@@ -219,77 +219,30 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
     if (obj == null) {
       return null;
     }
-    switch (dataType) {
-      case NULL:
-        return null;
-      case DOUBLE:
-        return new DoubleWritable((double) obj);
-      case INT:
-        return new IntWritable((int) obj);
-      case LONG:
-        return new LongWritable((long) obj);
-      case SHORT:
-        return new ShortWritable((short) obj);
-      case DATE:
-        Calendar c = Calendar.getInstance();
-        c.setTime(new Date(0));
-        c.add(Calendar.DAY_OF_YEAR, (Integer) obj);
-        Date date = new java.sql.Date(c.getTime().getTime());
-        return new DateWritable(date);
-      case TIMESTAMP:
-        return new TimestampWritable(new Timestamp((long) obj / 1000));
-      case STRING:
-        return new Text(obj.toString());
-      case DECIMAL:
-        return new HiveDecimalWritable(
-            HiveDecimal.create(new java.math.BigDecimal(obj.toString())));
-      default:
-        throw new IOException("unsupported data type:" + dataType);
-    }
-  }
-
-  /**
-   * If we need to use the same Writable[] then we can use this method
-   *
-   * @param writable
-   * @param obj
-   * @param carbonColumn
-   * @throws IOException
-   */
-  private void setPrimitive(Writable writable, Object obj, CarbonColumn carbonColumn)
-      throws IOException {
-    DataType dataType = carbonColumn.getDataType();
-    if (obj == null) {
-      writable.write(null);
-    }
-    switch (dataType) {
-      case DOUBLE:
-        ((DoubleWritable) writable).set((double) obj);
-        break;
-      case INT:
-        ((IntWritable) writable).set((int) obj);
-        break;
-      case LONG:
-        ((LongWritable) writable).set((long) obj);
-        break;
-      case SHORT:
-        ((ShortWritable) writable).set((short) obj);
-        break;
-      case DATE:
-        ((DateWritable) writable).set(new Date((Long) obj));
-        break;
-      case TIMESTAMP:
-        ((TimestampWritable) writable).set(new Timestamp((long) obj));
-        break;
-      case STRING:
-        ((Text) writable).set(obj.toString());
-        break;
-      case DECIMAL:
-        ((HiveDecimalWritable) writable)
-            .set(HiveDecimal.create(new java.math.BigDecimal(obj.toString())));
-        break;
-      default:
-        throw new IOException("unsupported data type:" + dataType);
+    if (dataType == DataTypes.NULL) {
+      return null;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new DoubleWritable((double) obj);
+    } else if (dataType == DataTypes.INT) {
+      return new IntWritable((int) obj);
+    } else if (dataType == DataTypes.LONG) {
+      return new LongWritable((long) obj);
+    } else if (dataType == DataTypes.SHORT) {
+      return new ShortWritable((short) obj);
+    } else if (dataType == DataTypes.DATE) {
+      Calendar c = Calendar.getInstance();
+      c.setTime(new Date(0));
+      c.add(Calendar.DAY_OF_YEAR, (Integer) obj);
+      Date date = new java.sql.Date(c.getTime().getTime());
+      return new DateWritable(date);
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return new TimestampWritable(new Timestamp((long) obj / 1000));
+    } else if (dataType == DataTypes.STRING) {
+      return new Text(obj.toString());
+    } else if (dataType == DataTypes.DECIMAL) {
+      return new HiveDecimalWritable(HiveDecimal.create(new java.math.BigDecimal(obj.toString())));
+    } else {
+      throw new IOException("unsupported data type:" + dataType);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
index 6cb2915..e2c9c68 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
@@ -6,28 +6,28 @@ import org.apache.spark.sql.types.DataTypes;
 
 public class CarbonTypeUtil {
 
-  public static org.apache.spark.sql.types.DataType convertCarbonToSparkDataType(
+  static org.apache.spark.sql.types.DataType convertCarbonToSparkDataType(
       DataType carbonDataType) {
-    switch (carbonDataType) {
-      case STRING:
-        return DataTypes.StringType;
-      case SHORT:
-        return DataTypes.ShortType;
-      case INT:
-        return DataTypes.IntegerType;
-      case LONG:
+    if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.STRING) {
+      return DataTypes.StringType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT) {
+      return DataTypes.ShortType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT) {
+      return DataTypes.IntegerType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
         return DataTypes.LongType;
-      case DOUBLE:
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE) {
         return DataTypes.DoubleType;
-      case BOOLEAN:
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN) {
         return DataTypes.BooleanType;
-      case DECIMAL:
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL) {
         return DataTypes.createDecimalType();
-      case TIMESTAMP:
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.TIMESTAMP) {
         return DataTypes.TimestampType;
-      case DATE:
-        return DataTypes.DateType;
-      default: return null;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DATE) {
+      return DataTypes.DateType;
+    } else {
+      return null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
index f474433..2db2d23 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
@@ -193,28 +194,24 @@ class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
             null);
       } else {
         fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-            CarbonTypeUtil.convertCarbonToSparkDataType(DataType.INT), true, null);
+            CarbonTypeUtil.convertCarbonToSparkDataType(DataTypes.INT), true, null);
       }
     }
 
     for (int i = 0; i < queryMeasures.size(); i++) {
       QueryMeasure msr = queryMeasures.get(i);
-      switch (msr.getMeasure().getDataType()) {
-        case SHORT:
-        case INT:
-        case LONG:
-          fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
-              CarbonTypeUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
-              null);
-          break;
-        case DECIMAL:
-          fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
-              new DecimalType(msr.getMeasure().getPrecision(),
-                  msr.getMeasure().getScale()), true, null);
-          break;
-        default:
-          fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
-              CarbonTypeUtil.convertCarbonToSparkDataType(DataType.DOUBLE), true, null);
+      DataType dataType = msr.getMeasure().getDataType();
+      if (dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) {
+        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+            CarbonTypeUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
+            null);
+      } else if (dataType == DataTypes.DECIMAL) {
+        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+            new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true,
+            null);
+      } else {
+        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+            CarbonTypeUtil.convertCarbonToSparkDataType(DataTypes.DOUBLE), true, null);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
index a81c06f..99109d1 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.presto;
 
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.presto.impl.CarbonTableReader;
 import com.facebook.presto.spi.*;
@@ -221,32 +222,30 @@ public class CarbondataMetadata implements ConnectorMetadata {
 
   public static Type carbonDataType2SpiMapper(ColumnSchema columnSchema) {
     DataType colType = columnSchema.getDataType();
-    switch (colType) {
-      case BOOLEAN:
-        return BooleanType.BOOLEAN;
-      case SHORT:
-        return SmallintType.SMALLINT;
-      case INT:
-        return IntegerType.INTEGER;
-      case LONG:
-        return BigintType.BIGINT;
-      case FLOAT:
-      case DOUBLE:
-        return DoubleType.DOUBLE;
-      case DECIMAL:
-        if(columnSchema.getPrecision() > 0){
-          return DecimalType.createDecimalType(columnSchema.getPrecision(), columnSchema.getScale());
-        } else {
-          return DecimalType.createDecimalType();
-        }
-      case STRING:
-        return VarcharType.VARCHAR;
-      case DATE:
-        return DateType.DATE;
-      case TIMESTAMP:
-        return TimestampType.TIMESTAMP;
-      default:
-        return VarcharType.VARCHAR;
+    if (colType == DataTypes.BOOLEAN) {
+      return BooleanType.BOOLEAN;
+    } else if (colType == DataTypes.SHORT) {
+      return SmallintType.SMALLINT;
+    } else if (colType == DataTypes.INT) {
+      return IntegerType.INTEGER;
+    } else if (colType == DataTypes.LONG) {
+      return BigintType.BIGINT;
+    } else if (colType == DataTypes.FLOAT || colType == DataTypes.DOUBLE) {
+      return DoubleType.DOUBLE;
+    } else if (colType == DataTypes.DECIMAL) {
+      if (columnSchema.getPrecision() > 0) {
+        return DecimalType.createDecimalType(columnSchema.getPrecision(), columnSchema.getScale());
+      } else {
+        return DecimalType.createDecimalType();
+      }
+    } else if (colType == DataTypes.STRING) {
+      return VarcharType.VARCHAR;
+    } else if (colType == DataTypes.DATE) {
+      return DateType.DATE;
+    } else if (colType == DataTypes.TIMESTAMP) {
+      return TimestampType.TIMESTAMP;
+    } else {
+      return VarcharType.VARCHAR;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
index a958e63..c9fb177 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
@@ -69,17 +70,17 @@ public class PrestoFilterUtil {
 
   private static DataType Spi2CarbondataTypeMapper(CarbondataColumnHandle carbondataColumnHandle) {
     Type colType = carbondataColumnHandle.getColumnType();
-    if (colType == BooleanType.BOOLEAN) return DataType.BOOLEAN;
-    else if (colType == SmallintType.SMALLINT) return DataType.SHORT;
-    else if (colType == IntegerType.INTEGER) return DataType.INT;
-    else if (colType == BigintType.BIGINT) return DataType.LONG;
-    else if (colType == DoubleType.DOUBLE) return DataType.DOUBLE;
-    else if (colType == VarcharType.VARCHAR) return DataType.STRING;
-    else if (colType == DateType.DATE) return DataType.DATE;
-    else if (colType == TimestampType.TIMESTAMP) return DataType.TIMESTAMP;
+    if (colType == BooleanType.BOOLEAN) return DataTypes.BOOLEAN;
+    else if (colType == SmallintType.SMALLINT) return DataTypes.SHORT;
+    else if (colType == IntegerType.INTEGER) return DataTypes.INT;
+    else if (colType == BigintType.BIGINT) return DataTypes.LONG;
+    else if (colType == DoubleType.DOUBLE) return DataTypes.DOUBLE;
+    else if (colType == VarcharType.VARCHAR) return DataTypes.STRING;
+    else if (colType == DateType.DATE) return DataTypes.DATE;
+    else if (colType == TimestampType.TIMESTAMP) return DataTypes.TIMESTAMP;
     else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.getPrecision(),
-        carbondataColumnHandle.getScale()))) return DataType.DECIMAL;
-    else return DataType.STRING;
+        carbondataColumnHandle.getScale()))) return DataTypes.DECIMAL;
+    else return DataTypes.STRING;
   }
 
   /**
@@ -171,10 +172,10 @@ public class PrestoFilterUtil {
       }
       if (singleValues.size() == 1) {
         Expression ex;
-        if (coltype.equals(DataType.STRING)) {
+        if (coltype.equals(DataTypes.STRING)) {
           ex = new EqualToExpression(colExpression,
               new LiteralExpression(singleValues.get(0), coltype));
-        } else if (coltype.equals(DataType.TIMESTAMP) || coltype.equals(DataType.DATE)) {
+        } else if (coltype.equals(DataTypes.TIMESTAMP) || coltype.equals(DataTypes.DATE)) {
           Long value = (Long) singleValues.get(0);
           ex = new EqualToExpression(colExpression, new LiteralExpression(value, coltype));
         } else ex = new EqualToExpression(colExpression,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
index e3985e0..b9a9f0d 100644
--- a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
+++ b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
@@ -21,10 +21,9 @@ import io.airlift.slice.{Slice, Slices}
 import io.airlift.slice.Slices._
 
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryChunksWrapper,
-DictionaryColumnUniqueIdentifier}
+import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryChunksWrapper, DictionaryColumnUniqueIdentifier}
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 import org.apache.carbondata.core.util.{CarbonUtil, DataTypeUtil}
@@ -111,7 +110,7 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] {
       data.map { value =>
         DataTypeUtil
           .getDataBasedOnDataType(dictionaries(columnNo)
-            .getDictionaryValueForKey(value.asInstanceOf[Int]), DataType.STRING)
+            .getDictionaryValueForKey(value.asInstanceOf[Int]), DataTypes.STRING)
       }
     } else {
       data

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 07f9699..0a811b7 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.fileoperations.{AtomicFileOperations, AtomicFileOperationsImpl, FileWriteOperation}
 import org.apache.carbondata.core.metadata.converter.{SchemaConverter, ThriftWrapperSchemaConverterImpl}
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension, CarbonMeasure, ColumnSchema}
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
@@ -153,7 +153,7 @@ object CarbonDataStoreCreator {
     val id: ColumnSchema = new ColumnSchema()
     id.setColumnName("ID")
     id.setColumnar(true)
-    id.setDataType(DataType.INT)
+    id.setDataType(DataTypes.INT)
     id.setEncodingList(encodings)
     id.setColumnUniqueId(UUID.randomUUID().toString)
     id.setColumnReferenceId(id.getColumnUniqueId)
@@ -169,7 +169,7 @@ object CarbonDataStoreCreator {
     val date: ColumnSchema = new ColumnSchema()
     date.setColumnName("date")
     date.setColumnar(true)
-    date.setDataType(DataType.DATE)
+    date.setDataType(DataTypes.DATE)
     date.setEncodingList(dictEncoding)
     date.setColumnUniqueId(UUID.randomUUID().toString)
     date.setDimensionColumn(true)
@@ -180,7 +180,7 @@ object CarbonDataStoreCreator {
     val country: ColumnSchema = new ColumnSchema()
     country.setColumnName("country")
     country.setColumnar(true)
-    country.setDataType(DataType.STRING)
+    country.setDataType(DataTypes.STRING)
     country.setEncodingList(encodings)
     country.setColumnUniqueId(UUID.randomUUID().toString)
     country.setColumnReferenceId(country.getColumnUniqueId)
@@ -192,7 +192,7 @@ object CarbonDataStoreCreator {
     val name: ColumnSchema = new ColumnSchema()
     name.setColumnName("name")
     name.setColumnar(true)
-    name.setDataType(DataType.STRING)
+    name.setDataType(DataTypes.STRING)
     name.setEncodingList(encodings)
     name.setColumnUniqueId(UUID.randomUUID().toString)
     name.setDimensionColumn(true)
@@ -203,7 +203,7 @@ object CarbonDataStoreCreator {
     val phonetype: ColumnSchema = new ColumnSchema()
     phonetype.setColumnName("phonetype")
     phonetype.setColumnar(true)
-    phonetype.setDataType(DataType.STRING)
+    phonetype.setDataType(DataTypes.STRING)
     phonetype.setEncodingList(encodings)
     phonetype.setColumnUniqueId(UUID.randomUUID().toString)
     phonetype.setDimensionColumn(true)
@@ -214,7 +214,7 @@ object CarbonDataStoreCreator {
     val serialname: ColumnSchema = new ColumnSchema()
     serialname.setColumnName("serialname")
     serialname.setColumnar(true)
-    serialname.setDataType(DataType.STRING)
+    serialname.setDataType(DataTypes.STRING)
     serialname.setEncodingList(encodings)
     serialname.setColumnUniqueId(UUID.randomUUID().toString)
     serialname.setDimensionColumn(true)
@@ -225,7 +225,7 @@ object CarbonDataStoreCreator {
     val salary: ColumnSchema = new ColumnSchema()
     salary.setColumnName("salary")
     salary.setColumnar(true)
-    salary.setDataType(DataType.DOUBLE)
+    salary.setDataType(DataTypes.DOUBLE)
     salary.setEncodingList(encodings)
     salary.setColumnUniqueId(UUID.randomUUID().toString)
     salary.setDimensionColumn(false)
@@ -236,7 +236,7 @@ object CarbonDataStoreCreator {
     val bonus: ColumnSchema = new ColumnSchema()
     bonus.setColumnName("bonus")
     bonus.setColumnar(true)
-    bonus.setDataType(DataType.DECIMAL)
+    bonus.setDataType(DataTypes.DECIMAL)
     bonus.setPrecision(10)
     bonus.setScale(4)
     bonus.setEncodingList(encodings)
@@ -249,7 +249,7 @@ object CarbonDataStoreCreator {
     val dob: ColumnSchema = new ColumnSchema()
     dob.setColumnName("dob")
     dob.setColumnar(true)
-    dob.setDataType(DataType.TIMESTAMP)
+    dob.setDataType(DataTypes.TIMESTAMP)
     dob.setEncodingList(dictEncoding)
     dob.setColumnUniqueId(UUID.randomUUID().toString)
     dob.setDimensionColumn(true)
@@ -260,7 +260,7 @@ object CarbonDataStoreCreator {
     val shortField: ColumnSchema = new ColumnSchema()
     shortField.setColumnName("shortField")
     shortField.setColumnar(true)
-    shortField.setDataType(DataType.SHORT)
+    shortField.setDataType(DataTypes.SHORT)
     shortField.setEncodingList(encodings)
     shortField.setColumnUniqueId(UUID.randomUUID().toString)
     shortField.setDimensionColumn(false)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index 0b5141e..ba6ad31 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -31,7 +31,7 @@ import org.apache.carbondata.core.datastore.page.ColumnPage
 import org.apache.carbondata.core.events.ChangeEvent
 import org.apache.carbondata.core.indexstore.schema.FilterType
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.util.CarbonProperties
 
 class C2DataMapFactory() extends DataMapFactory {
@@ -163,7 +163,7 @@ object DataMapWriterSuite {
         pageId: Int,
         pages: Array[ColumnPage]): Unit = {
       assert(pages.length == 1)
-      assert(pages(0).getDataType == DataType.STRING)
+      assert(pages(0).getDataType == DataTypes.STRING)
       val bytes: Array[Byte] = pages(0).getByteArrayPage()(0)
       assert(bytes.sameElements(Seq(0, 1, 'b'.toByte)))
       callbackSeq :+= s"add page data: blocklet $blockletId, page $pageId"

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
index c0dba74..3f99922 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
@@ -18,9 +18,10 @@
 package org.apache.carbondata.spark.testsuite.partition
 
 import org.scalatest.BeforeAndAfterAll
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.util.CarbonProperties
@@ -53,7 +54,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.INT)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.INT)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
     assert(partitionInfo.getPartitionType ==  PartitionType.HASH)
     assert(partitionInfo.getNumPartitions == 3)
@@ -76,7 +77,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.TIMESTAMP)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
@@ -103,7 +104,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("workgroupcategory"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.STRING)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.STRING)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 1)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.INVERTED_INDEX)
     assert(partitionInfo.getPartitionType == PartitionType.LIST)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
index 1d660e8..317e2e2 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
@@ -19,7 +19,7 @@ package org.apache.carbondata.spark.testsuite.partition
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.util.CarbonProperties
@@ -48,7 +48,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.INT)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.INT)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
     assert(partitionInfo.getPartitionType ==  PartitionType.HASH)
     assert(partitionInfo.getNumPartitions == 3)
@@ -71,7 +71,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.TIMESTAMP)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
@@ -99,7 +99,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.TIMESTAMP)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
@@ -131,7 +131,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.DATE)
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.DATE)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
     assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
index a038ff3..bf46f67 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
@@ -17,8 +17,6 @@
 
 package org.apache.carbondata.spark.rdd
 
-import java.io.{ByteArrayInputStream, DataInputStream}
-
 import scala.collection.JavaConverters._
 import scala.reflect.ClassTag
 
@@ -26,7 +24,7 @@ import org.apache.spark.{Dependency, OneToOneDependency, Partition, SparkContext
 import org.apache.spark.rdd.RDD
 
 import org.apache.carbondata.core.metadata.schema.table.TableInfo
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, CarbonTaskInfo, SessionParams, TaskMetricsMap, ThreadLocalSessionInfo, ThreadLocalTaskInfo}
+import org.apache.carbondata.core.util._
 
 /**
  * This RDD maintains session level ThreadLocal

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
index d38be0a..80cbab8 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
@@ -36,7 +36,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.datastore.block.{Distributable, SegmentProperties, TaskBlockInfo}
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonDimension, CarbonMeasure}
@@ -204,7 +204,7 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
       } else {  // normal dictionary
         val dict = CarbonLoaderUtil.getDictionary(carbonTableIdentifier,
           dimension.getColumnIdentifier, storePath, partitionDataType)
-        if (partitionDataType == DataType.STRING) {
+        if (partitionDataType == DataTypes.STRING) {
           if (partitionType == PartitionType.RANGE) {
             partitionValue = ByteUtil.
               toBytes(dict.getDictionaryValueForKey(keyArray(partColIdx).toInt))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index 65f2ba2..cf37a18 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -25,26 +25,26 @@ import org.apache.spark.sql.execution.command.DataTypeInfo
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonDataType}
+import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonDataType, DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 
 object CarbonScalaUtil {
   def convertSparkToCarbonDataType(
       dataType: org.apache.spark.sql.types.DataType): CarbonDataType = {
     dataType match {
-      case StringType => CarbonDataType.STRING
-      case ShortType => CarbonDataType.SHORT
-      case IntegerType => CarbonDataType.INT
-      case LongType => CarbonDataType.LONG
-      case DoubleType => CarbonDataType.DOUBLE
-      case FloatType => CarbonDataType.FLOAT
-      case DateType => CarbonDataType.DATE
-      case BooleanType => CarbonDataType.BOOLEAN
-      case TimestampType => CarbonDataType.TIMESTAMP
-      case ArrayType(_, _) => CarbonDataType.ARRAY
-      case StructType(_) => CarbonDataType.STRUCT
-      case NullType => CarbonDataType.NULL
-      case _ => CarbonDataType.DECIMAL
+      case StringType => CarbonDataTypes.STRING
+      case ShortType => CarbonDataTypes.SHORT
+      case IntegerType => CarbonDataTypes.INT
+      case LongType => CarbonDataTypes.LONG
+      case DoubleType => CarbonDataTypes.DOUBLE
+      case FloatType => CarbonDataTypes.FLOAT
+      case DateType => CarbonDataTypes.DATE
+      case BooleanType => CarbonDataTypes.BOOLEAN
+      case TimestampType => CarbonDataTypes.TIMESTAMP
+      case ArrayType(_, _) => CarbonDataTypes.ARRAY
+      case StructType(_) => CarbonDataTypes.STRUCT
+      case NullType => CarbonDataTypes.NULL
+      case _ => CarbonDataTypes.DECIMAL
     }
   }
 
@@ -67,15 +67,15 @@ object CarbonScalaUtil {
 
   def convertCarbonToSparkDataType(dataType: CarbonDataType): types.DataType = {
     dataType match {
-      case CarbonDataType.STRING => StringType
-      case CarbonDataType.SHORT => ShortType
-      case CarbonDataType.INT => IntegerType
-      case CarbonDataType.LONG => LongType
-      case CarbonDataType.DOUBLE => DoubleType
-      case CarbonDataType.BOOLEAN => BooleanType
-      case CarbonDataType.DECIMAL => DecimalType.SYSTEM_DEFAULT
-      case CarbonDataType.TIMESTAMP => TimestampType
-      case CarbonDataType.DATE => DateType
+      case CarbonDataTypes.STRING => StringType
+      case CarbonDataTypes.SHORT => ShortType
+      case CarbonDataTypes.INT => IntegerType
+      case CarbonDataTypes.LONG => LongType
+      case CarbonDataTypes.DOUBLE => DoubleType
+      case CarbonDataTypes.BOOLEAN => BooleanType
+      case CarbonDataTypes.DECIMAL => DecimalType.SYSTEM_DEFAULT
+      case CarbonDataTypes.TIMESTAMP => TimestampType
+      case CarbonDataTypes.DATE => DateType
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index df25a37..bc24c12 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.datastore.row.LoadStatusType
 import org.apache.carbondata.core.memory.{UnsafeMemoryManager, UnsafeSortMemoryManager}
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
@@ -328,14 +328,14 @@ object CommonUtil {
     }
     val comparator = Comparator.getComparator(columnDataType)
     var head = columnDataType match {
-      case DataType.STRING => ByteUtil.toBytes(rangeInfo.head)
+      case DataTypes.STRING => ByteUtil.toBytes(rangeInfo.head)
       case _ => PartitionUtil.getDataBasedOnDataType(rangeInfo.head, columnDataType,
         timestampFormatter, dateFormatter)
     }
     val iterator = rangeInfo.tail.toIterator
     while (iterator.hasNext) {
       val next = columnDataType match {
-        case DataType.STRING => ByteUtil.toBytes(iterator.next())
+        case DataTypes.STRING => ByteUtil.toBytes(iterator.next())
         case _ => PartitionUtil.getDataBasedOnDataType(iterator.next(), columnDataType,
           timestampFormatter, dateFormatter)
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
index fef542a..027c654 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.spark.util
 
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.format.{DataType => ThriftDataType}
 
 object DataTypeConverterUtil {
@@ -26,51 +26,51 @@ object DataTypeConverterUtil {
 
   def convertToCarbonType(dataType: String): DataType = {
     dataType.toLowerCase match {
-      case "string" => DataType.STRING
-      case "int" => DataType.INT
-      case "integer" => DataType.INT
-      case "tinyint" => DataType.SHORT
-      case "smallint" => DataType.SHORT
-      case "long" => DataType.LONG
-      case "bigint" => DataType.LONG
-      case "numeric" => DataType.DOUBLE
-      case "double" => DataType.DOUBLE
-      case "float" => DataType.DOUBLE
-      case "decimal" => DataType.DECIMAL
-      case FIXED_DECIMAL(_, _) => DataType.DECIMAL
-      case "timestamp" => DataType.TIMESTAMP
-      case "date" => DataType.DATE
-      case "array" => DataType.ARRAY
-      case "struct" => DataType.STRUCT
+      case "string" => DataTypes.STRING
+      case "int" => DataTypes.INT
+      case "integer" => DataTypes.INT
+      case "tinyint" => DataTypes.SHORT
+      case "smallint" => DataTypes.SHORT
+      case "long" => DataTypes.LONG
+      case "bigint" => DataTypes.LONG
+      case "numeric" => DataTypes.DOUBLE
+      case "double" => DataTypes.DOUBLE
+      case "float" => DataTypes.DOUBLE
+      case "decimal" => DataTypes.DECIMAL
+      case FIXED_DECIMAL(_, _) => DataTypes.DECIMAL
+      case "timestamp" => DataTypes.TIMESTAMP
+      case "date" => DataTypes.DATE
+      case "array" => DataTypes.ARRAY
+      case "struct" => DataTypes.STRUCT
       case _ => convertToCarbonTypeForSpark2(dataType)
     }
   }
 
   def convertToCarbonTypeForSpark2(dataType: String): DataType = {
     dataType.toLowerCase match {
-      case "stringtype" => DataType.STRING
-      case "inttype" => DataType.INT
-      case "integertype" => DataType.INT
-      case "tinyinttype" => DataType.SHORT
-      case "shorttype" => DataType.SHORT
-      case "longtype" => DataType.LONG
-      case "biginttype" => DataType.LONG
-      case "numerictype" => DataType.DOUBLE
-      case "doubletype" => DataType.DOUBLE
-      case "floattype" => DataType.DOUBLE
-      case "decimaltype" => DataType.DECIMAL
-      case FIXED_DECIMALTYPE(_, _) => DataType.DECIMAL
-      case "timestamptype" => DataType.TIMESTAMP
-      case "datetype" => DataType.DATE
+      case "stringtype" => DataTypes.STRING
+      case "inttype" => DataTypes.INT
+      case "integertype" => DataTypes.INT
+      case "tinyinttype" => DataTypes.SHORT
+      case "shorttype" => DataTypes.SHORT
+      case "longtype" => DataTypes.LONG
+      case "biginttype" => DataTypes.LONG
+      case "numerictype" => DataTypes.DOUBLE
+      case "doubletype" => DataTypes.DOUBLE
+      case "floattype" => DataTypes.DOUBLE
+      case "decimaltype" => DataTypes.DECIMAL
+      case FIXED_DECIMALTYPE(_, _) => DataTypes.DECIMAL
+      case "timestamptype" => DataTypes.TIMESTAMP
+      case "datetype" => DataTypes.DATE
       case others =>
         if (others != null && others.startsWith("arraytype")) {
-          DataType.ARRAY
+          DataTypes.ARRAY
         } else if (others != null && others.startsWith("structtype")) {
-          DataType.STRUCT
+          DataTypes.STRUCT
         } else if (others != null && others.startsWith("char")) {
-          DataType.STRING
+          DataTypes.STRING
         } else if (others != null && others.startsWith("varchar")) {
-          DataType.STRING
+          DataTypes.STRING
         } else {
           sys.error(s"Unsupported data type: $dataType")
         }
@@ -79,17 +79,17 @@ object DataTypeConverterUtil {
 
   def convertToString(dataType: DataType): String = {
     dataType match {
-      case DataType.STRING => "string"
-      case DataType.SHORT => "smallint"
-      case DataType.INT => "int"
-      case DataType.LONG => "bigint"
-      case DataType.DOUBLE => "double"
-      case DataType.FLOAT => "double"
-      case DataType.DECIMAL => "decimal"
-      case DataType.TIMESTAMP => "timestamp"
-      case DataType.DATE => "date"
-      case DataType.ARRAY => "array"
-      case DataType.STRUCT => "struct"
+      case DataTypes.STRING => "string"
+      case DataTypes.SHORT => "smallint"
+      case DataTypes.INT => "int"
+      case DataTypes.LONG => "bigint"
+      case DataTypes.DOUBLE => "double"
+      case DataTypes.FLOAT => "double"
+      case DataTypes.DECIMAL => "decimal"
+      case DataTypes.TIMESTAMP => "timestamp"
+      case DataTypes.DATE => "date"
+      case DataTypes.ARRAY => "array"
+      case DataTypes.STRUCT => "struct"
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index c121960..ddc4763 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -43,7 +43,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonDimension, ColumnSchema}
 import org.apache.carbondata.core.reader.CarbonDictionaryReader
@@ -260,11 +260,11 @@ object GlobalDictionaryUtil {
         None
       case Some(dim) =>
         dim.getDataType match {
-          case DataType.ARRAY =>
+          case DataTypes.ARRAY =>
             val arrDim = ArrayParser(dim, format)
             generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, arrDim)
             Some(arrDim)
-          case DataType.STRUCT =>
+          case DataTypes.STRUCT =>
             val stuDim = StructParser(dim, format)
             generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, stuDim)
             Some(stuDim)
@@ -478,7 +478,7 @@ object GlobalDictionaryUtil {
       // for Array, user set ArrayFiled: path, while ArrayField has a child Array.val
       val currentColName = {
         preDictDimension.getDataType match {
-          case DataType.ARRAY =>
+          case DataTypes.ARRAY =>
             if (children(0).isComplex) {
               "val." + colName.substring(middleDimName.length + 1)
             } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 7d4dd49..16301d6 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -35,7 +35,7 @@ import org.apache.spark.util.PartitionUtils
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
@@ -622,14 +622,14 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         dimFields += field
       } else if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
         dimFields += field
-      } else if (DataTypeUtil.getDataType(field.dataType.get.toUpperCase) == DataType.TIMESTAMP &&
+      } else if (DataTypeUtil.getDataType(field.dataType.get.toUpperCase) == DataTypes.TIMESTAMP &&
                  !dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
         noDictionaryDims :+= field.column
         dimFields += field
       } else if (isDetectAsDimentionDatatype(field.dataType.get)) {
         dimFields += field
         // consider all String cols as noDicitonaryDims by default
-        if (DataType.STRING.getName.equalsIgnoreCase(field.dataType.get)) {
+        if (DataTypes.STRING.getName.equalsIgnoreCase(field.dataType.get)) {
           noDictionaryDims :+= field.column
         }
       } else if (sortKeyDimsTmp.exists(x => x.equalsIgnoreCase(field.column)) &&

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 1a39741..e5cfc84 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema._
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
@@ -309,10 +309,10 @@ class AlterTableColumnSchemaGenerator(
     if (alterTableModel.highCardinalityDims.contains(colName)) {
       encoders.remove(Encoding.DICTIONARY)
     }
-    if (dataType == DataType.DATE) {
+    if (dataType == DataTypes.DATE) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
-    if (dataType == DataType.TIMESTAMP && !alterTableModel.highCardinalityDims.contains(colName)) {
+    if (dataType == DataTypes.TIMESTAMP && !alterTableModel.highCardinalityDims.contains(colName)) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
     val colPropMap = new java.util.HashMap[String, String]()
@@ -378,10 +378,10 @@ class TableNewProcessor(cm: TableModel) {
     if (highCardinalityDims.contains(colName)) {
       encoders.remove(Encoding.DICTIONARY)
     }
-    if (dataType == DataType.DATE) {
+    if (dataType == DataTypes.DATE) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
-    if (dataType == DataType.TIMESTAMP && !highCardinalityDims.contains(colName)) {
+    if (dataType == DataTypes.TIMESTAMP && !highCardinalityDims.contains(colName)) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
     columnSchema.setEncodingList(encoders)
@@ -508,7 +508,7 @@ class TableNewProcessor(cm: TableModel) {
     // Adding dummy measure if no measure is provided
     if (measureCount == 0) {
       val encoders = new java.util.ArrayList[Encoding]()
-      val columnSchema: ColumnSchema = getColumnSchema(DataType.DOUBLE,
+      val columnSchema: ColumnSchema = getColumnSchema(DataTypes.DOUBLE,
         CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE,
         index,
         true,
@@ -537,7 +537,7 @@ class TableNewProcessor(cm: TableModel) {
         val col = allColumns.find(_.getColumnName.equalsIgnoreCase(b))
         col match {
           case Some(colSchema: ColumnSchema) =>
-            if (colSchema.isDimensionColumn && !colSchema.isComplex) {
+            if (colSchema.isDimensionColumn && !colSchema.getDataType.isComplexType) {
               colSchema
             } else {
               LOGGER.error(s"Bucket field must be dimension column and " +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java b/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index 62767fd..92c8402 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@ -21,7 +21,7 @@ import java.sql.Date;
 import java.sql.Timestamp;
 
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -52,27 +52,21 @@ public class SparkRowReadSupportImpl extends DictionaryDecodeReadSupport<Row> {
         if (data[i] == null) {
           continue;
         }
-        switch (dataTypes[i]) {
-          case STRING:
-            data[i] = UTF8String.fromString(data[i].toString());
-            break;
-          case TIMESTAMP:
-            data[i] = new Timestamp((long) data[i]);
-            break;
-          case DATE:
-            data[i] = new Date((long) data[i]);
-            break;
-          case LONG:
-            data[i] = data[i];
-            break;
-          default:
+        if (dataTypes[i] == DataTypes.STRING) {
+          data[i] = UTF8String.fromString(data[i].toString());
+        } else if (dataTypes[i] == DataTypes.TIMESTAMP) {
+          data[i] = new Timestamp((long) data[i]);
+        } else if (dataTypes[i] == DataTypes.DATE) {
+          data[i] = new Date((long) data[i]);
+        } else if (dataTypes[i] == DataTypes.LONG) {
+          data[i] = data[i];
         }
       }
       else if (carbonColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         //convert the long to timestamp in case of direct dictionary column
-        if (DataType.TIMESTAMP == carbonColumns[i].getDataType()) {
+        if (DataTypes.TIMESTAMP == carbonColumns[i].getDataType()) {
           data[i] = new Timestamp((long) data[i] / 1000L);
-        } else if (DataType.DATE == carbonColumns[i].getDataType()) {
+        } else if (DataTypes.DATE == carbonColumns[i].getDataType()) {
           data[i] = new Date((long) data[i]);
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
index 0d1b1df..7881b93 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.types._
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonType}
+import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonType}
 
 class CarbonDataFrameWriter(val dataFrame: DataFrame) {
 
@@ -161,8 +161,8 @@ class CarbonDataFrameWriter(val dataFrame: DataFrame) {
     sparkType match {
       case StringType => CarbonType.STRING.getName
       case IntegerType => CarbonType.INT.getName
-      case ShortType => CarbonType.SHORT.getName
-      case LongType => CarbonType.LONG.getName
+      case ShortType => "smallint"
+      case LongType => "bigint"
       case FloatType => CarbonType.DOUBLE.getName
       case DoubleType => CarbonType.DOUBLE.getName
       case TimestampType => CarbonType.TIMESTAMP.getName

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index cf14a79..6eeeaf9 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.datastore.block.{Distributable, TableBlockInfo
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion}
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
@@ -1047,7 +1047,7 @@ object CarbonDataRDDFactory {
     }
 
     val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-    if (partitionColumnDataType == DataType.STRING) {
+    if (partitionColumnDataType == DataTypes.STRING) {
       if (partitionInfo.getPartitionType == PartitionType.RANGE) {
         inputRDD.map { row => (ByteUtil.toBytes(row._1), row._2) }
           .partitionBy(partitioner)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
index 2f65fbc..118249a 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
@@ -38,7 +38,7 @@ object CarbonSparkUtil {
       carbonTable.getDimensionByTableName(carbonTable.getFactTableName).asScala.map { f =>
         (f.getColName.toLowerCase,
             f.hasEncoding(Encoding.DICTIONARY) && !f.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-                !CarbonUtil.hasComplexDataType(f.getDataType))
+                !f.getDataType.isComplexType)
       }
     CarbonMetaData(dimensionsAttr, measureAttr, carbonTable, DictionaryMap(dictionary.toMap))
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
index 7e449b6..2e93a6c 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
@@ -162,7 +162,7 @@ case class CarbonRelation(
     extends LeafNode with MultiInstanceRelation {
 
   def recursiveMethod(dimName: String, childDim: CarbonDimension): String = {
-    childDim.getDataType.toString.toLowerCase match {
+    childDim.getDataType.getName.toLowerCase match {
       case "array" => s"${
         childDim.getColName.substring(dimName.length + 1)
       }:array<${ getArrayChildren(childDim.getColName) }>"
@@ -175,7 +175,7 @@ case class CarbonRelation(
 
   def getArrayChildren(dimName: String): String = {
     metaData.carbonTable.getChildren(dimName).asScala.map(childDim => {
-      childDim.getDataType.toString.toLowerCase match {
+      childDim.getDataType.getName.toLowerCase match {
         case "array" => s"array<${ getArrayChildren(childDim.getColName) }>"
         case "struct" => s"struct<${ getStructChildren(childDim.getColName) }>"
         case dType => addDecimalScaleAndPrecision(childDim, dType)
@@ -185,7 +185,7 @@ case class CarbonRelation(
 
   def getStructChildren(dimName: String): String = {
     metaData.carbonTable.getChildren(dimName).asScala.map(childDim => {
-      childDim.getDataType.toString.toLowerCase match {
+      childDim.getDataType.getName.toLowerCase match {
         case "array" => s"${
           childDim.getColName.substring(dimName.length + 1)
         }:array<${ getArrayChildren(childDim.getColName) }>"
@@ -213,8 +213,7 @@ case class CarbonRelation(
       .map(dim => {
       val dimval = metaData.carbonTable
           .getDimensionByName(metaData.carbonTable.getFactTableName, dim.getColName)
-      val output: DataType = dimval.getDataType
-          .toString.toLowerCase match {
+      val output: DataType = dimval.getDataType.getName.toLowerCase match {
         case "array" =>
           CarbonMetastoreTypes.toDataType(s"array<${ getArrayChildren(dim.getColName) }>")
         case "struct" =>
@@ -238,7 +237,7 @@ case class CarbonRelation(
           getMeasureByTableName(tableMeta.carbonTable.getFactTableName).
           asScala.asJava).asScala.toSeq.filter(!_.getColumnSchema.isInvisible)
         .map(x => AttributeReference(x.getColName, CarbonMetastoreTypes.toDataType(
-          metaData.carbonTable.getMeasureByName(factTable, x.getColName).getDataType.toString
+          metaData.carbonTable.getMeasureByName(factTable, x.getColName).getDataType.getName
               .toLowerCase match {
             case "float" => "double"
             case "decimal" => "decimal(" + x.getPrecision + "," + x.getScale + ")"
@@ -252,7 +251,7 @@ case class CarbonRelation(
         .asScala
     columns.filter(!_.isInvisible).map { column =>
       if (column.isDimension()) {
-        val output: DataType = column.getDataType.toString.toLowerCase match {
+        val output: DataType = column.getDataType.getName.toLowerCase match {
           case "array" =>
             CarbonMetastoreTypes.toDataType(s"array<${getArrayChildren(column.getColName)}>")
           case "struct" =>
@@ -266,8 +265,7 @@ case class CarbonRelation(
         )(qualifiers = tableName +: alias.toSeq)
       } else {
         AttributeReference(column.getColName, CarbonMetastoreTypes.toDataType(
-          column.getDataType.toString
-            .toLowerCase match {
+          column.getDataType.getName.toLowerCase match {
             case "float" => "double"
             case "decimal" => "decimal(" + column.getColumnSchema.getPrecision + "," + column
               .getColumnSchema.getScale + ")"
@@ -292,7 +290,7 @@ case class CarbonRelation(
 
   def addDecimalScaleAndPrecision(dimval: CarbonColumn, dataType: String): String = {
     var dType = dataType
-    if (dimval.getDataType == org.apache.carbondata.core.metadata.datatype.DataType.DECIMAL) {
+    if (dimval.getDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL) {
       dType +=
           "(" + dimval.getColumnSchema.getPrecision + "," + dimval.getColumnSchema.getScale + ")"
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 37505d0..c14a61a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.types._
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.datatype.DataType
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
@@ -96,13 +97,13 @@ case class CarbonDictionaryDecoder(
   def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
       relation: CarbonRelation): types.DataType = {
     carbonDimension.getDataType match {
-      case DataType.STRING => StringType
-      case DataType.SHORT => ShortType
-      case DataType.INT => IntegerType
-      case DataType.LONG => LongType
-      case DataType.DOUBLE => DoubleType
-      case DataType.BOOLEAN => BooleanType
-      case DataType.DECIMAL =>
+      case CarbonDataTypes.STRING => StringType
+      case CarbonDataTypes.SHORT => ShortType
+      case CarbonDataTypes.INT => IntegerType
+      case CarbonDataTypes.LONG => LongType
+      case CarbonDataTypes.DOUBLE => DoubleType
+      case CarbonDataTypes.BOOLEAN => BooleanType
+      case CarbonDataTypes.DECIMAL =>
         val scale: Int = carbonDimension.getColumnSchema.getScale
         val precision: Int = carbonDimension.getColumnSchema.getPrecision
         if (scale == 0 && precision == 0) {
@@ -110,12 +111,12 @@ case class CarbonDictionaryDecoder(
         } else {
           DecimalType(precision, scale)
         }
-      case DataType.TIMESTAMP => TimestampType
-      case DataType.DATE => DateType
-      case DataType.STRUCT =>
+      case CarbonDataTypes.TIMESTAMP => TimestampType
+      case CarbonDataTypes.DATE => DateType
+      case CarbonDataTypes.STRUCT =>
         CarbonMetastoreTypes
           .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
-      case DataType.ARRAY =>
+      case CarbonDataTypes.ARRAY =>
         CarbonMetastoreTypes
           .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
index c827827..bc62a55 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
@@ -39,7 +39,7 @@ object CarbonSparkUtil {
       carbonTable.getDimensionByTableName(carbonTable.getFactTableName).asScala.map { f =>
         (f.getColName.toLowerCase,
             f.hasEncoding(Encoding.DICTIONARY) && !f.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-                !CarbonUtil.hasComplexDataType(f.getDataType))
+                !f.getDataType.isComplexType)
       }
     CarbonMetaData(dimensionsAttr, measureAttr, carbonTable, DictionaryMap(dictionary.toMap))
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index 8fc367b..2e45954 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.optimizer.AttributeReferenceWrapper
 import org.apache.spark.sql.sources
 import org.apache.spark.sql.types._
 
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
@@ -120,9 +120,9 @@ object CarbonFilters {
     def getCarbonLiteralExpression(name: String, value: Any): CarbonExpression = {
       val dataTypeOfAttribute = CarbonScalaUtil.convertSparkToCarbonDataType(dataTypeOf(name))
       val dataType = if (Option(value).isDefined
-                         && dataTypeOfAttribute == DataType.STRING
+                         && dataTypeOfAttribute == CarbonDataTypes.STRING
                          && value.isInstanceOf[Double]) {
-        DataType.DOUBLE
+        CarbonDataTypes.DOUBLE
       } else {
         dataTypeOfAttribute
       }
@@ -410,11 +410,11 @@ object CarbonFilters {
     } else {
       carbonColumn = carbonTable.getMeasureByName(carbonTable.getFactTableName, column)
       carbonColumn.getDataType match {
-        case DataType.INT => DataType.INT
-        case DataType.SHORT => DataType.SHORT
-        case DataType.LONG => DataType.LONG
-        case DataType.DECIMAL => DataType.DECIMAL
-        case _ => DataType.DOUBLE
+        case CarbonDataTypes.INT => CarbonDataTypes.INT
+        case CarbonDataTypes.SHORT => CarbonDataTypes.SHORT
+        case CarbonDataTypes.LONG => CarbonDataTypes.LONG
+        case CarbonDataTypes.DECIMAL => CarbonDataTypes.DECIMAL
+        case _ => CarbonDataTypes.DOUBLE
       }
     }
     CarbonScalaUtil.convertCarbonToSparkDataType(dataType)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index e1946a9..1183d94 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
@@ -198,28 +199,24 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
             null);
       } else {
         fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-            CarbonScalaUtil.convertCarbonToSparkDataType(DataType.INT), true, null);
+            CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.INT), true, null);
       }
     }
 
     for (int i = 0; i < queryMeasures.size(); i++) {
       QueryMeasure msr = queryMeasures.get(i);
-      switch (msr.getMeasure().getDataType()) {
-        case SHORT:
-        case INT:
-        case LONG:
-          fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
-              CarbonScalaUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
-              null);
-          break;
-        case DECIMAL:
-          fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
-              new DecimalType(msr.getMeasure().getPrecision(),
-                  msr.getMeasure().getScale()), true, null);
-          break;
-        default:
-          fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
-              CarbonScalaUtil.convertCarbonToSparkDataType(DataType.DOUBLE), true, null);
+      DataType dataType = msr.getMeasure().getDataType();
+      if (dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) {
+        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+            CarbonScalaUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
+            null);
+      } else if (dataType == DataTypes.DECIMAL) {
+        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+            new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true,
+            null);
+      } else {
+        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+            CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.DOUBLE), true, null);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 5b76c79..87de8ae 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -46,7 +46,7 @@ import org.apache.carbondata.core.datastore.block.{Distributable, TableBlockInfo
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnarFormatVersion}
-import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
@@ -1233,7 +1233,7 @@ object CarbonDataRDDFactory {
     }
 
     val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-    if (partitionColumnDataType == DataType.STRING) {
+    if (partitionColumnDataType == DataTypes.STRING) {
       if (partitionInfo.getPartitionType == PartitionType.RANGE) {
         inputRDD.map { row => (ByteUtil.toBytes(row._1), row._2) }
           .partitionBy(partitioner)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
index de7f3fb..5dd5983 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
@@ -40,7 +40,7 @@ object CarbonSparkUtil {
       carbonTable.getDimensionByTableName(carbonTable.getFactTableName).asScala.map { f =>
         (f.getColName.toLowerCase,
             f.hasEncoding(Encoding.DICTIONARY) && !f.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-                !CarbonUtil.hasComplexDataType(f.getDataType))
+                !f.getDataType.isComplexType)
       }
     CarbonMetaData(dimensionsAttr, measureAttr, carbonTable, DictionaryMap(dictionary.toMap))
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
index a12d86b..98a37fa 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.types._
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonType}
+import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonType}
 import org.apache.carbondata.spark.CarbonOption
 
 class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {


[05/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java b/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
index 5681364..a43ed0f 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
@@ -20,26 +20,25 @@ package org.apache.carbondata.core.util.comparator;
 import java.math.BigDecimal;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 
 public final class Comparator {
 
   public static SerializableComparator getComparator(DataType dataType) {
-    switch (dataType) {
-      case INT:
-        return new IntSerializableComparator();
-      case SHORT:
-        return new ShortSerializableComparator();
-      case DOUBLE:
-        return new DoubleSerializableComparator();
-      case LONG:
-      case DATE:
-      case TIMESTAMP:
-        return new LongSerializableComparator();
-      case DECIMAL:
-        return new BigDecimalSerializableComparator();
-      default:
-        return new ByteArraySerializableComparator();
+    if (dataType == DataTypes.INT) {
+      return new IntSerializableComparator();
+    } else if (dataType == DataTypes.SHORT) {
+      return new ShortSerializableComparator();
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new DoubleSerializableComparator();
+    } else if (dataType == DataTypes.LONG || dataType == DataTypes.DATE
+        || dataType == DataTypes.TIMESTAMP) {
+      return new LongSerializableComparator();
+    } else if (dataType == DataTypes.DECIMAL) {
+      return new BigDecimalSerializableComparator();
+    } else {
+      return new ByteArraySerializableComparator();
     }
   }
 
@@ -50,19 +49,18 @@ public final class Comparator {
    * @return
    */
   public static SerializableComparator getComparatorByDataTypeForMeasure(DataType dataType) {
-    switch (dataType) {
-      case INT:
-        return new IntSerializableComparator();
-      case SHORT:
-        return new ShortSerializableComparator();
-      case LONG:
-        return new LongSerializableComparator();
-      case DOUBLE:
-        return new DoubleSerializableComparator();
-      case DECIMAL:
-        return new BigDecimalSerializableComparator();
-      default:
-        throw new IllegalArgumentException("Unsupported data type");
+    if (dataType == DataTypes.INT) {
+      return new IntSerializableComparator();
+    } else if (dataType == DataTypes.SHORT) {
+      return new ShortSerializableComparator();
+    } else if (dataType == DataTypes.LONG) {
+      return new LongSerializableComparator();
+    } else if (dataType == DataTypes.DOUBLE) {
+      return new DoubleSerializableComparator();
+    } else if (dataType == DataTypes.DECIMAL) {
+      return new BigDecimalSerializableComparator();
+    } else {
+      throw new IllegalArgumentException("Unsupported data type");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
index 5235dd7..4396d6a 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
@@ -22,6 +22,7 @@ import java.util.Date;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
@@ -61,68 +62,64 @@ public class CarbonDictionarySortModel implements Comparable<CarbonDictionarySor
    * Compare
    */
   @Override public int compareTo(CarbonDictionarySortModel o) {
-    switch (dataType) {
-      case SHORT:
-      case INT:
-      case LONG:
-      case DOUBLE:
-
-        Double d1 = null;
-        Double d2 = null;
-        try {
-          d1 = new Double(memberValue);
-        } catch (NumberFormatException e) {
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
-            return -1;
-          }
-          return 1;
-        }
-        try {
-          d2 = new Double(o.memberValue);
-        } catch (NumberFormatException e) {
+    if (dataType == DataTypes.SHORT ||
+        dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG ||
+        dataType == DataTypes.DOUBLE) {
+      Double d1 = null;
+      Double d2 = null;
+      try {
+        d1 = new Double(memberValue);
+      } catch (NumberFormatException e) {
+        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
           return -1;
         }
-        return d1.compareTo(d2);
-      case DECIMAL:
-        java.math.BigDecimal val1 = null;
-        java.math.BigDecimal val2 = null;
-        try {
-          val1 = new java.math.BigDecimal(memberValue);
-        } catch (NumberFormatException e) {
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
-            return -1;
-          }
-          return 1;
-        }
-        try {
-          val2 = new java.math.BigDecimal(o.memberValue);
-        } catch (NumberFormatException e) {
+        return 1;
+      }
+      try {
+        d2 = new Double(o.memberValue);
+      } catch (NumberFormatException e) {
+        return -1;
+      }
+      return d1.compareTo(d2);
+    } else if (dataType == DataTypes.DECIMAL) {
+      java.math.BigDecimal val1 = null;
+      java.math.BigDecimal val2 = null;
+      try {
+        val1 = new java.math.BigDecimal(memberValue);
+      } catch (NumberFormatException e) {
+        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
           return -1;
         }
-        return val1.compareTo(val2);
-      case DATE:
-      case TIMESTAMP:
-        String format = CarbonUtil.getFormatFromProperty(dataType);
-        SimpleDateFormat parser = new SimpleDateFormat(format);
-        Date date1 = null;
-        Date date2 = null;
-        try {
-          date1 = parser.parse(memberValue);
-        } catch (ParseException e) {
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
-            return -1;
-          }
-          return 1;
-        }
-        try {
-          date2 = parser.parse(o.memberValue);
-        } catch (ParseException e) {
+        return 1;
+      }
+      try {
+        val2 = new java.math.BigDecimal(o.memberValue);
+      } catch (NumberFormatException e) {
+        return -1;
+      }
+      return val1.compareTo(val2);
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      String format = CarbonUtil.getFormatFromProperty(dataType);
+      SimpleDateFormat parser = new SimpleDateFormat(format);
+      Date date1 = null;
+      Date date2 = null;
+      try {
+        date1 = parser.parse(memberValue);
+      } catch (ParseException e) {
+        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
           return -1;
         }
-        return date1.compareTo(date2);
-      case STRING:
-      default:
-        return this.memberValue.compareTo(o.memberValue);
+        return 1;
+      }
+      try {
+        date2 = parser.parse(o.memberValue);
+      } catch (ParseException e) {
+        return -1;
+      }
+      return date1.compareTo(date2);
+    } else {
+      return this.memberValue.compareTo(o.memberValue);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
index 957ea22..d945f2f 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
@@ -103,9 +104,9 @@ public class AbstractDictionaryCacheTest {
 
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
       String columnId) {
-	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataType.STRING);
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.STRING);
     return new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
-        DataType.STRING,
+        DataTypes.STRING,
         CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
index 1784364..33121e2 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
@@ -24,6 +24,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import mockit.Mock;
@@ -41,7 +42,7 @@ public class ColumnDictionaryInfoTest {
   private ColumnDictionaryInfo columnDictionaryInfo;
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionary() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     List<String> evaluateResultList = Arrays.asList("china", "france");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -79,7 +80,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithZeroSurrogate() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     List<String> evaluateResultList = Arrays.asList("china", "france");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -107,7 +108,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithNullValue() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     List<String> evaluateResultList = Arrays.asList("@NU#LL$!");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -134,7 +135,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithTypeException() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.INT);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
 
     List<String> evaluateResultList = Arrays.asList("china", "france");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -173,7 +174,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithDoubleType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.DOUBLE);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.DOUBLE);
 
     List<String> evaluateResultList = Arrays.asList("15999");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -208,7 +209,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithIntType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.INT);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
 
     List<String> evaluateResultList = Arrays.asList("998");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -243,7 +244,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithDecimalType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.DECIMAL);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.DECIMAL);
 
     List<String> evaluateResultList = Arrays.asList("150011.550");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -280,7 +281,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithLongType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.LONG);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.LONG);
 
     List<String> evaluateResultList = Arrays.asList("1500115505555");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
@@ -324,7 +325,7 @@ public class ColumnDictionaryInfoTest {
       }
     };
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
 
@@ -347,7 +348,7 @@ public class ColumnDictionaryInfoTest {
       }
     };
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
 
@@ -375,7 +376,7 @@ public class ColumnDictionaryInfoTest {
       }
     };
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
 
@@ -402,7 +403,7 @@ public class ColumnDictionaryInfoTest {
       }
     };
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
 
@@ -423,7 +424,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGtSortedIndexWithMinimumSurrogateKey() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
 
@@ -435,7 +436,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGtSortedIndexWithMaximumSurrogateKey() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
 
@@ -447,7 +448,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGtSortedIndexWithSurrogateKey() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
 
@@ -459,7 +460,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGetSizeOfLastDictionaryChunkWithDictionaryChunkZero() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     final int result = columnDictionaryInfo.getSizeOfLastDictionaryChunk();
 
@@ -469,7 +470,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGetSizeOfLastDictionaryChunk() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
 
@@ -485,7 +486,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGetDictionaryValueFromSortedIndexWithMinimumSurrogateKey() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
 
@@ -496,7 +497,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGetDictionaryValueFromSortedIndexWithMaximumSurrogateKey() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
 
@@ -507,7 +508,7 @@ public class ColumnDictionaryInfoTest {
 
   @Test public void testGetDictionaryValueFromSortedIndex() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(0, 1, 2, 3));
 
@@ -521,7 +522,7 @@ public class ColumnDictionaryInfoTest {
   @Test
   public void testGetSurrogateKey() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
 
     byte[] value = convertListElementsIntoByteArray(Arrays.asList("china")).get(0);
 
@@ -553,7 +554,7 @@ public class ColumnDictionaryInfoTest {
   @Test
   public void testGetSurrogateKeyWithIntType() {
 
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataType.INT);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
 
     byte[] value = convertListElementsIntoByteArray(Arrays.asList("998")).get(0);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
index a751120..59e348b 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
@@ -26,7 +26,7 @@ import java.util.Map;
 
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.reader.CarbonDictionaryReaderImpl;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
@@ -51,13 +51,13 @@ public class DictionaryCacheLoaderImplTest {
     Map<String, String> columnProperties = new HashMap<>();
     columnProperties.put("prop1", "value1");
     columnProperties.put("prop2", "value2");
-    columnIdentifier = new ColumnIdentifier("1", columnProperties, DataType.STRING);
+    columnIdentifier = new ColumnIdentifier("1", columnProperties, DataTypes.STRING);
     dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType(), CarbonStorePath.getCarbonTablePath("/tmp", carbonTableIdentifier));
     dictionaryCacheLoader = new DictionaryCacheLoaderImpl(carbonTableIdentifier, "/tmp/",
         dictionaryColumnUniqueIdentifier);
-    dictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    dictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
     new MockUp<CarbonDictionaryReaderImpl>() {
       @Mock @SuppressWarnings("unused") Iterator<byte[]> read(long startOffset, long endOffset)
           throws IOException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
index 0e2eed9..ec9a19a 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
@@ -16,20 +16,22 @@
  */
 package org.apache.carbondata.core.cache.dictionary;
 
-import mockit.Mock;
-import mockit.MockUp;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
+import mockit.Mock;
+import mockit.MockUp;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.util.HashMap;
-import java.util.Map;
-
-import static junit.framework.TestCase.*;
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertNotNull;
+import static junit.framework.TestCase.assertTrue;
 
 public class DictionaryColumnUniqueIdentifierTest {
 
@@ -43,21 +45,21 @@ public class DictionaryColumnUniqueIdentifierTest {
     CarbonTableIdentifier carbonTableIdentifier2 =
         new CarbonTableIdentifier("testDatabase", "testTable", "2");
     Map<String, String> properties = new HashMap<>();
-    ColumnIdentifier columnIdentifier = new ColumnIdentifier("2", properties, DataType.STRING);
-    ColumnIdentifier columnIdentifier2 = new ColumnIdentifier("1", properties, DataType.INT);
+    ColumnIdentifier columnIdentifier = new ColumnIdentifier("2", properties, DataTypes.STRING);
+    ColumnIdentifier columnIdentifier2 = new ColumnIdentifier("1", properties, DataTypes.INT);
     dictionaryColumnUniqueIdentifier1 =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier1, columnIdentifier,
-            DataType.MAP, null);
+            DataTypes.MAP, null);
     dictionaryColumnUniqueIdentifier2 =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier2, columnIdentifier2,
-            DataType.MAP, null);
+            DataTypes.MAP, null);
     dictionaryColumnUniqueIdentifier3 =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier2, columnIdentifier,
-            DataType.MAP, null);
+            DataTypes.MAP, null);
   }
 
   @Test public void testToGetDataType() {
-    assertEquals(dictionaryColumnUniqueIdentifier1.getDataType(), DataType.MAP);
+    assertEquals(dictionaryColumnUniqueIdentifier1.getDataType(), DataTypes.MAP);
   }
 
   @Test public void testForEqualsWithDifferentObjectsWithDifferentColumnIdentifier() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java
index 559dcb0..e84eb06 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java
@@ -20,6 +20,7 @@ import mockit.Mock;
 import mockit.MockUp;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -35,7 +36,7 @@ public class ForwardDictionaryTest {
   private static ForwardDictionary forwardDictionary;
 
   @BeforeClass public static void setUp() {
-    ColumnDictionaryInfo columnDictionaryInfo = new ColumnDictionaryInfo(DataType.INT);
+    ColumnDictionaryInfo columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
     forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
index b06fc4d..d3d5658 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.util.CarbonProperties;
 
@@ -270,7 +271,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   }
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
 	      String columnId) {
-	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataType.DOUBLE);
+	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.DOUBLE);
     return new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
 	  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
index d5af5c3..f7696f7 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/ColumnIdentifierTest.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.carbon;
 
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -35,7 +36,7 @@ public class ColumnIdentifierTest {
   @BeforeClass public static void setup() {
     columnProperties = new HashMap<String, String>();
     columnProperties.put("key", "value");
-    columnIdentifier = new ColumnIdentifier("columnId", columnProperties, DataType.INT);
+    columnIdentifier = new ColumnIdentifier("columnId", columnProperties, DataTypes.INT);
   }
 
   @Test public void hashCodeTest() {
@@ -51,7 +52,7 @@ public class ColumnIdentifierTest {
 
   @Test public void equalsTestwithSimilarObject() {
     ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier("columnId", columnProperties, DataType.INT);
+        new ColumnIdentifier("columnId", columnProperties, DataTypes.INT);
     Boolean res = columnIdentifier.equals(columnIdentifierTest);
     assert (res);
   }
@@ -68,14 +69,14 @@ public class ColumnIdentifierTest {
 
   @Test public void equalsTestwithNullColumnId() {
     ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier(null, columnProperties, DataType.INT);
+        new ColumnIdentifier(null, columnProperties, DataTypes.INT);
     Boolean res = columnIdentifierTest.equals(columnIdentifier);
     assert (!res);
   }
 
   @Test public void equalsTestwithDiffColumnId() {
     ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier("diffColumnId", columnProperties, DataType.INT);
+        new ColumnIdentifier("diffColumnId", columnProperties, DataTypes.INT);
     Boolean res = columnIdentifierTest.equals(columnIdentifier);
     assert (!res);
   }
@@ -87,7 +88,7 @@ public class ColumnIdentifierTest {
 
   @Test public void getColumnPropertyTest() {
     ColumnIdentifier columnIdentifierTest =
-        new ColumnIdentifier("diffColumnId", null, DataType.INT);
+        new ColumnIdentifier("diffColumnId", null, DataTypes.INT);
     String res = columnIdentifierTest.getColumnProperty("key");
     assertEquals(res, null);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
index f6defec..2020dd8 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
@@ -24,10 +24,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.UUID;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 
 import junit.framework.TestCase;
 import org.junit.BeforeClass;
@@ -173,7 +173,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -188,7 +188,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI1");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -202,7 +202,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI2");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -218,7 +218,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI3");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -234,7 +234,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI4");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -248,7 +248,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI9");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -264,7 +264,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI10");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -280,7 +280,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI11");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -296,7 +296,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI5");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.ARRAY);
+    dimColumn.setDataType(DataTypes.ARRAY);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -311,7 +311,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI6");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -325,7 +325,7 @@ public class SegmentPropertiesTest extends TestCase {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     encodeList.add(Encoding.DELTA);
@@ -337,7 +337,7 @@ public class SegmentPropertiesTest extends TestCase {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT1");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     encodeList.add(Encoding.DELTA);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
index 5a941b3..1f9c712 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -55,7 +56,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -70,7 +71,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI1");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -84,7 +85,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI2");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -100,7 +101,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI3");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -116,7 +117,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI4");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -130,7 +131,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI9");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -146,7 +147,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI10");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -162,7 +163,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(false);
     dimColumn.setColumnName("IMEI11");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -178,7 +179,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI5");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.ARRAY);
+    dimColumn.setDataType(DataTypes.ARRAY);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -193,7 +194,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI6");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -207,7 +208,7 @@ public class SegmentPropertiesTestUtil {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     encodeList.add(Encoding.DELTA);
@@ -219,7 +220,7 @@ public class SegmentPropertiesTestUtil {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT1");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     encodeList.add(Encoding.DELTA);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java b/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
index 79d3388..be898c2 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.datastore.page.encoding.rle.RLEEncoderMeta;
 import org.apache.carbondata.core.datastore.page.statistics.PrimitivePageStatsCollector;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -45,9 +46,9 @@ public class RLECodecSuite {
     TestData(byte[] inputByteData, byte[] expectedEncodedByteData) throws IOException, MemoryException {
       this.inputByteData = inputByteData;
       inputBytePage = ColumnPage.newPage(
-          new TableSpec.ColumnSpec("test", DataType.BYTE, ColumnType.MEASURE),
-          DataType.BYTE, inputByteData.length);
-      inputBytePage.setStatsCollector(PrimitivePageStatsCollector.newInstance(DataType.BYTE, 0, 0));
+          new TableSpec.ColumnSpec("test", DataTypes.BYTE, ColumnType.MEASURE),
+          DataTypes.BYTE, inputByteData.length);
+      inputBytePage.setStatsCollector(PrimitivePageStatsCollector.newInstance(DataTypes.BYTE, 0, 0));
       for (int i = 0; i < inputByteData.length; i++) {
         inputBytePage.putData(i, inputByteData[i]);
       }
@@ -130,8 +131,8 @@ public class RLECodecSuite {
   private void testBytePageDecode(byte[] inputBytes, byte[] expectedDecodedBytes) throws IOException, MemoryException {
     RLECodec codec = new RLECodec();
     RLEEncoderMeta meta = new RLEEncoderMeta(
-        new TableSpec.ColumnSpec("test", DataType.BYTE, ColumnType.MEASURE),
-        DataType.BYTE, expectedDecodedBytes.length, null);
+        new TableSpec.ColumnSpec("test", DataTypes.BYTE, ColumnType.MEASURE),
+        DataTypes.BYTE, expectedDecodedBytes.length, null);
     ColumnPageDecoder decoder = codec.createDecoder(meta);
     ColumnPage page = decoder.decode(inputBytes, 0, inputBytes.length);
     byte[] decoded = page.getBytePage();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java b/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
index 52cda28..c077eeb 100644
--- a/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.keygenerator.directdictionary;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -30,13 +31,13 @@ public class DirectDictionaryKeyGeneratorFactoryUnitTest {
 
     int expectedResult = 1;
     DirectDictionaryGenerator result =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.TIMESTAMP);
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
     assertEquals (expectedResult,result.generateDirectSurrogateKey("TimeStamp"));
   }
 
   @Test public void testGetDirectDictionaryGeneratorReturnNull() throws Exception {
     DirectDictionaryGenerator result =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.ARRAY);
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.ARRAY);
     Assert.assertNull(result);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
index 3af0bdb..ddb5e3a 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
@@ -120,7 +121,7 @@ public class CarbonMetadataTest {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -134,7 +135,7 @@ public class CarbonMetadataTest {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     return dimColumn;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
index 3961d9c..8bce684 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
@@ -265,7 +266,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.BOOLEAN;
+        return DataTypes.BOOLEAN;
       }
 
       @Mock public String getColumnName() {
@@ -349,7 +350,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.BOOLEAN;
+        return DataTypes.BOOLEAN;
       }
 
       @Mock public String getColumnName() {
@@ -426,7 +427,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.BOOLEAN;
+        return DataTypes.BOOLEAN;
       }
 
       @Mock public String getColumnName() {
@@ -500,7 +501,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.STRING;
+        return DataTypes.STRING;
       }
 
       @Mock public String getColumnName() {
@@ -575,7 +576,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.INT;
+        return DataTypes.INT;
       }
 
       @Mock public String getColumnName() {
@@ -648,7 +649,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.SHORT;
+        return DataTypes.SHORT;
       }
 
       @Mock public String getColumnName() {
@@ -722,7 +723,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.LONG;
+        return DataTypes.LONG;
       }
 
       @Mock public String getColumnName() {
@@ -796,7 +797,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.DOUBLE;
+        return DataTypes.DOUBLE;
       }
 
       @Mock public String getColumnName() {
@@ -869,7 +870,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.DECIMAL;
+        return DataTypes.DECIMAL;
       }
 
       @Mock public String getColumnName() {
@@ -944,7 +945,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.TIMESTAMP;
+        return DataTypes.TIMESTAMP;
       }
 
       @Mock public String getColumnName() {
@@ -1018,7 +1019,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.ARRAY;
+        return DataTypes.ARRAY;
       }
 
       @Mock public String getColumnName() {
@@ -1092,7 +1093,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.STRUCT;
+        return DataTypes.STRUCT;
       }
 
       @Mock public String getColumnName() {
@@ -1241,7 +1242,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.BOOLEAN;
+        return DataTypes.BOOLEAN;
       }
 
       @Mock public String getColumnName() {
@@ -1326,7 +1327,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.BOOLEAN;
+        return DataTypes.BOOLEAN;
       }
 
       @Mock public String getColumnName() {
@@ -1444,7 +1445,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataType.BOOLEAN;
+        return DataTypes.BOOLEAN;
       }
 
       @Mock public String getColumnName() {
@@ -1539,7 +1540,7 @@ long time =1112745600000L;
     wrapperColumnSchema.setColumnUniqueId("1");
     wrapperColumnSchema.setColumnName("columnName");
     wrapperColumnSchema.setColumnar(true);
-    wrapperColumnSchema.setDataType(DataType.STRING);
+    wrapperColumnSchema.setDataType(DataTypes.STRING);
     wrapperColumnSchema.setDimensionColumn(true);
     wrapperColumnSchema.setEncodingList(encodings);
     wrapperColumnSchema.setNumberOfChild(1);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
index f5ffe51..fa5765a 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -73,7 +74,7 @@ public class CarbonTableTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -87,7 +88,7 @@ public class CarbonTableTest extends TestCase {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     return dimColumn;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
index 69cab49..4979038 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
@@ -20,10 +20,10 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 
 import junit.framework.TestCase;
 import org.junit.AfterClass;
@@ -70,7 +70,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -83,7 +83,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     structColumn.setColumnar(true);
     structColumn.setColumnName("mobile");
     structColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    structColumn.setDataType(DataType.STRUCT);
+    structColumn.setDataType(DataTypes.STRUCT);
     structColumn.setDimensionColumn(true);
     structColumn.setEncodingList(encodeList);
     structColumn.setNumberOfChild(2);
@@ -93,7 +93,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     primitiveColumn.setColumnar(true);
     primitiveColumn.setColumnName("mobile.stdcode");
     primitiveColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    primitiveColumn.setDataType(DataType.STRING);
+    primitiveColumn.setDataType(DataTypes.STRING);
     primitiveColumn.setDimensionColumn(true);
     primitiveColumn.setEncodingList(encodeList);
     primitiveColumn.setNumberOfChild(0);
@@ -103,7 +103,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     arrayColumn.setColumnar(true);
     arrayColumn.setColumnName("mobile.val");
     arrayColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    arrayColumn.setDataType(DataType.ARRAY);
+    arrayColumn.setDataType(DataTypes.ARRAY);
     arrayColumn.setDimensionColumn(true);
     arrayColumn.setEncodingList(encodeList);
     arrayColumn.setNumberOfChild(1);
@@ -113,7 +113,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     primitiveColumn1.setColumnar(true);
     primitiveColumn1.setColumnName("mobile.val.phoneno");
     primitiveColumn1.setColumnUniqueId(UUID.randomUUID().toString());
-    primitiveColumn1.setDataType(DataType.STRING);
+    primitiveColumn1.setDataType(DataTypes.STRING);
     primitiveColumn1.setDimensionColumn(true);
     primitiveColumn1.setEncodingList(encodeList);
     primitiveColumn1.setNumberOfChild(0);
@@ -126,7 +126,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(false);
     return dimColumn;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
index 55de2e2..85bdfd8 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
@@ -49,7 +49,7 @@ public class RawBasedResultCollectorTest {
 //    aggregatorInfo.setMeasureExists(new boolean[] { true, false, false, false });
 //    aggregatorInfo.setDefaultValues(new Object[] { 1, 2, 3, 4 });
 //    aggregatorInfo.setMeasureDataTypes(
-//        new DataType[] { DataType.INT, DataType.TIMESTAMP, DataType.INT, DataType.INT });
+//        new DataType[] { DataTypes.INT, DataTypes.TIMESTAMP, DataTypes.INT, DataTypes.INT });
 //    QueryMeasure queryMeasure1 = new QueryMeasure("QMCol1");
 //    queryMeasure1.setQueryOrder(1);
 //    QueryMeasure queryMeasure2 = new QueryMeasure("QMCol2");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
index 6beb81c..0915d78 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
@@ -25,6 +25,7 @@ import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.keygenerator.mdkey.Bits;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 import mockit.Mock;
@@ -53,27 +54,27 @@ public class PrimitiveQueryTypeTest {
     int keySize = 1;
     boolean isDirectDictionary = true;
     primitiveQueryType =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.STRING, keySize, dictionary,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.STRING, keySize, dictionary,
             isDirectDictionary);
     primitiveQueryTypeForInt =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.INT, keySize, dictionary,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.INT, keySize, dictionary,
             isDirectDictionary);
     primitiveQueryTypeForDouble =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.DOUBLE, keySize, dictionary,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.DOUBLE, keySize, dictionary,
             isDirectDictionary);
     primitiveQueryTypeForLong =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.LONG, keySize, dictionary,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.LONG, keySize, dictionary,
             isDirectDictionary);
     primitiveQueryTypeForBoolean =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.BOOLEAN, keySize, dictionary,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.BOOLEAN, keySize, dictionary,
             isDirectDictionary);
     primitiveQueryTypeForTimeStamp =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.TIMESTAMP, keySize,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.TIMESTAMP, keySize,
             dictionary, isDirectDictionary);
-    ColumnDictionaryInfo columnDictionaryInfo = new ColumnDictionaryInfo(DataType.STRING);
+    ColumnDictionaryInfo columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
     ForwardDictionary forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
     primitiveQueryTypeForTimeStampForIsDictionaryFalse =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataType.TIMESTAMP, keySize,
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.TIMESTAMP, keySize,
             forwardDictionary, false);
 
   }
@@ -141,11 +142,11 @@ public class PrimitiveQueryTypeTest {
     Object actualData = null;
     if (isDirectDictionary) {
       DirectDictionaryGenerator directDictionaryGenerator =
-          DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.TIMESTAMP);
+          DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
       actualData = directDictionaryGenerator.getValueFromSurrogate(surrgateValue);
     } else {
       String dictionaryValueForKey = dictionary.getDictionaryValueForKey(surrgateValue);
-      actualData = DataTypeUtil.getDataBasedOnDataType(dictionaryValueForKey, DataType.TIMESTAMP);
+      actualData = DataTypeUtil.getDataBasedOnDataType(dictionaryValueForKey, DataTypes.TIMESTAMP);
     }
     return actualData;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
index df14381..048b194 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
@@ -16,7 +16,12 @@
  */
 package org.apache.carbondata.core.scan.executor.util;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -32,11 +37,6 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.UUID;
-
 public class RestructureUtilTest {
 
   @Test public void testToGetUpdatedQueryDimension() {
@@ -99,15 +99,15 @@ public class RestructureUtilTest {
   @Test public void testToGetAggregatorInfos() {
     ColumnSchema columnSchema1 = new ColumnSchema();
     columnSchema1.setColumnName("Id");
-    columnSchema1.setDataType(DataType.STRING);
+    columnSchema1.setDataType(DataTypes.STRING);
     columnSchema1.setColumnUniqueId(UUID.randomUUID().toString());
     ColumnSchema columnSchema2 = new ColumnSchema();
     columnSchema2.setColumnName("Name");
-    columnSchema2.setDataType(DataType.STRING);
+    columnSchema2.setDataType(DataTypes.STRING);
     columnSchema2.setColumnUniqueId(UUID.randomUUID().toString());
     ColumnSchema columnSchema3 = new ColumnSchema();
     columnSchema3.setColumnName("Age");
-    columnSchema3.setDataType(DataType.STRING);
+    columnSchema3.setDataType(DataTypes.STRING);
     columnSchema3.setColumnUniqueId(UUID.randomUUID().toString());
 
     CarbonMeasure carbonMeasure1 = new CarbonMeasure(columnSchema1, 1);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/ColumnExpressionTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/ColumnExpressionTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/ColumnExpressionTest.java
index 865af20..f2147b2 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/ColumnExpressionTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/ColumnExpressionTest.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 
 import mockit.Mock;
@@ -33,7 +34,7 @@ public class ColumnExpressionTest {
 
   @Before public void setUp() {
     String columnName = "IMEI";
-    DataType dataType = DataType.STRING;
+    DataType dataType = DataTypes.STRING;
     columnExpression = new ColumnExpression(columnName, dataType);
   }
 
@@ -45,7 +46,7 @@ public class ColumnExpressionTest {
         return 1;
       }
     };
-    ExpressionResult expectedValue = new ExpressionResult(DataType.INT, 1);
+    ExpressionResult expectedValue = new ExpressionResult(DataTypes.INT, 1);
     assertEquals(expectedValue, columnExpression.evaluate(rowImpl));
   }
 


[07/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index 6a281dd..ddc75ff 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -217,21 +218,17 @@ public class RestructureUtil {
     Object noDictionaryDefaultValue = null;
     String value = null;
     if (!isDefaultValueNull(defaultValue)) {
-      switch (datatype) {
-        case INT:
-          value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          noDictionaryDefaultValue = Integer.parseInt(value);
-          break;
-        case LONG:
-          value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          noDictionaryDefaultValue = Long.parseLong(value);
-          break;
-        case TIMESTAMP:
-          long timestampValue = ByteUtil.toLong(defaultValue, 0, defaultValue.length);
-          noDictionaryDefaultValue = timestampValue * 1000L;
-          break;
-        default:
-          noDictionaryDefaultValue = UTF8String.fromBytes(defaultValue);
+      if (datatype == DataTypes.INT) {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        noDictionaryDefaultValue = Integer.parseInt(value);
+      } else if (datatype == DataTypes.LONG) {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        noDictionaryDefaultValue = Long.parseLong(value);
+      } else if (datatype == DataTypes.TIMESTAMP) {
+        long timestampValue = ByteUtil.toLong(defaultValue, 0, defaultValue.length);
+        noDictionaryDefaultValue = timestampValue * 1000L;
+      } else {
+        noDictionaryDefaultValue = UTF8String.fromBytes(defaultValue);
       }
     }
     return noDictionaryDefaultValue;
@@ -244,10 +241,7 @@ public class RestructureUtil {
    * @return
    */
   private static boolean isDefaultValueNull(byte[] defaultValue) {
-    if (null == defaultValue) {
-      return true;
-    }
-    return false;
+    return null == defaultValue;
   }
 
   /**
@@ -294,28 +288,22 @@ public class RestructureUtil {
     Object measureDefaultValue = null;
     if (!isDefaultValueNull(defaultValue)) {
       String value = null;
-      switch (columnSchema.getDataType()) {
-        case SHORT:
-        case INT:
-        case LONG:
-          value =
-              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          measureDefaultValue = Long.parseLong(value);
-          break;
-        case DECIMAL:
-          BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
-          if (columnSchema.getScale() > decimal.scale()) {
-            decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);
-          }
-          measureDefaultValue = decimal;
-          break;
-        default:
-          value =
-              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          Double parsedValue = Double.valueOf(value);
-          if (!Double.isInfinite(parsedValue) && !Double.isNaN(parsedValue)) {
-            measureDefaultValue = parsedValue;
-          }
+      DataType dataType = columnSchema.getDataType();
+      if (dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        measureDefaultValue = Long.parseLong(value);
+      } else if (dataType == DataTypes.DECIMAL) {
+        BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
+        if (columnSchema.getScale() > decimal.scale()) {
+          decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);
+        }
+        measureDefaultValue = decimal;
+      } else {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        Double parsedValue = Double.valueOf(value);
+        if (!Double.isInfinite(parsedValue) && !Double.isNaN(parsedValue)) {
+          measureDefaultValue = parsedValue;
+        }
       }
     }
     return measureDefaultValue;
@@ -333,36 +321,28 @@ public class RestructureUtil {
     Object measureDefaultValue = null;
     if (!isDefaultValueNull(defaultValue)) {
       String value = null;
-      switch (columnSchema.getDataType()) {
-        case SHORT:
-          value =
-              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          measureDefaultValue = Short.parseShort(value);
-          break;
-        case INT:
-          value =
-              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          measureDefaultValue = Integer.parseInt(value);
-          break;
-        case LONG:
-          value =
-              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          measureDefaultValue = Long.parseLong(value);
-          break;
-        case DECIMAL:
-          BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
-          if (columnSchema.getScale() > decimal.scale()) {
-            decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);
-          }
-          measureDefaultValue = Decimal.apply(decimal);
-          break;
-        default:
-          value =
-              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          Double parsedValue = Double.valueOf(value);
-          if (!Double.isInfinite(parsedValue) && !Double.isNaN(parsedValue)) {
-            measureDefaultValue = parsedValue;
-          }
+      DataType dataType = columnSchema.getDataType();
+      if (dataType == DataTypes.SHORT) {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        measureDefaultValue = Short.parseShort(value);
+      } else if (dataType == DataTypes.INT) {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        measureDefaultValue = Integer.parseInt(value);
+      } else if (dataType == DataTypes.LONG) {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        measureDefaultValue = Long.parseLong(value);
+      } else if (dataType == DataTypes.DECIMAL) {
+        BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
+        if (columnSchema.getScale() > decimal.scale()) {
+          decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);
+        }
+        measureDefaultValue = Decimal.apply(decimal);
+      } else {
+        value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        Double parsedValue = Double.valueOf(value);
+        if (!Double.isInfinite(parsedValue) && !Double.isNaN(parsedValue)) {
+          measureDefaultValue = parsedValue;
+        }
       }
     }
     return measureDefaultValue;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
index 08b1972..e3892be 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
@@ -29,6 +29,7 @@ import java.util.TimeZone;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.DateDirectDictionaryGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
 import org.apache.carbondata.core.util.CarbonUtil;
 
@@ -71,48 +72,48 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          try {
-            return Integer.parseInt(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterIllegalMemberException(e);
-          }
-        case SHORT:
-          return ((Short) value).intValue();
-        case INT:
-        case DOUBLE:
-          if (value instanceof Double) {
-            return ((Double) value).intValue();
-          }
-          if (value instanceof Long) {
-            return ((Long) value).intValue();
-          }
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        try {
+          return Integer.parseInt(value.toString());
+        } catch (NumberFormatException e) {
+          throw new FilterIllegalMemberException(e);
+        }
+      } else if (dataType == DataTypes.SHORT) {
+        return ((Short) value).intValue();
+      } else if (dataType == DataTypes.INT ||
+          dataType == DataTypes.DOUBLE) {
+        if (value instanceof Double) {
+          return ((Double) value).intValue();
+        }
+        if (value instanceof Long) {
+          return ((Long) value).intValue();
+        }
+        return (Integer) value;
+      } else if (dataType == DataTypes.DATE) {
+        if (value instanceof java.sql.Date) {
+          return (int) (((java.sql.Date) value).getTime());
+        } else {
           return (Integer) value;
-        case DATE:
-          if (value instanceof java.sql.Date) {
-            return (int) (((java.sql.Date) value).getTime());
-          } else {
-            return (Integer) value;
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (value instanceof Timestamp) {
+          return (int) (((Timestamp) value).getTime());
+        } else {
+          if (isLiteral) {
+            Long l = (Long) value / 1000;
+            return l.intValue();
           }
-        case TIMESTAMP:
-          if (value instanceof Timestamp) {
-            return (int) (((Timestamp) value).getTime());
-          } else {
-            if (isLiteral) {
-              Long l = (Long) value / 1000;
-              return l.intValue();
-            }
-            return (Integer) value;
-          }
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to integer type value");
+          return (Integer) value;
+        }
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to integer type value");
       }
 
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to Integer type value");
+          "Cannot convert" + this.getDataType().getName() + " to Integer type value");
     }
   }
 
@@ -121,51 +122,46 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          try {
-            return Short.parseShort(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterIllegalMemberException(e);
-          }
-        case SHORT:
-        case INT:
-        case DOUBLE:
-
-          if (value instanceof Double) {
-            return ((Double) value).shortValue();
-          } else if (value instanceof Integer) {
-            return ((Integer) value).shortValue();
-          }
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        try {
+          return Short.parseShort(value.toString());
+        } catch (NumberFormatException e) {
+          throw new FilterIllegalMemberException(e);
+        }
+      } else if (dataType == DataTypes.SHORT ||
+          dataType == DataTypes.INT ||
+          dataType == DataTypes.DOUBLE) {
+        if (value instanceof Double) {
+          return ((Double) value).shortValue();
+        } else if (value instanceof Integer) {
+          return ((Integer) value).shortValue();
+        }
+        return (Short) value;
+      } else if (dataType == DataTypes.DATE) {
+        if (value instanceof java.sql.Date) {
+          return (short) (((java.sql.Date) value).getTime());
+        } else {
           return (Short) value;
-
-        case DATE:
-
-          if (value instanceof java.sql.Date) {
-            return (short) (((java.sql.Date) value).getTime());
-          } else {
-            return (Short) value;
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (value instanceof Timestamp) {
+          return (short) (((Timestamp) value).getTime());
+        } else {
+          if (isLiteral) {
+            Long l = ((long) value / 1000);
+            return l.shortValue();
           }
-        case TIMESTAMP:
-
-          if (value instanceof Timestamp) {
-            return (short) (((Timestamp) value).getTime());
-          } else {
-            if (isLiteral) {
-              Long l = ((long) value / 1000);
-              return l.shortValue();
-            }
-            return (Short) value;
-          }
-
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to integer type value");
+          return (Short) value;
+        }
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to integer type value");
       }
 
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to Integer type value");
+          "Cannot convert" + this.getDataType().getName() + " to Integer type value");
     }
   }
 
@@ -174,34 +170,33 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case DATE:
-        case TIMESTAMP:
-          String format = CarbonUtil.getFormatFromProperty(this.getDataType());
-          SimpleDateFormat parser = new SimpleDateFormat(format);
-          if (this.getDataType() == DataType.DATE) {
-            parser.setTimeZone(TimeZone.getTimeZone("GMT"));
-          }
-          if (value instanceof Timestamp) {
-            return parser.format((Timestamp) value);
-          } else if (value instanceof java.sql.Date) {
-            return parser.format((java.sql.Date) value);
-          } else if (value instanceof Long) {
-            if (isLiteral) {
-              return parser.format(new Timestamp((long) value / 1000));
-            }
-            return parser.format(new Timestamp((long) value));
-          } else if (value instanceof Integer) {
-            long date = ((int) value) * DateDirectDictionaryGenerator.MILLIS_PER_DAY;
-            return parser.format(new java.sql.Date(date));
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+        String format = CarbonUtil.getFormatFromProperty(this.getDataType());
+        SimpleDateFormat parser = new SimpleDateFormat(format);
+        if (this.getDataType() == DataTypes.DATE) {
+          parser.setTimeZone(TimeZone.getTimeZone("GMT"));
+        }
+        if (value instanceof Timestamp) {
+          return parser.format((Timestamp) value);
+        } else if (value instanceof java.sql.Date) {
+          return parser.format((java.sql.Date) value);
+        } else if (value instanceof Long) {
+          if (isLiteral) {
+            return parser.format(new Timestamp((long) value / 1000));
           }
-          return value.toString();
-        default:
-          return value.toString();
+          return parser.format(new Timestamp((long) value));
+        } else if (value instanceof Integer) {
+          long date = ((int) value) * DateDirectDictionaryGenerator.MILLIS_PER_DAY;
+          return parser.format(new java.sql.Date(date));
+        }
+        return value.toString();
+      } else {
+        return value.toString();
       }
     } catch (Exception e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to String type value");
+          "Cannot convert" + this.getDataType().getName() + " to String type value");
     }
   }
 
@@ -210,44 +205,44 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          try {
-            return Double.parseDouble(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterIllegalMemberException(e);
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        try {
+          return Double.parseDouble(value.toString());
+        } catch (NumberFormatException e) {
+          throw new FilterIllegalMemberException(e);
+        }
+      } else if (dataType == DataTypes.SHORT) {
+        return ((Short) value).doubleValue();
+      } else if (dataType == DataTypes.INT) {
+        return ((Integer) value).doubleValue();
+      } else if (dataType == DataTypes.LONG) {
+        return ((Long) value).doubleValue();
+      } else if (dataType == DataTypes.DOUBLE) {
+        return (Double) value;
+      } else if (dataType == DataTypes.DATE) {
+        if (value instanceof java.sql.Date) {
+          return (double) ((java.sql.Date) value).getTime();
+        } else {
+          return (Double) (value);
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (value instanceof Timestamp) {
+          return (double) ((Timestamp) value).getTime();
+        } else {
+          if (isLiteral) {
+            Long l = (Long) value / 1000;
+            return l.doubleValue();
           }
-        case SHORT:
-          return ((Short) value).doubleValue();
-        case INT:
-          return ((Integer) value).doubleValue();
-        case LONG:
-          return ((Long) value).doubleValue();
-        case DOUBLE:
-          return (Double) value;
-        case DATE:
-          if (value instanceof java.sql.Date) {
-            return (double) ((java.sql.Date) value).getTime();
-          } else {
-            return (Double) (value);
-          }
-        case TIMESTAMP:
-          if (value instanceof Timestamp) {
-            return (double) ((Timestamp) value).getTime();
-          } else {
-            if (isLiteral) {
-              Long l = (Long) value / 1000;
-              return l.doubleValue();
-            }
-            return (Double) (value);
-          }
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to double type value");
+          return (Double) (value);
+        }
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to double type value");
       }
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to Double type value");
+          "Cannot convert" + this.getDataType().getName() + " to Double type value");
     }
   }
 
@@ -256,40 +251,40 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          try {
-            return Long.parseLong(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterIllegalMemberException(e);
-          }
-        case SHORT:
-          return ((Short) value).longValue();
-        case INT:
-          return (Long) value;
-        case LONG:
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        try {
+          return Long.parseLong(value.toString());
+        } catch (NumberFormatException e) {
+          throw new FilterIllegalMemberException(e);
+        }
+      } else if (dataType == DataTypes.SHORT) {
+        return ((Short) value).longValue();
+      } else if (dataType == DataTypes.INT) {
+        return (Long) value;
+      } else if (dataType == DataTypes.LONG) {
+        return (Long) value;
+      } else if (dataType == DataTypes.DOUBLE) {
+        return (Long) value;
+      } else if (dataType == DataTypes.DATE) {
+        if (value instanceof java.sql.Date) {
+          return ((java.sql.Date) value).getTime();
+        } else {
           return (Long) value;
-        case DOUBLE:
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (value instanceof Timestamp) {
+          return ((Timestamp) value).getTime();
+        } else {
           return (Long) value;
-        case DATE:
-          if (value instanceof java.sql.Date) {
-            return ((java.sql.Date) value).getTime();
-          } else {
-            return (Long) value;
-          }
-        case TIMESTAMP:
-          if (value instanceof Timestamp) {
-            return ((Timestamp) value).getTime();
-          } else {
-            return (Long) value;
-          }
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to Long type value");
+        }
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to Long type value");
       }
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(e.getMessage() + " " +
-          "Cannot convert" + this.getDataType().name() + " to Long type value");
+          "Cannot convert" + this.getDataType().getName() + " to Long type value");
     }
 
   }
@@ -300,44 +295,43 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          try {
-            return new BigDecimal(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterIllegalMemberException(e);
-          }
-        case SHORT:
-          return new BigDecimal((short) value);
-        case INT:
-          return new BigDecimal((int) value);
-        case LONG:
-          return new BigDecimal((long) value);
-        case DOUBLE:
-        case DECIMAL:
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        try {
           return new BigDecimal(value.toString());
-        case DATE:
-          if (value instanceof java.sql.Date) {
-            return new BigDecimal(((java.sql.Date) value).getTime());
-          } else {
-            return new BigDecimal((long) value);
-          }
-        case TIMESTAMP:
-          if (value instanceof Timestamp) {
-            return new BigDecimal(((Timestamp) value).getTime());
-          } else {
-            if (isLiteral) {
-              return new BigDecimal((long) value / 1000);
-            }
-            return new BigDecimal((long) value);
+        } catch (NumberFormatException e) {
+          throw new FilterIllegalMemberException(e);
+        }
+      } else if (dataType == DataTypes.SHORT) {
+        return new BigDecimal((short) value);
+      } else if (dataType == DataTypes.INT) {
+        return new BigDecimal((int) value);
+      } else if (dataType == DataTypes.LONG) {
+        return new BigDecimal((long) value);
+      } else if (dataType == DataTypes.DOUBLE || dataType == DataTypes.DECIMAL) {
+        return new BigDecimal(value.toString());
+      } else if (dataType == DataTypes.DATE) {
+        if (value instanceof java.sql.Date) {
+          return new BigDecimal(((java.sql.Date) value).getTime());
+        } else {
+          return new BigDecimal((long) value);
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (value instanceof Timestamp) {
+          return new BigDecimal(((Timestamp) value).getTime());
+        } else {
+          if (isLiteral) {
+            return new BigDecimal((long) value / 1000);
           }
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to Long type value");
+          return new BigDecimal((long) value);
+        }
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to Decimal type value");
       }
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to Long type value");
+          "Cannot convert" + this.getDataType().getName() + " to Decimal type value");
     }
 
   }
@@ -347,51 +341,50 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          // 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.
-          SimpleDateFormat parser =
-              new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-          Date dateToStr;
-          try {
-            dateToStr = parser.parse(value.toString());
-            return dateToStr.getTime();
-          } catch (ParseException e) {
-            throw new FilterIllegalMemberException(
-                "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
-          }
-        case SHORT:
-          return ((Short) value).longValue();
-        case INT:
-        case LONG:
-          return (Long) value;
-        case DOUBLE:
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        // 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.
+        SimpleDateFormat parser =
+            new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+        Date dateToStr;
+        try {
+          dateToStr = parser.parse(value.toString());
+          return dateToStr.getTime();
+        } catch (ParseException e) {
+          throw new FilterIllegalMemberException(
+              "Cannot convert" + this.getDataType().getName() + " to Time type value");
+        }
+      } else if (dataType == DataTypes.SHORT) {
+        return ((Short) value).longValue();
+      } else if (dataType == DataTypes.INT || dataType == DataTypes.LONG) {
+        return (Long) value;
+      } else if (dataType == DataTypes.DOUBLE) {
+        return (Long) value;
+      } else if (dataType == DataTypes.DATE) {
+        if (value instanceof java.sql.Date) {
+          return ((Date) value).getTime();
+        } else {
           return (Long) value;
-        case DATE:
-          if (value instanceof java.sql.Date) {
-            return ((Date) value).getTime();
-          } else {
-            return (Long) value;
-          }
-        case TIMESTAMP:
-          if (value instanceof Timestamp) {
-            return ((Timestamp) value).getTime();
-          } else {
-            if (isLiteral) {
-              return (Long) value / 1000;
-            }
-            return (Long) value;
+        }
+      } else if (dataType == DataTypes.TIMESTAMP) {
+        if (value instanceof Timestamp) {
+          return ((Timestamp) value).getTime();
+        } else {
+          if (isLiteral) {
+            return (Long) value / 1000;
           }
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
+          return (Long) value;
+        }
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to Time type value");
       }
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
+          "Cannot convert" + this.getDataType().getName() + " to Time type value");
     }
 
   }
@@ -401,24 +394,22 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       return null;
     }
     try {
-      switch (this.getDataType()) {
-        case STRING:
-          try {
-            return Boolean.parseBoolean(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterIllegalMemberException(e);
-          }
-
-        case BOOLEAN:
+      DataType dataType = this.getDataType();
+      if (dataType == DataTypes.STRING) {
+        try {
           return Boolean.parseBoolean(value.toString());
-
-        default:
-          throw new FilterIllegalMemberException(
-              "Cannot convert" + this.getDataType().name() + " to boolean type value");
+        } catch (NumberFormatException e) {
+          throw new FilterIllegalMemberException(e);
+        }
+      } else if (dataType == DataTypes.BOOLEAN) {
+        return Boolean.parseBoolean(value.toString());
+      } else {
+        throw new FilterIllegalMemberException(
+            "Cannot convert" + this.getDataType().getName() + " to boolean type value");
       }
     } catch (ClassCastException e) {
       throw new FilterIllegalMemberException(
-          "Cannot convert" + this.getDataType().name() + " to Boolean type value");
+          "Cannot convert" + this.getDataType().getName() + " to Boolean type value");
     }
   }
 
@@ -491,29 +482,20 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
       dataType = objToCompare.getDataType();
     }
     try {
-      switch (dataType) {
-        case STRING:
-          result = this.getString().equals(objToCompare.getString());
-          break;
-        case SHORT:
-          result = this.getShort().equals(objToCompare.getShort());
-          break;
-        case INT:
-          result = this.getInt().equals(objToCompare.getInt());
-          break;
-        case LONG:
-        case DATE:
-        case TIMESTAMP:
-          result = this.getLong().equals(objToCompare.getLong());
-          break;
-        case DOUBLE:
-          result = this.getDouble().equals(objToCompare.getDouble());
-          break;
-        case DECIMAL:
-          result = this.getDecimal().equals(objToCompare.getDecimal());
-          break;
-        default:
-          break;
+      if (dataType == DataTypes.STRING) {
+        result = this.getString().equals(objToCompare.getString());
+      } else if (dataType == DataTypes.SHORT) {
+        result = this.getShort().equals(objToCompare.getShort());
+      } else if (dataType == DataTypes.INT) {
+        result = this.getInt().equals(objToCompare.getInt());
+      } else if (dataType == DataTypes.LONG ||
+          dataType == DataTypes.DATE ||
+          dataType == DataTypes.TIMESTAMP) {
+        result = this.getLong().equals(objToCompare.getLong());
+      } else if (dataType == DataTypes.DOUBLE) {
+        result = this.getDouble().equals(objToCompare.getDouble());
+      } else if (dataType == DataTypes.DECIMAL) {
+        result = this.getDecimal().equals(objToCompare.getDecimal());
       }
     } catch (FilterIllegalMemberException ex) {
       return false;
@@ -528,32 +510,28 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
 
   @Override public int compareTo(ExpressionResult o) {
     try {
-      switch (o.dataType) {
-        case SHORT:
-        case INT:
-        case LONG:
-        case DOUBLE:
-          Double d1 = this.getDouble();
-          Double d2 = o.getDouble();
-          return d1.compareTo(d2);
-        case DECIMAL:
-          java.math.BigDecimal val1 = this.getDecimal();
-          java.math.BigDecimal val2 = o.getDecimal();
-          return val1.compareTo(val2);
-        case DATE:
-        case TIMESTAMP:
-          String format = CarbonUtil.getFormatFromProperty(o.dataType);
-          SimpleDateFormat parser = new SimpleDateFormat(format);
-          Date date1 = parser.parse(this.getString());
-          Date date2 = parser.parse(o.getString());
-          return date1.compareTo(date2);
-        case STRING:
-        default:
-          return this.getString().compareTo(o.getString());
+      DataType type = o.dataType;
+      if (type == DataTypes.SHORT ||
+          type == DataTypes.INT ||
+          type == DataTypes.LONG ||
+          type == DataTypes.DOUBLE) {
+        Double d1 = this.getDouble();
+        Double d2 = o.getDouble();
+        return d1.compareTo(d2);
+      } else if (type == DataTypes.DECIMAL) {
+        java.math.BigDecimal val1 = this.getDecimal();
+        java.math.BigDecimal val2 = o.getDecimal();
+        return val1.compareTo(val2);
+      } else if (type == DataTypes.DATE || type == DataTypes.TIMESTAMP) {
+        String format = CarbonUtil.getFormatFromProperty(o.dataType);
+        SimpleDateFormat parser = new SimpleDateFormat(format);
+        Date date1 = parser.parse(this.getString());
+        Date date2 = parser.parse(o.getString());
+        return date1.compareTo(date2);
+      } else {
+        return this.getString().compareTo(o.getString());
       }
-    } catch (ParseException e) {
-      return -1;
-    } catch (FilterIllegalMemberException e) {
+    } catch (ParseException | FilterIllegalMemberException e) {
       return -1;
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
index d98593c..9d418ee 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
@@ -27,6 +27,7 @@ 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.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.conditional.GreaterThanEqualToExpression;
 import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression;
 import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
@@ -279,8 +280,8 @@ public class RangeExpressionEvaluator {
         if (((ColumnExpression) exp).isDimension() == false) {
           return false;
         }
-        if ((((ColumnExpression) exp).getDimension().getDataType() == DataType.ARRAY) || (
-            ((ColumnExpression) exp).getDimension().getDataType() == DataType.STRUCT)) {
+        if ((((ColumnExpression) exp).getDimension().getDataType() == DataTypes.ARRAY) || (
+            ((ColumnExpression) exp).getDimension().getDataType() == DataTypes.STRUCT)) {
           return false;
         } else {
           return true;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
index bac54c8..780ca89 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.conditional;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -51,9 +52,9 @@ public class EqualToExpression extends BinaryConditionalExpression {
 
     if (elRes.isNull() || erRes.isNull()) {
       if (isNull) {
-        elRes.set(DataType.BOOLEAN, elRes.isNull() == erRes.isNull());
+        elRes.set(DataTypes.BOOLEAN, elRes.isNull() == erRes.isNull());
       } else {
-        elRes.set(DataType.BOOLEAN, false);
+        elRes.set(DataTypes.BOOLEAN, false);
       }
       return elRes;
     }
@@ -65,34 +66,26 @@ public class EqualToExpression extends BinaryConditionalExpression {
       }
     }
 
-    switch (val1.getDataType()) {
-      case STRING:
-        result = val1.getString().equals(val2.getString());
-        break;
-      case SHORT:
-        result = val1.getShort().equals(val2.getShort());
-        break;
-      case INT:
-        result = val1.getInt().equals(val2.getInt());
-        break;
-      case DOUBLE:
-        result = FilterUtil.nanSafeEqualsDoubles(val1.getDouble(), val2.getDouble());
-        break;
-      case DATE:
-      case TIMESTAMP:
-        result = val1.getTime().equals(val2.getTime());
-        break;
-      case LONG:
-        result = val1.getLong().equals(val2.getLong());
-        break;
-      case DECIMAL:
-        result = val1.getDecimal().compareTo(val2.getDecimal()) == 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    DataType dataType = val1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = val1.getString().equals(val2.getString());
+    } else if (dataType == DataTypes.SHORT) {
+      result = val1.getShort().equals(val2.getShort());
+    } else if (dataType == DataTypes.INT) {
+      result = val1.getInt().equals(val2.getInt());
+    } else if (dataType == DataTypes.DOUBLE) {
+      result = FilterUtil.nanSafeEqualsDoubles(val1.getDouble(), val2.getDouble());
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      result = val1.getTime().equals(val2.getTime());
+    } else if (dataType == DataTypes.LONG) {
+      result = val1.getLong().equals(val2.getLong());
+    } else if (dataType == DataTypes.DECIMAL) {
+      result = val1.getDecimal().compareTo(val2.getDecimal()) == 0;
+    } else {
+      throw new FilterUnsupportedException(
+          "DataType: " + val1.getDataType() + " not supported for the filter expression");
     }
-    val1.set(DataType.BOOLEAN, result);
+    val1.set(DataTypes.BOOLEAN, result);
     return val1;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
index d51ff01..469672e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.conditional;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -38,7 +39,7 @@ public class GreaterThanEqualToExpression extends BinaryConditionalExpression {
     ExpressionResult erRes = right.evaluate(value);
     ExpressionResult exprResVal1 = elRes;
     if (elRes.isNull() || erRes.isNull()) {
-      elRes.set(DataType.BOOLEAN, false);
+      elRes.set(DataTypes.BOOLEAN, false);
       return elRes;
     }
     if (elRes.getDataType() != erRes.getDataType()) {
@@ -48,34 +49,26 @@ public class GreaterThanEqualToExpression extends BinaryConditionalExpression {
 
     }
     boolean result = false;
-    switch (exprResVal1.getDataType()) {
-      case STRING:
-        result = elRes.getString().compareTo(erRes.getString()) >= 0;
-        break;
-      case SHORT:
-        result = elRes.getShort() >= (erRes.getShort());
-        break;
-      case INT:
-        result = elRes.getInt() >= (erRes.getInt());
-        break;
-      case DOUBLE:
-        result = elRes.getDouble() >= (erRes.getDouble());
-        break;
-      case DATE:
-      case TIMESTAMP:
-        result = elRes.getTime() >= (erRes.getTime());
-        break;
-      case LONG:
-        result = elRes.getLong() >= (erRes.getLong());
-        break;
-      case DECIMAL:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) >= 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + exprResVal1.getDataType() + " not supported for the filter expression");
+    DataType dataType = exprResVal1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = elRes.getString().compareTo(erRes.getString()) >= 0;
+    } else if (dataType == DataTypes.SHORT) {
+      result = elRes.getShort() >= (erRes.getShort());
+    } else if (dataType == DataTypes.INT) {
+      result = elRes.getInt() >= (erRes.getInt());
+    } else if (dataType == DataTypes.DOUBLE) {
+      result = elRes.getDouble() >= (erRes.getDouble());
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      result = elRes.getTime() >= (erRes.getTime());
+    } else if (dataType == DataTypes.LONG) {
+      result = elRes.getLong() >= (erRes.getLong());
+    } else if (dataType == DataTypes.DECIMAL) {
+      result = elRes.getDecimal().compareTo(erRes.getDecimal()) >= 0;
+    } else  {
+      throw new FilterUnsupportedException(
+          "DataType: " + exprResVal1.getDataType() + " not supported for the filter expression");
     }
-    exprResVal1.set(DataType.BOOLEAN, result);
+    exprResVal1.set(DataTypes.BOOLEAN, result);
     return exprResVal1;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
index 2c3dd7e..931de6c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.conditional;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -39,7 +40,7 @@ public class GreaterThanExpression extends BinaryConditionalExpression {
     ExpressionResult exprRightRes = right.evaluate(value);
     ExpressionResult val1 = exprLeftRes;
     if (exprLeftRes.isNull() || exprRightRes.isNull()) {
-      exprLeftRes.set(DataType.BOOLEAN, false);
+      exprLeftRes.set(DataTypes.BOOLEAN, false);
       return exprLeftRes;
     }
     if (exprLeftRes.getDataType() != exprRightRes.getDataType()) {
@@ -50,34 +51,26 @@ public class GreaterThanExpression extends BinaryConditionalExpression {
 
     }
     boolean result = false;
-    switch (val1.getDataType()) {
-      case STRING:
-        result = exprLeftRes.getString().compareTo(exprRightRes.getString()) > 0;
-        break;
-      case DOUBLE:
-        result = exprLeftRes.getDouble() > (exprRightRes.getDouble());
-        break;
-      case SHORT:
-        result = exprLeftRes.getShort() > (exprRightRes.getShort());
-        break;
-      case INT:
-        result = exprLeftRes.getInt() > (exprRightRes.getInt());
-        break;
-      case DATE:
-      case TIMESTAMP:
-        result = exprLeftRes.getTime() > (exprRightRes.getTime());
-        break;
-      case LONG:
-        result = exprLeftRes.getLong() > (exprRightRes.getLong());
-        break;
-      case DECIMAL:
-        result = exprLeftRes.getDecimal().compareTo(exprRightRes.getDecimal()) > 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    DataType dataType = val1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = exprLeftRes.getString().compareTo(exprRightRes.getString()) > 0;
+    } else if (dataType == DataTypes.DOUBLE) {
+      result = exprLeftRes.getDouble() > (exprRightRes.getDouble());
+    } else if (dataType == DataTypes.SHORT) {
+      result = exprLeftRes.getShort() > (exprRightRes.getShort());
+    } else if (dataType == DataTypes.INT) {
+      result = exprLeftRes.getInt() > (exprRightRes.getInt());
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      result = exprLeftRes.getTime() > (exprRightRes.getTime());
+    } else if (dataType == DataTypes.LONG) {
+      result = exprLeftRes.getLong() > (exprRightRes.getLong());
+    } else if (dataType == DataTypes.DECIMAL) {
+      result = exprLeftRes.getDecimal().compareTo(exprRightRes.getDecimal()) > 0;
+    } else {
+      throw new FilterUnsupportedException(
+          "DataType: " + val1.getDataType() + " not supported for the filter expression");
     }
-    val1.set(DataType.BOOLEAN, result);
+    val1.set(DataTypes.BOOLEAN, result);
     return val1;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
index 130063d..bc714f7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
@@ -21,6 +21,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -52,32 +53,24 @@ public class InExpression extends BinaryConditionalExpression {
         } else {
           val = expressionResVal;
         }
-        switch (val.getDataType()) {
-          case STRING:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getString());
-            break;
-          case SHORT:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getShort());
-            break;
-          case INT:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getInt());
-            break;
-          case DOUBLE:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getDouble());
-            break;
-          case LONG:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getLong());
-            break;
-          case DATE:
-          case TIMESTAMP:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getTime());
-            break;
-          case DECIMAL:
-            val = new ExpressionResult(val.getDataType(), expressionResVal.getDecimal());
-            break;
-          default:
-            throw new FilterUnsupportedException(
-                "DataType: " + val.getDataType() + " not supported for the filter expression");
+        DataType dataType = val.getDataType();
+        if (dataType == DataTypes.STRING) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getString());
+        } else if (dataType == DataTypes.SHORT) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getShort());
+        } else if (dataType == DataTypes.INT) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getInt());
+        } else if (dataType == DataTypes.DOUBLE) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getDouble());
+        } else if (dataType == DataTypes.LONG) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getLong());
+        } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getTime());
+        } else if (dataType == DataTypes.DECIMAL) {
+          val = new ExpressionResult(val.getDataType(), expressionResVal.getDecimal());
+        } else {
+          throw new FilterUnsupportedException(
+              "DataType: " + val.getDataType() + " not supported for the filter expression");
         }
         setOfExprResult.add(val);
       }
@@ -91,9 +84,9 @@ public class InExpression extends BinaryConditionalExpression {
     // for a check on the right result.
     // Example: (null==null) -> Left null return false, (1==null) would automatically be false.
     if (leftRsult.isNull()) {
-      leftRsult.set(DataType.BOOLEAN, false);
+      leftRsult.set(DataTypes.BOOLEAN, false);
     } else {
-      leftRsult.set(DataType.BOOLEAN, setOfExprResult.contains(leftRsult));
+      leftRsult.set(DataTypes.BOOLEAN, setOfExprResult.contains(leftRsult));
     }
     return leftRsult;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
index d61498b..f416a6b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.conditional;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -38,7 +39,7 @@ public class LessThanEqualToExpression extends BinaryConditionalExpression {
     ExpressionResult erRes = right.evaluate(value);
     ExpressionResult exprResValue1 = elRes;
     if (elRes.isNull() || erRes.isNull()) {
-      elRes.set(DataType.BOOLEAN, false);
+      elRes.set(DataTypes.BOOLEAN, false);
       return elRes;
     }
     if (elRes.getDataType() != erRes.getDataType()) {
@@ -48,34 +49,26 @@ public class LessThanEqualToExpression extends BinaryConditionalExpression {
 
     }
     boolean result = false;
-    switch (exprResValue1.getDataType()) {
-      case STRING:
-        result = elRes.getString().compareTo(erRes.getString()) <= 0;
-        break;
-      case SHORT:
-        result = elRes.getShort() <= (erRes.getShort());
-        break;
-      case INT:
-        result = elRes.getInt() <= (erRes.getInt());
-        break;
-      case DOUBLE:
-        result = elRes.getDouble() <= (erRes.getDouble());
-        break;
-      case DATE:
-      case TIMESTAMP:
-        result = elRes.getTime() <= (erRes.getTime());
-        break;
-      case LONG:
-        result = elRes.getLong() <= (erRes.getLong());
-        break;
-      case DECIMAL:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) <= 0;
-        break;
-      default:
-        throw new FilterUnsupportedException("DataType: " + exprResValue1.getDataType()
-            + " not supported for the filter expression");
+    DataType dataType = exprResValue1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = elRes.getString().compareTo(erRes.getString()) <= 0;
+    } else if (dataType == DataTypes.SHORT) {
+      result = elRes.getShort() <= (erRes.getShort());
+    } else if (dataType == DataTypes.INT) {
+      result = elRes.getInt() <= (erRes.getInt());
+    } else if (dataType == DataTypes.DOUBLE) {
+      result = elRes.getDouble() <= (erRes.getDouble());
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      result = elRes.getTime() <= (erRes.getTime());
+    } else if (dataType == DataTypes.LONG) {
+      result = elRes.getLong() <= (erRes.getLong());
+    } else if (dataType == DataTypes.DECIMAL) {
+      result = elRes.getDecimal().compareTo(erRes.getDecimal()) <= 0;
+    } else {
+      throw new FilterUnsupportedException("DataType: " + exprResValue1.getDataType()
+          + " not supported for the filter expression");
     }
-    exprResValue1.set(DataType.BOOLEAN, result);
+    exprResValue1.set(DataTypes.BOOLEAN, result);
     return exprResValue1;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
index b06b389..c0d7c10 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.conditional;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -43,7 +44,7 @@ public class LessThanExpression extends BinaryConditionalExpression {
     boolean result = false;
 
     if (elRes.isNull() || erRes.isNull()) {
-      elRes.set(DataType.BOOLEAN, false);
+      elRes.set(DataTypes.BOOLEAN, false);
       return elRes;
     }
     if (elRes.getDataType() != erRes.getDataType()) {
@@ -52,34 +53,26 @@ public class LessThanExpression extends BinaryConditionalExpression {
       }
 
     }
-    switch (val1.getDataType()) {
-      case STRING:
-        result = elRes.getString().compareTo(erRes.getString()) < 0;
-        break;
-      case SHORT:
-        result = elRes.getShort() < (erRes.getShort());
-        break;
-      case INT:
-        result = elRes.getInt() < (erRes.getInt());
-        break;
-      case DOUBLE:
-        result = elRes.getDouble() < (erRes.getDouble());
-        break;
-      case DATE:
-      case TIMESTAMP:
-        result = elRes.getTime() < (erRes.getTime());
-        break;
-      case LONG:
-        result = elRes.getLong() < (erRes.getLong());
-        break;
-      case DECIMAL:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) < 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    DataType dataType = val1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = elRes.getString().compareTo(erRes.getString()) < 0;
+    } else if (dataType == DataTypes.SHORT) {
+      result = elRes.getShort() < (erRes.getShort());
+    } else if (dataType == DataTypes.INT) {
+      result = elRes.getInt() < (erRes.getInt());
+    } else if (dataType == DataTypes.DOUBLE) {
+      result = elRes.getDouble() < (erRes.getDouble());
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      result = elRes.getTime() < (erRes.getTime());
+    } else if (dataType == DataTypes.LONG) {
+      result = elRes.getLong() < (erRes.getLong());
+    } else if (dataType == DataTypes.DECIMAL) {
+      result = elRes.getDecimal().compareTo(erRes.getDecimal()) < 0;
+    } else {
+      throw new FilterUnsupportedException("DataType: " + val1.getDataType() +
+          " not supported for the filter expression");
     }
-    val1.set(DataType.BOOLEAN, result);
+    val1.set(DataTypes.BOOLEAN, result);
     return val1;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
index f139802..8930c94 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.conditional;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -48,9 +49,9 @@ public class NotEqualsExpression extends BinaryConditionalExpression {
     ExpressionResult val2 = erRes;
     if (elRes.isNull() || erRes.isNull()) {
       if (isNotNull) {
-        elRes.set(DataType.BOOLEAN, elRes.isNull() != erRes.isNull());
+        elRes.set(DataTypes.BOOLEAN, elRes.isNull() != erRes.isNull());
       } else {
-        elRes.set(DataType.BOOLEAN, false);
+        elRes.set(DataTypes.BOOLEAN, false);
       }
       return elRes;
     }
@@ -61,34 +62,26 @@ public class NotEqualsExpression extends BinaryConditionalExpression {
         val2 = elRes;
       }
     }
-    switch (val1.getDataType()) {
-      case STRING:
-        result = !val1.getString().equals(val2.getString());
-        break;
-      case SHORT:
-        result = val1.getShort().shortValue() != val2.getShort().shortValue();
-        break;
-      case INT:
-        result = val1.getInt().intValue() != val2.getInt().intValue();
-        break;
-      case DOUBLE:
-        result = val1.getDouble().doubleValue() != val2.getDouble().doubleValue();
-        break;
-      case DATE:
-      case TIMESTAMP:
-        result = val1.getTime().longValue() != val2.getTime().longValue();
-        break;
-      case LONG:
-        result = elRes.getLong().longValue() != (erRes.getLong()).longValue();
-        break;
-      case DECIMAL:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) != 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    DataType dataType = val1.getDataType();
+    if (dataType == DataTypes.STRING) {
+      result = !val1.getString().equals(val2.getString());
+    } else if (dataType == DataTypes.SHORT) {
+      result = val1.getShort().shortValue() != val2.getShort().shortValue();
+    } else if (dataType == DataTypes.INT) {
+      result = val1.getInt().intValue() != val2.getInt().intValue();
+    } else if (dataType == DataTypes.DOUBLE) {
+      result = val1.getDouble().doubleValue() != val2.getDouble().doubleValue();
+    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+      result = val1.getTime().longValue() != val2.getTime().longValue();
+    } else if (dataType == DataTypes.LONG) {
+      result = elRes.getLong().longValue() != (erRes.getLong()).longValue();
+    } else if (dataType == DataTypes.DECIMAL) {
+      result = elRes.getDecimal().compareTo(erRes.getDecimal()) != 0;
+    } else {
+      throw new FilterUnsupportedException(
+          "DataType: " + val1.getDataType() + " not supported for the filter expression");
     }
-    val1.set(DataType.BOOLEAN, result);
+    val1.set(DataTypes.BOOLEAN, result);
     return val1;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
index 9f385ec..5f6359b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
@@ -21,6 +21,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -51,7 +52,7 @@ public class NotInExpression extends BinaryConditionalExpression {
 
     ExpressionResult leftRsult = left.evaluate(value);
     if (leftRsult.isNull()) {
-      leftRsult.set(DataType.BOOLEAN, false);
+      leftRsult.set(DataTypes.BOOLEAN, false);
       return leftRsult;
     }
 
@@ -62,8 +63,8 @@ public class NotInExpression extends BinaryConditionalExpression {
       for (ExpressionResult exprResVal : rightRsult.getList()) {
 
         if (exprResVal.isNull()) {
-          nullValuePresent = new ExpressionResult(DataType.BOOLEAN, false);
-          leftRsult.set(DataType.BOOLEAN, false);
+          nullValuePresent = new ExpressionResult(DataTypes.BOOLEAN, false);
+          leftRsult.set(DataTypes.BOOLEAN, false);
           return leftRsult;
         }
 
@@ -73,39 +74,31 @@ public class NotInExpression extends BinaryConditionalExpression {
         } else {
           val = exprResVal;
         }
-        switch (val.getDataType()) {
-          case STRING:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getString());
-            break;
-          case SHORT:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getShort());
-            break;
-          case INT:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getInt());
-            break;
-          case DOUBLE:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getDouble());
-            break;
-          case DATE:
-          case TIMESTAMP:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getTime());
-            break;
-          case LONG:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getLong());
-            break;
-          case DECIMAL:
-            val = new ExpressionResult(val.getDataType(), exprResVal.getDecimal());
-            break;
-          default:
-            throw new FilterUnsupportedException(
-                "DataType: " + val.getDataType() + " not supported for the filter expression");
+        DataType dataType = val.getDataType();
+        if (dataType == DataTypes.STRING) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getString());
+        } else if (dataType == DataTypes.SHORT) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getShort());
+        } else if (dataType == DataTypes.INT) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getInt());
+        } else if (dataType == DataTypes.DOUBLE) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getDouble());
+        } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getTime());
+        } else if (dataType == DataTypes.LONG) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getLong());
+        } else if (dataType == DataTypes.DECIMAL) {
+          val = new ExpressionResult(val.getDataType(), exprResVal.getDecimal());
+        } else {
+          throw new FilterUnsupportedException(
+              "DataType: " + val.getDataType() + " not supported for the filter expression");
         }
 
         setOfExprResult.add(val);
       }
     }
 
-    leftRsult.set(DataType.BOOLEAN, !setOfExprResult.contains(leftRsult));
+    leftRsult.set(DataTypes.BOOLEAN, !setOfExprResult.contains(leftRsult));
     return leftRsult;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/AndExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/AndExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/AndExpression.java
index 58d650a..4a3508b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/AndExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/AndExpression.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.scan.expression.logical;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -37,13 +37,11 @@ public class AndExpression extends BinaryLogicalExpression {
       throws FilterUnsupportedException, FilterIllegalMemberException {
     ExpressionResult resultLeft = left.evaluate(value);
     ExpressionResult resultRight = right.evaluate(value);
-    switch (resultLeft.getDataType()) {
-      case BOOLEAN:
-        resultLeft.set(DataType.BOOLEAN, (resultLeft.getBoolean() && resultRight.getBoolean()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying AND Expression Filter");
+    if (resultLeft.getDataType() == DataTypes.BOOLEAN) {
+      resultLeft.set(DataTypes.BOOLEAN, (resultLeft.getBoolean() && resultRight.getBoolean()));
+    } else {
+      throw new FilterUnsupportedException(
+          "Incompatible datatype for applying AND Expression Filter");
     }
     return resultLeft;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/FalseExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/FalseExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/FalseExpression.java
index c2aab0f..f62980b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/FalseExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/FalseExpression.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.scan.expression.logical;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
@@ -52,7 +52,7 @@ public class FalseExpression  extends BinaryConditionalExpression {
    */
   @Override public ExpressionResult evaluate(RowIntf value)
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    return new ExpressionResult(DataType.BOOLEAN,false);
+    return new ExpressionResult(DataTypes.BOOLEAN,false);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/OrExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/OrExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/OrExpression.java
index a37251a..bd10f10 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/OrExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/OrExpression.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.scan.expression.logical;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -37,15 +37,12 @@ public class OrExpression extends BinaryLogicalExpression {
       throws FilterIllegalMemberException, FilterUnsupportedException {
     ExpressionResult resultLeft = left.evaluate(value);
     ExpressionResult resultRight = right.evaluate(value);
-    switch (resultLeft.getDataType()) {
-      case BOOLEAN:
-        resultLeft.set(DataType.BOOLEAN, (resultLeft.getBoolean() || resultRight.getBoolean()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying OR Expression Filter");
+    if (resultLeft.getDataType() == DataTypes.BOOLEAN) {
+      resultLeft.set(DataTypes.BOOLEAN, (resultLeft.getBoolean() || resultRight.getBoolean()));
+    } else {
+      throw new FilterUnsupportedException(
+          "Incompatible datatype for applying OR Expression Filter");
     }
-
     return resultLeft;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/RangeExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/RangeExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/RangeExpression.java
index bce9320..01e3270 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/RangeExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/RangeExpression.java
@@ -22,7 +22,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
@@ -44,13 +44,11 @@ public class RangeExpression extends BinaryConditionalExpression {
       throws FilterUnsupportedException, FilterIllegalMemberException {
     ExpressionResult resultLeft = left.evaluate(value);
     ExpressionResult resultRight = right.evaluate(value);
-    switch (resultLeft.getDataType()) {
-      case BOOLEAN:
-        resultLeft.set(DataType.BOOLEAN, (resultLeft.getBoolean() && resultRight.getBoolean()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying RANGE Expression Filter");
+    if (resultLeft.getDataType() == DataTypes.BOOLEAN) {
+      resultLeft.set(DataTypes.BOOLEAN, (resultLeft.getBoolean() && resultRight.getBoolean()));
+    } else {
+      throw new FilterUnsupportedException(
+          "Incompatible datatype for applying RANGE Expression Filter");
     }
     return resultLeft;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/TrueExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/TrueExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/TrueExpression.java
index 7897f4a..39db1d2 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/TrueExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/logical/TrueExpression.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.scan.expression.logical;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
@@ -50,7 +50,7 @@ public class TrueExpression extends BinaryConditionalExpression {
    */
   @Override public ExpressionResult evaluate(RowIntf value)
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    return new ExpressionResult(DataType.BOOLEAN,true);
+    return new ExpressionResult(DataTypes.BOOLEAN,true);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index 1290f8b..68787e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -30,7 +30,7 @@ import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.scan.expression.BinaryExpression;
@@ -404,9 +404,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
         currentCondExpression = (BinaryConditionalExpression) expression;
         if (currentCondExpression.isSingleColumn()
             && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.ARRAY
+            != DataTypes.ARRAY
             && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.STRUCT) {
+            != DataTypes.STRUCT) {
 
           if (currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()) {
             if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
@@ -465,9 +465,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
         currentCondExpression = (BinaryConditionalExpression) expression;
         if (currentCondExpression.isSingleColumn()
             && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.ARRAY
+            != DataTypes.ARRAY
             && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.STRUCT) {
+            != DataTypes.STRUCT) {
 
           if (currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()) {
             if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
@@ -524,9 +524,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
           condExpression = (ConditionalExpression) expression;
           if (condExpression.isSingleColumn()
               && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-              != DataType.ARRAY
+              != DataTypes.ARRAY
               && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-              != DataType.STRUCT) {
+              != DataTypes.STRUCT) {
             condExpression = (ConditionalExpression) expression;
             if ((condExpression.getColumnList().get(0).getCarbonColumn()
                 .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0)


[03/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
index 35e784c..1cb008c 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -45,9 +46,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithBothStringISSame()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -77,9 +78,9 @@ public class NotEqualsExpressionUnitTest {
   @Test public void testEvaluateForNotEqualsExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
 
-    ColumnExpression left = new ColumnExpression("left_id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("left_id", DataTypes.SHORT);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("right_id", DataTypes.SHORT);
     right.setColIndex(1);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -111,9 +112,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_number", DataTypes.INT);
     right.setColIndex(1);
-    ColumnExpression left = new ColumnExpression("left_number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_number", DataTypes.INT);
     left.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -142,9 +143,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(1);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -173,9 +174,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.LONG);
     right.setColIndex(1);
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.LONG);
     left.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -204,9 +205,9 @@ public class NotEqualsExpressionUnitTest {
   @Test public void testEvaluateForNotEqualsExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("left_timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("left_timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("right_timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("right_timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
 
       notEqualsExpression = new NotEqualsExpression(left, right);
@@ -248,7 +249,7 @@ public class NotEqualsExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testForNotEqualsExpressionWithDefaultCase()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(right, right);
     RowImpl value = new RowImpl();
@@ -260,9 +261,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateWithForNotEqualsExpressionDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
     right.setColIndex(1);
-    ColumnExpression left = new ColumnExpression("contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
     left.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -290,7 +291,7 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithIsNullReturnTrue()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(right, right);
     RowImpl value = new RowImpl();
@@ -317,9 +318,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("number", DataType.INT);
+    ColumnExpression right = new ColumnExpression("number", DataTypes.INT);
     right.setColIndex(1);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();
@@ -346,8 +347,8 @@ public class NotEqualsExpressionUnitTest {
   }
 
   @Test public void testForNotEqualsExpressionWithGetString() throws Exception {
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     notEqualsExpression = new NotEqualsExpression(left, right);
     String expected_result = "NotEquals(ColumnExpression(left_name),ColumnExpression(right_name))";
     String result = notEqualsExpression.getString();
@@ -356,7 +357,7 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateForNotEqualsExpressionWithNullWhileCreatingObject()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(right, right, false);
     RowImpl value = new RowImpl();
@@ -377,13 +378,13 @@ public class NotEqualsExpressionUnitTest {
     };
 
     ExpressionResult result = notEqualsExpression.evaluate(value);
-    assertEquals(DataType.BOOLEAN, result.getDataType());
+    assertEquals(DataTypes.BOOLEAN, result.getDataType());
 
   }
 
   @Test public void testEvaluateForNotEqualsExpressionWithNullISTureWhileCreatingObject()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("id", DataTypes.SHORT);
     right.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(right, right, true);
     RowImpl value = new RowImpl();
@@ -398,7 +399,7 @@ public class NotEqualsExpressionUnitTest {
     };
 
     ExpressionResult result = notEqualsExpression.evaluate(value);
-    assertEquals(DataType.BOOLEAN, result.getDataType());
+    assertEquals(DataTypes.BOOLEAN, result.getDataType());
 
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
index cb8bb00..230e305 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
@@ -24,6 +24,7 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -44,9 +45,9 @@ public class NotInExpressionUnitTest {
 
   @Test public void testEvaluateForNotInExpressionWithString()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -57,7 +58,7 @@ public class NotInExpressionUnitTest {
     new MockUp<ExpressionResult>() {
 
       @Mock public DataType getDataType() {
-        return DataType.STRING;
+        return DataTypes.STRING;
       }
 
       @Mock public String getString() {
@@ -73,9 +74,9 @@ public class NotInExpressionUnitTest {
   @Test public void testEvaluateForNotInExpressionWithShortDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
 
-    ColumnExpression left = new ColumnExpression("left_id", DataType.SHORT);
+    ColumnExpression left = new ColumnExpression("left_id", DataTypes.SHORT);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_id", DataType.SHORT);
+    ColumnExpression right = new ColumnExpression("right_id", DataTypes.SHORT);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -91,16 +92,16 @@ public class NotInExpressionUnitTest {
     };
 
     ExpressionResult result = notInExpression.evaluate(value);
-    assertEquals(result.getDataType(), DataType.BOOLEAN);
+    assertEquals(result.getDataType(), DataTypes.BOOLEAN);
 
   }
 
   @Test public void testEvaluateForNotInExpressionWithIntDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
 
-    ColumnExpression left = new ColumnExpression("left_id", DataType.INT);
+    ColumnExpression left = new ColumnExpression("left_id", DataTypes.INT);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_id", DataType.INT);
+    ColumnExpression right = new ColumnExpression("right_id", DataTypes.INT);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -116,15 +117,15 @@ public class NotInExpressionUnitTest {
     };
 
     ExpressionResult result = notInExpression.evaluate(value);
-    assertEquals(result.getDataType(), DataType.BOOLEAN);
+    assertEquals(result.getDataType(), DataTypes.BOOLEAN);
 
   }
 
   @Test public void testEvaluateForNotInExpressionWithDoubleDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DOUBLE);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DOUBLE);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DOUBLE);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DOUBLE);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -145,9 +146,9 @@ public class NotInExpressionUnitTest {
 
   @Test public void testEvaluateForNotInExpressionWithLongDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.LONG);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.LONG);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.LONG);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.LONG);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -169,9 +170,9 @@ public class NotInExpressionUnitTest {
   @Test public void testEvaluateForNotInExpressionWithTimestampDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
     try {
-      ColumnExpression left = new ColumnExpression("left_timestamp", DataType.TIMESTAMP);
+      ColumnExpression left = new ColumnExpression("left_timestamp", DataTypes.TIMESTAMP);
       left.setColIndex(0);
-      ColumnExpression right = new ColumnExpression("right_timestamp", DataType.TIMESTAMP);
+      ColumnExpression right = new ColumnExpression("right_timestamp", DataTypes.TIMESTAMP);
       right.setColIndex(1);
       notInExpression = new NotInExpression(right, right);
 
@@ -201,9 +202,9 @@ public class NotInExpressionUnitTest {
 
   @Test public void testEvaluateForNotInExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataType.DECIMAL);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DECIMAL);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataType.DECIMAL);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DECIMAL);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -224,9 +225,9 @@ public class NotInExpressionUnitTest {
 
   @Test(expected = FilterUnsupportedException.class) public void testDefaultCaseForNotInExpression()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.BOOLEAN);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("contact", DataType.BOOLEAN);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.BOOLEAN);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();
@@ -238,9 +239,9 @@ public class NotInExpressionUnitTest {
   }
 
   @Test public void testForNotInExpressionWithGetString() throws Exception {
-    ColumnExpression left = new ColumnExpression("left_name", DataType.STRING);
+    ColumnExpression left = new ColumnExpression("left_name", DataTypes.STRING);
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("right_name", DataTypes.STRING);
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     String expected_result = "NOT IN(ColumnExpression(left_name),ColumnExpression(right_name))";
@@ -249,9 +250,9 @@ public class NotInExpressionUnitTest {
   }
   @Test public void testEvaluateForNotInExpressionWithLeftAndRightDifferentDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("name", DataType.STRING);
+    ColumnExpression right = new ColumnExpression("name", DataTypes.STRING);
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("number", DataType.INT);
+    ColumnExpression left = new ColumnExpression("number", DataTypes.INT);
     left.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/AndExpressionTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/AndExpressionTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/AndExpressionTest.java
index 22213e3..55f4fdf 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/AndExpressionTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/AndExpressionTest.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.logical;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -38,8 +39,8 @@ public class AndExpressionTest {
   private AndExpression andExpression;
 
   @Before public void setUp() {
-    ColumnExpression leftExpression = new ColumnExpression("IMEI", DataType.BOOLEAN);
-    ColumnExpression rightExpression = new ColumnExpression("IMEI", DataType.BOOLEAN);
+    ColumnExpression leftExpression = new ColumnExpression("IMEI", DataTypes.BOOLEAN);
+    ColumnExpression rightExpression = new ColumnExpression("IMEI", DataTypes.BOOLEAN);
     andExpression = new AndExpression(leftExpression, rightExpression);
   }
 
@@ -47,7 +48,7 @@ public class AndExpressionTest {
       throws FilterUnsupportedException, FilterIllegalMemberException {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new Boolean[] { true });
-    final ExpressionResult expressionResult = new ExpressionResult(DataType.STRING, "test");
+    final ExpressionResult expressionResult = new ExpressionResult(DataTypes.STRING, "test");
     new MockUp<ColumnExpression>() {
       @Mock public ExpressionResult evaluate(RowIntf value)
           throws FilterUnsupportedException, FilterIllegalMemberException {
@@ -60,7 +61,7 @@ public class AndExpressionTest {
   @Test public void testEvaluate() throws FilterUnsupportedException, FilterIllegalMemberException {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new Boolean[] { false });
-    final ExpressionResult expressionResult = new ExpressionResult(DataType.BOOLEAN, "test");
+    final ExpressionResult expressionResult = new ExpressionResult(DataTypes.BOOLEAN, "test");
     new MockUp<ColumnExpression>() {
       @Mock public ExpressionResult evaluate(RowIntf value)
           throws FilterUnsupportedException, FilterIllegalMemberException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/FalseExpressionTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/FalseExpressionTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/FalseExpressionTest.java
index 57467d9..59bf0b5 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/FalseExpressionTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/FalseExpressionTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.scan.expression.logical;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -33,7 +33,7 @@ public class FalseExpressionTest {
   private FalseExpression falseExpression;
 
   @Before public void setUp() {
-    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataType.BOOLEAN);
+    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataTypes.BOOLEAN);
     falseExpression = new FalseExpression(columnExpression);
   }
 
@@ -41,6 +41,6 @@ public class FalseExpressionTest {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new Boolean[] { true });
     ExpressionResult actualValue = falseExpression.evaluate(rowImpl);
-    assertEquals(new ExpressionResult(DataType.BOOLEAN, false), actualValue);
+    assertEquals(new ExpressionResult(DataTypes.BOOLEAN, false), actualValue);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/OrExpressionTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/OrExpressionTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/OrExpressionTest.java
index b09dee9..6ad4673 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/OrExpressionTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/logical/OrExpressionTest.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.scan.expression.logical;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -37,8 +38,8 @@ public class OrExpressionTest {
   private OrExpression orExpression;
 
   @Before public void setUp() {
-    ColumnExpression leftExpression = new ColumnExpression("IMEI", DataType.BOOLEAN);
-    ColumnExpression rightExpression = new ColumnExpression("IMEI", DataType.BOOLEAN);
+    ColumnExpression leftExpression = new ColumnExpression("IMEI", DataTypes.BOOLEAN);
+    ColumnExpression rightExpression = new ColumnExpression("IMEI", DataTypes.BOOLEAN);
     orExpression = new OrExpression(leftExpression, rightExpression);
   }
 
@@ -51,7 +52,7 @@ public class OrExpressionTest {
   @Test public void testEvaluate() throws FilterIllegalMemberException, FilterUnsupportedException {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new Boolean[] { false });
-    final ExpressionResult expressionResult = new ExpressionResult(DataType.BOOLEAN, "test");
+    final ExpressionResult expressionResult = new ExpressionResult(DataTypes.BOOLEAN, "test");
     new MockUp<ColumnExpression>() {
       @Mock public ExpressionResult evaluate(RowIntf value) {
         return expressionResult;
@@ -65,7 +66,7 @@ public class OrExpressionTest {
       throws FilterUnsupportedException, FilterIllegalMemberException {
     RowImpl rowImpl = new RowImpl();
     rowImpl.setValues(new Boolean[] { true });
-    final ExpressionResult expressionResult = new ExpressionResult(DataType.STRING, "test");
+    final ExpressionResult expressionResult = new ExpressionResult(DataTypes.STRING, "test");
     new MockUp<ColumnExpression>() {
       @Mock public ExpressionResult evaluate(RowIntf value)
           throws FilterUnsupportedException, FilterIllegalMemberException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 6667d53..a6f483b 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -67,7 +68,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     columnSchema.setColumnar(true);
     columnSchema.setColumnName("IMEI");
     columnSchema.setColumnUniqueId(UUID.randomUUID().toString());
-    columnSchema.setDataType(DataType.STRING);
+    columnSchema.setDataType(DataTypes.STRING);
     columnSchema.setDimensionColumn(true);
   }
 
@@ -80,7 +81,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
 
   @Test
   public void testCheckIfLeftExpressionRequireEvaluationWithExpressionNotInstanceOfColumnExpression() {
-    ColumnExpression expression = new ColumnExpression("test", DataType.STRING);
+    ColumnExpression expression = new ColumnExpression("test", DataTypes.STRING);
     boolean result = FilterUtil.checkIfLeftExpressionRequireEvaluation(expression);
     assertFalse(result);
   }
@@ -103,7 +104,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "1";
     String memberVal = "1";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.SHORT);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.SHORT);
     int expectedResult = 0;
     assertEquals(expectedResult, actualResult);
   }
@@ -112,7 +113,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "1000";
     String memberVal = "1001";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.INT);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.INT);
     int expectedResult = -1;
     assertEquals(expectedResult, actualResult);
   }
@@ -121,7 +122,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "1.90";
     String memberVal = "1.89";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.DOUBLE);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.DOUBLE);
     int expectedResult = 1;
     assertEquals(expectedResult, actualResult);
   }
@@ -130,7 +131,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "111111111111111";
     String memberVal = "1111111111111111";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.LONG);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.LONG);
     int expectedResult = -1;
     assertEquals(expectedResult, actualResult);
   }
@@ -139,7 +140,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "true";
     String memberVal = "false";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.BOOLEAN);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.BOOLEAN);
     int expectedResult = 1;
     assertEquals(expectedResult, actualResult);
   }
@@ -148,7 +149,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "1111111";
     String memberVal = "1111";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.DECIMAL);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.DECIMAL);
     int expectedResult = 1;
     assertEquals(expectedResult, actualResult);
   }
@@ -157,7 +158,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "11.78";
     String memberVal = "1111.90";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.FLOAT);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.FLOAT);
     int expectedResult = -1;
     assertEquals(expectedResult, actualResult);
   }
@@ -166,7 +167,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "2008-01-01 00:00:01";
     String memberVal = "2008-01-01 00:00:01";
     int actualValue =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.TIMESTAMP);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.TIMESTAMP);
     int expectedValue = 0;
     assertEquals(expectedValue, actualValue);
   }
@@ -175,7 +176,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "test";
     String memberVal = "1";
     int actualValue =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataType.INT);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.INT);
     int expectedValue = -1;
     assertEquals(expectedValue, actualValue);
   }
@@ -194,7 +195,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
 
   @Test public void testCheckIfExpressionContainsColumn() {
     String columnName = "IMEI";
-    Expression expression = new ColumnExpression(columnName, DataType.STRING);
+    Expression expression = new ColumnExpression(columnName, DataTypes.STRING);
     boolean result = FilterUtil.checkIfExpressionContainsColumn(expression);
     assertTrue(result);
   }
@@ -202,7 +203,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   @Test
   public void testCheckIfExpressionContainsColumnWithExpressionNotInstanceOfColumnExpression() {
     String columnName = "IMEI";
-    Expression expression = new LiteralExpression(columnName, DataType.STRING);
+    Expression expression = new LiteralExpression(columnName, DataTypes.STRING);
     boolean result = FilterUtil.checkIfExpressionContainsColumn(expression);
     assertFalse(result);
   }
@@ -210,7 +211,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   @Test public void testIsExpressionNeedsToResolved() {
     boolean isIncludeFilter = true;
     Object obj = "test";
-    LiteralExpression literalExpression = new LiteralExpression(obj, DataType.STRING);
+    LiteralExpression literalExpression = new LiteralExpression(obj, DataTypes.STRING);
     boolean result = FilterUtil.isExpressionNeedsToResolved(literalExpression, isIncludeFilter);
     assertFalse(result);
   }
@@ -218,7 +219,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   @Test public void testIsExpressionNeedsToResolvedWithDataTypeNullAndIsIncludeFilterFalse() {
     boolean isIncludeFilter = false;
     Object obj = "test";
-    LiteralExpression literalExpression = new LiteralExpression(obj, DataType.NULL);
+    LiteralExpression literalExpression = new LiteralExpression(obj, DataTypes.NULL);
     boolean result = FilterUtil.isExpressionNeedsToResolved(literalExpression, isIncludeFilter);
     assertTrue(result);
   }
@@ -236,7 +237,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     columnSchema.setColumnar(true);
     columnSchema.setColumnName("IMEI");
     columnSchema.setColumnUniqueId(UUID.randomUUID().toString());
-    columnSchema.setDataType(DataType.STRING);
+    columnSchema.setDataType(DataTypes.STRING);
     columnSchema.setDimensionColumn(true);
     CarbonDimension carbonDimension =
         new CarbonDimension(columnSchema, ordinal, keyOrdinal, columnGroupOrdinal,
@@ -248,8 +249,8 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   }
 
   @Test public void testGetFilterListForAllMembersRS() throws Exception {
-    Expression expression = new ColumnExpression("IMEI", DataType.STRING);
-    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataType.STRING);
+    Expression expression = new ColumnExpression("IMEI", DataTypes.STRING);
+    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataTypes.STRING);
     String defaultValues = "test";
     int defaultSurrogate = 1;
     boolean isIncludeFilter = true;
@@ -258,7 +259,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     final CarbonColumn carbonColumn = new CarbonColumn(dimColumn, ordinal, -1);
     new MockUp<ColumnExpression>() {
@@ -279,8 +280,8 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
 
   @Test public void testGetFilterListForAllMembersRSWithDefaultValuesEqualsToNull()
       throws Exception {
-    Expression expression = new ColumnExpression("IMEI", DataType.STRING);
-    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataType.STRING);
+    Expression expression = new ColumnExpression("IMEI", DataTypes.STRING);
+    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataTypes.STRING);
     String defaultValues = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
     int defaultSurrogate = 1;
     boolean isIncludeFilter = true;
@@ -289,7 +290,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataType.STRING);
+    dimColumn.setDataType(DataTypes.STRING);
     dimColumn.setDimensionColumn(true);
     final CarbonColumn carbonColumn = new CarbonColumn(dimColumn, ordinal, -1);
     new MockUp<ColumnExpression>() {
@@ -309,8 +310,8 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   }
 
   @Test public void testgetFilterListForRS() throws Exception {
-    Expression expression = new ColumnExpression("IMEI", DataType.STRING);
-    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataType.STRING);
+    Expression expression = new ColumnExpression("IMEI", DataTypes.STRING);
+    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataTypes.STRING);
     String defaultValues = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
     int defaultSurrogate = 1;
     int ordinal = 1;
@@ -331,7 +332,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   }
 
   @Test public void testCheckIfDataTypeNotTimeStamp() {
-    Expression expression = new ColumnExpression("test", DataType.STRING);
+    Expression expression = new ColumnExpression("test", DataTypes.STRING);
     boolean result = FilterUtil.checkIfDataTypeNotTimeStamp(expression);
     assertFalse(result);
   }
@@ -352,14 +353,14 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
   }
 
   @Test public void testCheckIfRightExpressionRequireEvaluation() {
-    Expression expression = new ColumnExpression("test", DataType.STRING);
+    Expression expression = new ColumnExpression("test", DataTypes.STRING);
     boolean result = FilterUtil.checkIfRightExpressionRequireEvaluation(expression);
     assertTrue(result);
   }
 
   @Test
   public void testCheckIfRightExpressionRequireEvaluationWithExpressionIsInstanceOfLiteralExpression() {
-    Expression expression = new LiteralExpression("test", DataType.STRING);
+    Expression expression = new LiteralExpression("test", DataTypes.STRING);
     boolean result = FilterUtil.checkIfRightExpressionRequireEvaluation(expression);
     assertFalse(result);
   }
@@ -368,13 +369,13 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     boolean isIncludeFilter = true;
     AbsoluteTableIdentifier absoluteTableIdentifier =
         new AbsoluteTableIdentifier(this.carbonStorePath, carbonTableIdentifier);
-    ColumnExpression expression = new ColumnExpression("test", DataType.STRING);
+    ColumnExpression expression = new ColumnExpression("test", DataTypes.STRING);
     List<String> evaluateResultListFinal = new ArrayList<>();
     evaluateResultListFinal.add("test1");
     evaluateResultListFinal.add("test2");
     assertTrue(FilterUtil
         .getNoDictionaryValKeyMemberForFilter(evaluateResultListFinal, isIncludeFilter,
-            DataType.STRING) instanceof ColumnFilterInfo);
+            DataTypes.STRING) instanceof ColumnFilterInfo);
   }
 
   @Test public void testPrepareDefaultStartIndexKey() throws KeyGenException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
index d4bd06b..ca78e66 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
@@ -189,7 +189,7 @@ public class CarbonMetadataUtilTest {
 
     final EncodedColumnPage measure = new EncodedColumnPage(new DataChunk2(), new byte[]{0,1},
         PrimitivePageStatsCollector.newInstance(
-        org.apache.carbondata.core.metadata.datatype.DataType.BYTE, 0, 0));
+        org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE, 0, 0));
     new MockUp<EncodedTablePage>() {
       @SuppressWarnings("unused") @Mock
       public EncodedColumnPage getMeasure(int measureIndex) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index 4a8483f..658dda9 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -39,6 +39,7 @@ import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -507,14 +508,14 @@ public class CarbonUtilTest {
   }
 
   @Test public void testForHasDataTypes() {
-    DataType[] dataTypes = { DataType.DECIMAL, DataType.BOOLEAN, DataType.INT };
-    assertTrue(CarbonUtil.hasDataType(DataType.BOOLEAN, dataTypes));
-    assertTrue(!CarbonUtil.hasDataType(DataType.DATE, dataTypes));
+    DataType[] dataTypes = { DataTypes.DECIMAL, DataTypes.BOOLEAN, DataTypes.INT };
+    assertTrue(CarbonUtil.hasDataType(DataTypes.BOOLEAN, dataTypes));
+    assertTrue(!CarbonUtil.hasDataType(DataTypes.DATE, dataTypes));
   }
 
   @Test public void testForHasComplexDataTypes() {
-    assertTrue(CarbonUtil.hasComplexDataType(DataType.ARRAY));
-    assertTrue(!CarbonUtil.hasComplexDataType(DataType.DATE));
+    assertTrue(DataTypes.ARRAY.isComplexType());
+    assertTrue(!DataTypes.DATE.isComplexType());
   }
 
   @Test public void testToGetDictionaryEncodingArray() {
@@ -575,11 +576,11 @@ public class CarbonUtilTest {
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
-    column1Schema.setDataType(DataType.DATE);
+    column1Schema.setDataType(DataTypes.DATE);
     column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
-    column2Schema.setDataType(DataType.ARRAY);
+    column2Schema.setDataType(DataTypes.ARRAY);
     column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
     QueryDimension[] queryDimensions = { column1, column2 };

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
index 8161fae..30b3247 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
@@ -17,32 +17,39 @@
 
 package org.apache.carbondata.core.util;
 
-import mockit.Mock;
-import mockit.MockUp;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
 import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.impl.FileHolderImpl;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.reader.CarbonFooterReader;
 import org.apache.carbondata.core.reader.CarbonIndexFileReader;
 import org.apache.carbondata.core.reader.ThriftReader;
-import org.apache.carbondata.format.*;
+import org.apache.carbondata.format.BlockIndex;
+import org.apache.carbondata.format.BlockletBTreeIndex;
+import org.apache.carbondata.format.BlockletMinMaxIndex;
 import org.apache.carbondata.format.ColumnSchema;
+import org.apache.carbondata.format.DataType;
+import org.apache.carbondata.format.Encoding;
+import org.apache.carbondata.format.FileFooter;
+import org.apache.carbondata.format.IndexHeader;
 
+import mockit.Mock;
+import mockit.MockUp;
 import org.junit.Test;
 
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import java.io.*;
-import java.util.*;
-
-import static junit.framework.TestCase.*;
+import static junit.framework.TestCase.assertEquals;
 
 public class DataFileFooterConverterTest {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
index a02bc10..b6ea765 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
@@ -17,23 +17,29 @@
 
 package org.apache.carbondata.core.util;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 
 import org.junit.Test;
 
-import java.math.BigDecimal;
-import java.math.BigInteger;
-
-import static org.apache.carbondata.core.util.DataTypeUtil.*;
-import static junit.framework.TestCase.*;
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+import static org.apache.carbondata.core.util.DataTypeUtil.bigDecimalToByte;
+import static org.apache.carbondata.core.util.DataTypeUtil.byteToBigDecimal;
+import static org.apache.carbondata.core.util.DataTypeUtil.getColumnDataTypeDisplayName;
 import static org.apache.carbondata.core.util.DataTypeUtil.getDataBasedOnDataType;
+import static org.apache.carbondata.core.util.DataTypeUtil.getDataType;
+import static org.apache.carbondata.core.util.DataTypeUtil.getMeasureValueBasedOnDataType;
+import static org.apache.carbondata.core.util.DataTypeUtil.normalizeIntAndLongValues;
 
 public class DataTypeUtilTest {
 
   @Test public void testGetColumnDataTypeDisplayName() {
-    String expected = DataType.INT.getName();
+    String expected = DataTypes.INT.getName();
     String result = getColumnDataTypeDisplayName("INT");
     assertEquals(expected, result);
 
@@ -55,57 +61,57 @@ public class DataTypeUtilTest {
   }
 
   @Test public void testGetDataType() {
-    assertEquals(DataType.TIMESTAMP, getDataType("TIMESTAMP"));
-    assertEquals(DataType.DATE, getDataType("DATE"));
-    assertEquals(DataType.STRING, getDataType("STRING"));
-    assertEquals(DataType.INT, getDataType("INT"));
-    assertEquals(DataType.SHORT, getDataType("SMALLINT"));
-    assertEquals(DataType.LONG, getDataType("LONG"));
-    assertEquals(DataType.DOUBLE, getDataType("DOUBLE"));
-    assertEquals(DataType.DECIMAL, getDataType("DECIMAL"));
-    assertEquals(DataType.ARRAY, getDataType("ARRAY"));
-    assertEquals(DataType.STRUCT, getDataType("STRUCT"));
-    assertEquals(DataType.STRING, getDataType("MAP"));
-    assertEquals(DataType.STRING, getDataType("default"));
+    assertEquals(DataTypes.TIMESTAMP, getDataType("TIMESTAMP"));
+    assertEquals(DataTypes.DATE, getDataType("DATE"));
+    assertEquals(DataTypes.STRING, getDataType("STRING"));
+    assertEquals(DataTypes.INT, getDataType("INT"));
+    assertEquals(DataTypes.SHORT, getDataType("SMALLINT"));
+    assertEquals(DataTypes.LONG, getDataType("LONG"));
+    assertEquals(DataTypes.DOUBLE, getDataType("DOUBLE"));
+    assertEquals(DataTypes.DECIMAL, getDataType("DECIMAL"));
+    assertEquals(DataTypes.ARRAY, getDataType("ARRAY"));
+    assertEquals(DataTypes.STRUCT, getDataType("STRUCT"));
+    assertEquals(DataTypes.STRING, getDataType("MAP"));
+    assertEquals(DataTypes.STRING, getDataType("default"));
 
   }
 
   @Test public void testGetDataBasedOnDataType() throws NumberFormatException {
     String data = " ";
     if (data.isEmpty()) {
-      assertEquals(getDataBasedOnDataType(data, DataType.INT), null);
+      assertEquals(getDataBasedOnDataType(data, DataTypes.INT), null);
     }
-    assertEquals(getDataBasedOnDataType("1", DataType.INT), 1);
-    assertEquals(getDataBasedOnDataType(" ", DataType.INT), null);
-    assertEquals(getDataBasedOnDataType("0", DataType.DOUBLE), 0.0d);
-    assertEquals(getDataBasedOnDataType("0", DataType.LONG), 0L);
+    assertEquals(getDataBasedOnDataType("1", DataTypes.INT), 1);
+    assertEquals(getDataBasedOnDataType(" ", DataTypes.INT), null);
+    assertEquals(getDataBasedOnDataType("0", DataTypes.DOUBLE), 0.0d);
+    assertEquals(getDataBasedOnDataType("0", DataTypes.LONG), 0L);
     java.math.BigDecimal javaDecVal = new java.math.BigDecimal(1);
     scala.math.BigDecimal scalaDecVal = new scala.math.BigDecimal(javaDecVal);
-    assertEquals(getDataBasedOnDataType("1", DataType.DECIMAL),
+    assertEquals(getDataBasedOnDataType("1", DataTypes.DECIMAL),
         DataTypeUtil.getDataTypeConverter().convertToDecimal(scalaDecVal));
-    assertEquals(getDataBasedOnDataType("default", DataType.NULL),
+    assertEquals(getDataBasedOnDataType("default", DataTypes.NULL),
         DataTypeUtil.getDataTypeConverter().convertFromStringToUTF8String("default"));
-    assertEquals(getDataBasedOnDataType((String) null, DataType.NULL), null);
+    assertEquals(getDataBasedOnDataType((String) null, DataTypes.NULL), null);
   }
 
   @Test public void testGetMeasureValueBasedOnDataType() {
     ColumnSchema columnSchema = new ColumnSchema();
     CarbonMeasure carbonMeasure = new CarbonMeasure(columnSchema, 1);
-    Object resultInt = getMeasureValueBasedOnDataType("1", DataType.INT, carbonMeasure);
+    Object resultInt = getMeasureValueBasedOnDataType("1", DataTypes.INT, carbonMeasure);
     Object expectedInt = Double.valueOf(1).intValue();
     assertEquals(expectedInt, resultInt);
-    Object resultLong = getMeasureValueBasedOnDataType("1", DataType.LONG, carbonMeasure);
+    Object resultLong = getMeasureValueBasedOnDataType("1", DataTypes.LONG, carbonMeasure);
     Object expectedLong = Long.valueOf(1);
     assertEquals(expectedLong, resultLong);
-    Object resultDefault = getMeasureValueBasedOnDataType("1", DataType.DOUBLE, carbonMeasure);
+    Object resultDefault = getMeasureValueBasedOnDataType("1", DataTypes.DOUBLE, carbonMeasure);
     Double expectedDefault = Double.valueOf(1);
     assertEquals(expectedDefault, resultDefault);
 
   }
 
   @Test public void testNormalizeIntAndLongValues() throws NumberFormatException {
-    assertEquals(null, normalizeIntAndLongValues("INT", DataType.INT));
-    assertEquals("1", normalizeIntAndLongValues("1", DataType.STRING));
+    assertEquals(null, normalizeIntAndLongValues("INT", DataTypes.INT));
+    assertEquals("1", normalizeIntAndLongValues("1", DataTypes.STRING));
 
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
index 952b7f7..3cee633 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.core.util;
 import java.util.Arrays;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -74,7 +75,7 @@ public class RangeFilterProcessorTest {
 
     Expression inputFilter;
     boolean result = false;
-    ColumnExpression cola = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola = new ColumnExpression("a", DataTypes.STRING);
     cola.setDimension(true);
 
     ColumnSchema empColumnSchema = new ColumnSchema();
@@ -82,25 +83,25 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setColumnUniqueId("empNameCol");
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empColumnSchema.setDataType(DataType.STRING);
+    empColumnSchema.setDataType(DataTypes.STRING);
     CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
     cola.setDimension(empDimension);
 
     Expression greaterThan =
-        new GreaterThanEqualToExpression(cola, new LiteralExpression("11", DataType.STRING));
+        new GreaterThanEqualToExpression(cola, new LiteralExpression("11", DataTypes.STRING));
 
-    ColumnExpression colb = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb = new ColumnExpression("a", DataTypes.STRING);
     colb.setDimension(true);
     colb.setDimension(empDimension);
     Expression lessThan =
-        new LessThanEqualToExpression(colb, new LiteralExpression("20", DataType.STRING));
+        new LessThanEqualToExpression(colb, new LiteralExpression("20", DataTypes.STRING));
     inputFilter = new AndExpression(greaterThan, lessThan);
 
     Expression output = new AndExpression(new RangeExpression(
-        new GreaterThanEqualToExpression(new ColumnExpression("a", DataType.STRING),
-            new LiteralExpression("11", DataType.STRING)),
-        new LessThanEqualToExpression(new ColumnExpression("a", DataType.STRING),
-            new LiteralExpression("20", DataType.STRING))), new TrueExpression(null));
+        new GreaterThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
+            new LiteralExpression("11", DataTypes.STRING)),
+        new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
+            new LiteralExpression("20", DataTypes.STRING))), new TrueExpression(null));
     FilterOptimizer rangeFilterOptimizer =
         new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
     rangeFilterOptimizer.optimizeFilter();
@@ -115,7 +116,7 @@ public class RangeFilterProcessorTest {
 
     Expression inputFilter;
     boolean result = false;
-    ColumnExpression cola = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola = new ColumnExpression("a", DataTypes.STRING);
     cola.setDimension(true);
 
     ColumnSchema empColumnSchema = new ColumnSchema();
@@ -123,25 +124,25 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setColumnUniqueId("empNameCol");
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empColumnSchema.setDataType(DataType.STRING);
+    empColumnSchema.setDataType(DataTypes.STRING);
     CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
     cola.setDimension(empDimension);
 
     Expression greaterThan =
-        new GreaterThanEqualToExpression(cola, new LiteralExpression("20", DataType.STRING));
+        new GreaterThanEqualToExpression(cola, new LiteralExpression("20", DataTypes.STRING));
 
-    ColumnExpression colb = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb = new ColumnExpression("a", DataTypes.STRING);
     colb.setDimension(true);
     colb.setDimension(empDimension);
     Expression lessThan =
-        new LessThanEqualToExpression(colb, new LiteralExpression("05", DataType.STRING));
+        new LessThanEqualToExpression(colb, new LiteralExpression("05", DataTypes.STRING));
     inputFilter = new AndExpression(greaterThan, lessThan);
 
     Expression output = new AndExpression(
-        new GreaterThanEqualToExpression(new ColumnExpression("a", DataType.STRING),
-            new LiteralExpression("20", DataType.STRING)),
-        new LessThanEqualToExpression(new ColumnExpression("a", DataType.STRING),
-            new LiteralExpression("05", DataType.STRING)));
+        new GreaterThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
+            new LiteralExpression("20", DataTypes.STRING)),
+        new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
+            new LiteralExpression("05", DataTypes.STRING)));
     FilterOptimizer rangeFilterOptimizer =
         new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
     rangeFilterOptimizer.optimizeFilter();
@@ -162,33 +163,33 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setColumnUniqueId("a");
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empColumnSchema.setDataType(DataType.STRING);
+    empColumnSchema.setDataType(DataTypes.STRING);
     CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
 
-    ColumnExpression cola1 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola1 = new ColumnExpression("a", DataTypes.STRING);
     cola1.setDimension(true);
     cola1.setDimension(empDimension);
 
-    ColumnExpression cola2 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola2 = new ColumnExpression("a", DataTypes.STRING);
     cola2.setDimension(true);
     cola2.setDimension(empDimension);
 
-    ColumnExpression cola3 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola3 = new ColumnExpression("a", DataTypes.STRING);
     cola3.setDimension(true);
     cola3.setDimension(empDimension);
 
-    ColumnExpression cola4 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola4 = new ColumnExpression("a", DataTypes.STRING);
     cola4.setDimension(true);
     cola4.setDimension(empDimension);
 
     Expression lessThan1 =
-        new LessThanEqualToExpression(cola1, new LiteralExpression("15", DataType.STRING));
+        new LessThanEqualToExpression(cola1, new LiteralExpression("15", DataTypes.STRING));
     Expression lessThan2 =
-        new LessThanEqualToExpression(cola2, new LiteralExpression("20", DataType.STRING));
+        new LessThanEqualToExpression(cola2, new LiteralExpression("20", DataTypes.STRING));
     Expression greaterThan1 =
-        new GreaterThanExpression(cola3, new LiteralExpression("12", DataType.STRING));
+        new GreaterThanExpression(cola3, new LiteralExpression("12", DataTypes.STRING));
     Expression greaterThan2 =
-        new GreaterThanEqualToExpression(cola4, new LiteralExpression("11", DataType.STRING));
+        new GreaterThanEqualToExpression(cola4, new LiteralExpression("11", DataTypes.STRING));
 
     Expression And1 = new AndExpression(new NotEqualsExpression(null, null), greaterThan2);
     Expression And2 = new AndExpression(And1, greaterThan1);
@@ -197,19 +198,19 @@ public class RangeFilterProcessorTest {
 
     // Build The output
 
-    ColumnExpression colb1 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb1 = new ColumnExpression("a", DataTypes.STRING);
     cola1.setDimension(true);
     cola1.setDimension(empDimension);
 
-    ColumnExpression colb2 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb2 = new ColumnExpression("a", DataTypes.STRING);
     cola2.setDimension(true);
     cola2.setDimension(empDimension);
 
     Expression greaterThanb1 =
-        new GreaterThanExpression(cola3, new LiteralExpression("12", DataType.STRING));
+        new GreaterThanExpression(cola3, new LiteralExpression("12", DataTypes.STRING));
 
     Expression lessThanb1 =
-        new LessThanEqualToExpression(cola1, new LiteralExpression("15", DataType.STRING));
+        new LessThanEqualToExpression(cola1, new LiteralExpression("15", DataTypes.STRING));
 
     Expression Andb1 =
         new AndExpression(new NotEqualsExpression(null, null), new TrueExpression(null));
@@ -237,33 +238,33 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setColumnUniqueId("a");
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empColumnSchema.setDataType(DataType.STRING);
+    empColumnSchema.setDataType(DataTypes.STRING);
     CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
 
-    ColumnExpression cola1 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola1 = new ColumnExpression("a", DataTypes.STRING);
     cola1.setDimension(true);
     cola1.setDimension(empDimension);
 
-    ColumnExpression cola2 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola2 = new ColumnExpression("a", DataTypes.STRING);
     cola2.setDimension(true);
     cola2.setDimension(empDimension);
 
-    ColumnExpression cola3 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola3 = new ColumnExpression("a", DataTypes.STRING);
     cola3.setDimension(true);
     cola3.setDimension(empDimension);
 
-    ColumnExpression cola4 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression cola4 = new ColumnExpression("a", DataTypes.STRING);
     cola4.setDimension(true);
     cola4.setDimension(empDimension);
 
     Expression lessThan1 =
-        new LessThanEqualToExpression(cola1, new LiteralExpression("15", DataType.STRING));
+        new LessThanEqualToExpression(cola1, new LiteralExpression("15", DataTypes.STRING));
     Expression lessThan2 =
-        new LessThanEqualToExpression(cola2, new LiteralExpression("20", DataType.STRING));
+        new LessThanEqualToExpression(cola2, new LiteralExpression("20", DataTypes.STRING));
     Expression greaterThan1 =
-        new GreaterThanExpression(cola3, new LiteralExpression("12", DataType.STRING));
+        new GreaterThanExpression(cola3, new LiteralExpression("12", DataTypes.STRING));
     Expression greaterThan2 =
-        new GreaterThanEqualToExpression(cola4, new LiteralExpression("11", DataType.STRING));
+        new GreaterThanEqualToExpression(cola4, new LiteralExpression("11", DataTypes.STRING));
 
     Expression Or1 = new OrExpression(new NotEqualsExpression(null, null), greaterThan2);
     Expression Or2 = new OrExpression(Or1, greaterThan1);
@@ -272,30 +273,30 @@ public class RangeFilterProcessorTest {
 
     // Build The output
 
-    ColumnExpression colb1 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb1 = new ColumnExpression("a", DataTypes.STRING);
     cola1.setDimension(true);
     cola1.setDimension(empDimension);
 
-    ColumnExpression colb2 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb2 = new ColumnExpression("a", DataTypes.STRING);
     cola2.setDimension(true);
     cola2.setDimension(empDimension);
 
-    ColumnExpression colb3 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb3 = new ColumnExpression("a", DataTypes.STRING);
     cola3.setDimension(true);
     cola3.setDimension(empDimension);
 
-    ColumnExpression colb4 = new ColumnExpression("a", DataType.STRING);
+    ColumnExpression colb4 = new ColumnExpression("a", DataTypes.STRING);
     cola4.setDimension(true);
     cola4.setDimension(empDimension);
 
     Expression lessThanb1 =
-        new LessThanEqualToExpression(colb1, new LiteralExpression("15", DataType.STRING));
+        new LessThanEqualToExpression(colb1, new LiteralExpression("15", DataTypes.STRING));
     Expression lessThanb2 =
-        new LessThanEqualToExpression(colb2, new LiteralExpression("20", DataType.STRING));
+        new LessThanEqualToExpression(colb2, new LiteralExpression("20", DataTypes.STRING));
     Expression greaterThanb1 =
-        new GreaterThanExpression(colb3, new LiteralExpression("12", DataType.STRING));
+        new GreaterThanExpression(colb3, new LiteralExpression("12", DataTypes.STRING));
     Expression greaterThanb2 =
-        new GreaterThanEqualToExpression(colb4, new LiteralExpression("11", DataType.STRING));
+        new GreaterThanEqualToExpression(colb4, new LiteralExpression("11", DataTypes.STRING));
 
     Expression Orb1 = new OrExpression(new NotEqualsExpression(null, null), greaterThanb2);
     Expression Orb2 = new OrExpression(Orb1, greaterThanb1);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
index 6958c97..cff1291 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryChunksWrapper;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -67,7 +68,7 @@ public class CarbonDictionarySortInfoPreparatorTest {
     };
 
     CarbonDictionarySortInfo carbonDictionarySortInfo = carbonDictionarySortInfoPreparator
-        .getDictionarySortInfo(newDistinctValues, dictionary, DataType.ARRAY);
+        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.ARRAY);
     int expectedGetSortIndexValue = 1;
     int expectedGetSortInvertedIndexLength = 3;
     int actualGetSortIndexValue = carbonDictionarySortInfo.getSortIndex().get(0);
@@ -86,7 +87,7 @@ public class CarbonDictionarySortInfoPreparatorTest {
     newDistinctValues.add("xyz");
     Dictionary dictionary = null;
     CarbonDictionarySortInfo carbonDictionarySortInfo = carbonDictionarySortInfoPreparator
-        .getDictionarySortInfo(newDistinctValues, dictionary, DataType.ARRAY);
+        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.ARRAY);
     int expectedGetSortIndexValue = 1;
     int expectedGetSortInvertedIndexLength = 2;
     int actualGetSortIndexValue = carbonDictionarySortInfo.getSortIndex().get(0);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
index 83e48c3..a42b214 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
@@ -17,6 +17,7 @@
 package org.apache.carbondata.core.writer.sortindex;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.junit.Test;
 
@@ -32,63 +33,63 @@ public class CarbonDictionarySortModelTest {
   private CarbonDictionarySortModel carbonDictionarySortModel = null;
 
   @Test public void testCompareToForDataTypeDoubleCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DOUBLE, "7234");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "7234");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DOUBLE, "5678");
+        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "5678");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDoubleExceptionCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DOUBLE, "double");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "double");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DOUBLE, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDoubleExceptionCaseForOtherObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DOUBLE, "1234");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "1234");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DOUBLE, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeBooleanCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.BOOLEAN, "memberValue");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "memberValue");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DOUBLE, "value");
+        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "value");
     int expectedResult = -9;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDecimalCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "72.34");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "72.34");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DECIMAL, "56.78");
+        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "56.78");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDecimalExceptionCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "decimal");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "decimal");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DECIMAL, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDecimalExceptionCaseForOtherObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "15.24");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "15.24");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DECIMAL, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
@@ -96,9 +97,9 @@ public class CarbonDictionarySortModelTest {
 
   @Test public void testCompareToForDataTypeTimestampCase() {
     carbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataType.TIMESTAMP, "2014-09-22 12:00:00");
+        new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09-22 12:00:00");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.TIMESTAMP, "2015-09-22 12:08:49");
+        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "2015-09-22 12:08:49");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
@@ -106,9 +107,9 @@ public class CarbonDictionarySortModelTest {
 
   @Test public void testCompareToForDataTypeTimestampExceptionCase() {
     carbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataType.TIMESTAMP, "2014-09 12:00:00");
+        new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09 12:00:00");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.TIMESTAMP, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
@@ -116,88 +117,88 @@ public class CarbonDictionarySortModelTest {
 
   @Test public void testCompareToForDataTypeTimestampExceptionCaseForOtherObject() {
     carbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataType.TIMESTAMP, "2014-09-22 12:00:00");
+        new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09-22 12:00:00");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.TIMESTAMP, "2014-09-22 12");
+        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "2014-09-22 12");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testHashCode() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "15.24");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "15.24");
     int actualResult = carbonDictionarySortModel.hashCode();
     int expectedResult = 46877260;
     assertTrue(actualResult == expectedResult);
   }
 
   @Test public void testHashCodeNullCaseForMemberValue() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, null);
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, null);
     int actualResult = carbonDictionarySortModel.hashCode();
     int expectedResult = 0;
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testEquals() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "15.24");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "15.24");
     CarbonDictionarySortModel testCarbonDictionarySortModel = carbonDictionarySortModel;
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertTrue(result);
   }
 
   @Test public void testEqualsMemberValueNullCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, null);
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, null);
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataType.BOOLEAN, "false");
+        new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "false");
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertFalse(result);
   }
 
   @Test public void testEqualsWhenMemberValueDiffers() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "12.45");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataType.BOOLEAN, "false");
+        new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "false");
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertFalse(result);
   }
 
   @Test public void testEqualsWhenMemberValueIsSame() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "12.45");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataType.DECIMAL, "12.45");
+        new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertTrue(result);
   }
 
   @Test public void testEqualsForDifferentObjects() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "12.45");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
     Object testCarbonDictionarySortModel = new Object();
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertFalse(result);
   }
 
   @Test public void testCompareToForDataTypeDoubleExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DOUBLE, "double");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "double");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DOUBLE, "1234");
+        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "1234");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDecimalExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.DECIMAL, "12.il");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.il");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.DECIMAL, "12.89");
+        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "12.89");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeTimestampExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataType.TIMESTAMP, "2014-09");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataType.TIMESTAMP, "2014-09-22 12:00:00");
+        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "2014-09-22 12:00:00");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputFormat_FT.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputFormat_FT.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputFormat_FT.java
index 654aff7..4a5a9cf 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputFormat_FT.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputFormat_FT.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.hadoop.ft;
 import java.util.List;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.hadoop.CarbonInputFormat;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -62,8 +63,8 @@ public class CarbonInputFormat_FT extends TestCase {
     Job job = Job.getInstance(jobConf);
     FileInputFormat.addInputPath(job, new Path("/opt/carbonstore/db/table1"));
     job.getConfiguration().set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, "1,2");
-    Expression expression = new EqualToExpression(new ColumnExpression("c1", DataType.STRING),
-        new LiteralExpression("a", DataType.STRING));
+    Expression expression = new EqualToExpression(new ColumnExpression("c1", DataTypes.STRING),
+        new LiteralExpression("a", DataTypes.STRING));
     CarbonInputFormat.setFilterPredicates(job.getConfiguration(), expression);
     List splits = carbonInputFormat.getSplits(job);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
index 86a3326..865dabe 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.hadoop.CarbonInputFormat;
@@ -105,8 +106,8 @@ public class CarbonInputMapperTest extends TestCase {
       carbonProjection.addColumn("country");
       carbonProjection.addColumn("salary");
       Expression expression =
-          new EqualToExpression(new ColumnExpression("country", DataType.STRING),
-              new LiteralExpression("france", DataType.STRING));
+          new EqualToExpression(new ColumnExpression("country", DataTypes.STRING),
+              new LiteralExpression("france", DataTypes.STRING));
       runJob(outPath, carbonProjection, expression);
       Assert.assertEquals("Count lines are not matching", 101, countTheLines(outPath));
       Assert.assertEquals("Column count are not matching", 3, countTheColumns(outPath));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
index 960c836..ac90aa5 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.hadoop.test.util;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -36,15 +37,15 @@ public class ObjectSerializationUtilTest extends TestCase {
   }
 
   @Test public void testConvertObjectToString() throws Exception {
-    Expression expression = new EqualToExpression(new ColumnExpression("c1", DataType.STRING),
-        new LiteralExpression("a", DataType.STRING));
+    Expression expression = new EqualToExpression(new ColumnExpression("c1", DataTypes.STRING),
+        new LiteralExpression("a", DataTypes.STRING));
     String string = ObjectSerializationUtil.convertObjectToString(expression);
     Assert.assertTrue(string != null);
   }
 
   @Test public void testConvertStringToObject() throws Exception {
-    Expression expression = new EqualToExpression(new ColumnExpression("c1", DataType.STRING),
-        new LiteralExpression("a", DataType.STRING));
+    Expression expression = new EqualToExpression(new ColumnExpression("c1", DataTypes.STRING),
+        new LiteralExpression("a", DataTypes.STRING));
     String string = ObjectSerializationUtil.convertObjectToString(expression);
     Assert.assertTrue(string != null);
     Object object = ObjectSerializationUtil.convertStringToObject(string);


[09/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
index ce16ad5..d78d144 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
@@ -34,6 +34,7 @@ import org.apache.carbondata.core.datastore.page.encoding.dimension.legacy.Direc
 import org.apache.carbondata.core.datastore.page.encoding.dimension.legacy.HighCardDictDimensionIndexCodec;
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.DecimalConverterFactory;
 
 /**
@@ -111,21 +112,21 @@ public class DefaultEncodingFactory extends EncodingFactory {
 
   private ColumnPageEncoder createEncoderForMeasure(ColumnPage columnPage) {
     SimpleStatsResult stats = columnPage.getStatistics();
-    switch (stats.getDataType()) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-        return selectCodecByAlgorithmForIntegral(stats).createEncoder(null);
-      case DECIMAL:
-        return createEncoderForDecimalDataTypeMeasure(columnPage);
-      case FLOAT:
-      case DOUBLE:
-        return selectCodecByAlgorithmForFloating(stats).createEncoder(null);
-      case BYTE_ARRAY:
-        return new DirectCompressCodec(columnPage.getDataType()).createEncoder(null);
-      default:
-        throw new RuntimeException("unsupported data type: " + stats.getDataType());
+    DataType dataType = stats.getDataType();
+    if (dataType == DataTypes.BYTE ||
+        dataType == DataTypes.SHORT ||
+        dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG) {
+      return selectCodecByAlgorithmForIntegral(stats).createEncoder(null);
+    } else if (dataType == DataTypes.DECIMAL) {
+      return createEncoderForDecimalDataTypeMeasure(columnPage);
+    } else if (dataType == DataTypes.FLOAT ||
+        dataType == DataTypes.DOUBLE) {
+      return selectCodecByAlgorithmForFloating(stats).createEncoder(null);
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
+      return new DirectCompressCodec(columnPage.getDataType()).createEncoder(null);
+    } else {
+      throw new RuntimeException("unsupported data type: " + stats.getDataType());
     }
   }
 
@@ -144,32 +145,31 @@ public class DefaultEncodingFactory extends EncodingFactory {
 
   private static DataType fitLongMinMax(long max, long min) {
     if (max <= Byte.MAX_VALUE && min >= Byte.MIN_VALUE) {
-      return DataType.BYTE;
+      return DataTypes.BYTE;
     } else if (max <= Short.MAX_VALUE && min >= Short.MIN_VALUE) {
-      return DataType.SHORT;
+      return DataTypes.SHORT;
     } else if (max <= THREE_BYTES_MAX && min >= THREE_BYTES_MIN) {
-      return DataType.SHORT_INT;
+      return DataTypes.SHORT_INT;
     } else if (max <= Integer.MAX_VALUE && min >= Integer.MIN_VALUE) {
-      return DataType.INT;
+      return DataTypes.INT;
     } else {
-      return DataType.LONG;
+      return DataTypes.LONG;
     }
   }
 
   private static DataType fitMinMax(DataType dataType, Object max, Object min) {
-    switch (dataType) {
-      case BYTE:
-        return fitLongMinMax((byte) max, (byte) min);
-      case SHORT:
-        return fitLongMinMax((short) max, (short) min);
-      case INT:
-        return fitLongMinMax((int) max, (int) min);
-      case LONG:
-        return fitLongMinMax((long) max, (long) min);
-      case DOUBLE:
-        return fitLongMinMax((long) (double) max, (long) (double) min);
-      default:
-        throw new RuntimeException("internal error: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      return fitLongMinMax((byte) max, (byte) min);
+    } else if (dataType == DataTypes.SHORT) {
+      return fitLongMinMax((short) max, (short) min);
+    } else if (dataType == DataTypes.INT) {
+      return fitLongMinMax((int) max, (int) min);
+    } else if (dataType == DataTypes.LONG) {
+      return fitLongMinMax((long) max, (long) min);
+    } else if (dataType == DataTypes.DOUBLE) {
+      return fitLongMinMax((long) (double) max, (long) (double) min);
+    } else {
+      throw new RuntimeException("internal error: " + dataType);
     }
   }
 
@@ -196,7 +196,7 @@ public class DefaultEncodingFactory extends EncodingFactory {
         long value = maxValue - minValue;
         return compareMinMaxAndSelectDataType(value);
       case DECIMAL_LONG:
-        return DataType.LONG;
+        return DataTypes.LONG;
       default:
         throw new RuntimeException("internal error: " + dataType);
     }
@@ -206,38 +206,34 @@ public class DefaultEncodingFactory extends EncodingFactory {
   private static DataType fitDelta(DataType dataType, Object max, Object min) {
     // use long data type to calculate delta to avoid overflow
     long value;
-    switch (dataType) {
-      case BYTE:
-        value = (long)(byte) max - (long)(byte) min;
-        break;
-      case SHORT:
-        value = (long)(short) max - (long)(short) min;
-        break;
-      case INT:
-        value = (long)(int) max - (long)(int) min;
-        break;
-      case LONG:
-        // TODO: add overflow detection and return delta type
-        return DataType.LONG;
-      case DOUBLE:
-        return DataType.LONG;
-      default:
-        throw new RuntimeException("internal error: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      value = (long) (byte) max - (long) (byte) min;
+    } else if (dataType == DataTypes.SHORT) {
+      value = (long) (short) max - (long) (short) min;
+    } else if (dataType == DataTypes.INT) {
+      value = (long) (int) max - (long) (int) min;
+    } else if (dataType == DataTypes.LONG) {
+      // TODO: add overflow detection and return delta type
+      return DataTypes.LONG;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DataTypes.LONG;
+    } else {
+      throw new RuntimeException("internal error: " + dataType);
     }
     return compareMinMaxAndSelectDataType(value);
   }
 
   private static DataType compareMinMaxAndSelectDataType(long value) {
     if (value <= Byte.MAX_VALUE && value >= Byte.MIN_VALUE) {
-      return DataType.BYTE;
+      return DataTypes.BYTE;
     } else if (value <= Short.MAX_VALUE && value >= Short.MIN_VALUE) {
-      return DataType.SHORT;
+      return DataTypes.SHORT;
     } else if (value <= THREE_BYTES_MAX && value >= THREE_BYTES_MIN) {
-      return DataType.SHORT_INT;
+      return DataTypes.SHORT_INT;
     } else if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
-      return DataType.INT;
+      return DataTypes.INT;
     } else {
-      return DataType.LONG;
+      return DataTypes.LONG;
     }
   }
 
@@ -250,8 +246,8 @@ public class DefaultEncodingFactory extends EncodingFactory {
     DataType adaptiveDataType = fitMinMax(stats.getDataType(), stats.getMax(), stats.getMin());
     DataType deltaDataType;
 
-    if (adaptiveDataType == DataType.LONG) {
-      deltaDataType = DataType.LONG;
+    if (adaptiveDataType == DataTypes.LONG) {
+      deltaDataType = DataTypes.LONG;
     } else {
       deltaDataType = fitDelta(stats.getDataType(), stats.getMax(), stats.getMin());
     }
@@ -287,15 +283,15 @@ public class DefaultEncodingFactory extends EncodingFactory {
       // short, int, long
       return selectCodecByAlgorithmForIntegral(stats);
     } else if (decimalCount < 0) {
-      return new DirectCompressCodec(DataType.DOUBLE);
+      return new DirectCompressCodec(DataTypes.DOUBLE);
     } else {
       // double
       long max = (long) (Math.pow(10, decimalCount) * absMaxValue);
       DataType adaptiveDataType = fitLongMinMax(max, 0);
-      if (adaptiveDataType.getSizeInBytes() < DataType.DOUBLE.getSizeInBytes()) {
+      if (adaptiveDataType.getSizeInBytes() < DataTypes.DOUBLE.getSizeInBytes()) {
         return new AdaptiveFloatingCodec(srcDataType, adaptiveDataType, stats);
       } else {
-        return new DirectCompressCodec(DataType.DOUBLE);
+        return new DirectCompressCodec(DataTypes.DOUBLE);
       }
     }
   }
@@ -312,8 +308,8 @@ public class DefaultEncodingFactory extends EncodingFactory {
             decimalConverterType);
     DataType deltaDataType;
 
-    if (adaptiveDataType == DataType.LONG) {
-      deltaDataType = DataType.LONG;
+    if (adaptiveDataType == DataTypes.LONG) {
+      deltaDataType = DataTypes.LONG;
     } else {
       deltaDataType = fitDeltaForDecimalType(stats.getDataType(), stats.getMax(), stats.getMin(),
           decimalConverterType);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
index 1cb1613..180228a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.datastore.page.statistics.PrimitivePageStatsCo
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.Encoding;
 
@@ -108,60 +109,60 @@ public abstract class EncodingFactory {
     TableSpec.ColumnSpec spec = new TableSpec.ColumnSpec("legacy", stats.getDataType(),
         ColumnType.MEASURE);
     String compressor = "snappy";
-    switch (DataType.getDataType(metadata.getType())) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-        // create the codec based on algorithm and create decoder by recovering the metadata
-        ColumnPageCodec codec = DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(stats);
-        if (codec instanceof AdaptiveIntegralCodec) {
-          AdaptiveIntegralCodec adaptiveCodec = (AdaptiveIntegralCodec) codec;
-          ColumnPageEncoderMeta meta = new ColumnPageEncoderMeta(spec,
-              adaptiveCodec.getTargetDataType(), stats, compressor);
-          return codec.createDecoder(meta);
-        } else if (codec instanceof AdaptiveDeltaIntegralCodec) {
-          AdaptiveDeltaIntegralCodec adaptiveCodec = (AdaptiveDeltaIntegralCodec) codec;
-          ColumnPageEncoderMeta meta = new ColumnPageEncoderMeta(spec,
-              adaptiveCodec.getTargetDataType(), stats, compressor);
-          return codec.createDecoder(meta);
-        } else if (codec instanceof DirectCompressCodec) {
-          ColumnPageEncoderMeta meta = new ColumnPageEncoderMeta(spec,
-              DataType.getDataType(metadata.getType()), stats, compressor);
-          return codec.createDecoder(meta);
-        } else {
-          throw new RuntimeException("internal error");
-        }
-      case FLOAT:
-      case DOUBLE:
-        // create the codec based on algorithm and create decoder by recovering the metadata
-        codec = DefaultEncodingFactory.selectCodecByAlgorithmForFloating(stats);
-        if (codec instanceof AdaptiveFloatingCodec) {
-          AdaptiveFloatingCodec adaptiveCodec = (AdaptiveFloatingCodec) codec;
-          ColumnPageEncoderMeta meta = new ColumnPageEncoderMeta(spec,
-              adaptiveCodec.getTargetDataType(), stats, compressor);
-          return codec.createDecoder(meta);
-        } else if (codec instanceof DirectCompressCodec) {
-          ColumnPageEncoderMeta meta = new ColumnPageEncoderMeta(spec,
-              DataType.getDataType(metadata.getType()), stats, compressor);
-          return codec.createDecoder(meta);
-        } else {
-          throw new RuntimeException("internal error");
-        }
-      case DECIMAL:
-      case BYTE_ARRAY:
-        // no dictionary dimension
-        return new DirectCompressCodec(stats.getDataType()).createDecoder(
-            new ColumnPageEncoderMeta(spec, stats.getDataType(), stats, compressor));
-        // In case of older versions like in V1 format it has special datatype to handle
-      case LEGACY_LONG:
-        AdaptiveIntegralCodec adaptiveCodec =
-            new AdaptiveIntegralCodec(DataType.LONG, DataType.LONG, stats);
-        ColumnPageEncoderMeta meta = new ColumnPageEncoderMeta(spec,
-            adaptiveCodec.getTargetDataType(), stats, compressor);
-        return adaptiveCodec.createDecoder(meta);
-      default:
-        throw new RuntimeException("unsupported data type: " + stats.getDataType());
+    DataType dataType = DataType.getDataType(metadata.getType());
+    if (dataType == DataTypes.BYTE ||
+        dataType == DataTypes.SHORT ||
+        dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG) {
+      // create the codec based on algorithm and create decoder by recovering the metadata
+      ColumnPageCodec codec = DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(stats);
+      if (codec instanceof AdaptiveIntegralCodec) {
+        AdaptiveIntegralCodec adaptiveCodec = (AdaptiveIntegralCodec) codec;
+        ColumnPageEncoderMeta meta =
+            new ColumnPageEncoderMeta(spec, adaptiveCodec.getTargetDataType(), stats, compressor);
+        return codec.createDecoder(meta);
+      } else if (codec instanceof AdaptiveDeltaIntegralCodec) {
+        AdaptiveDeltaIntegralCodec adaptiveCodec = (AdaptiveDeltaIntegralCodec) codec;
+        ColumnPageEncoderMeta meta =
+            new ColumnPageEncoderMeta(spec, adaptiveCodec.getTargetDataType(), stats, compressor);
+        return codec.createDecoder(meta);
+      } else if (codec instanceof DirectCompressCodec) {
+        ColumnPageEncoderMeta meta =
+            new ColumnPageEncoderMeta(spec, DataType.getDataType(metadata.getType()), stats,
+                compressor);
+        return codec.createDecoder(meta);
+      } else {
+        throw new RuntimeException("internal error");
+      }
+    } else if (dataType == DataTypes.FLOAT || dataType == DataTypes.DOUBLE) {
+      // create the codec based on algorithm and create decoder by recovering the metadata
+      ColumnPageCodec codec = DefaultEncodingFactory.selectCodecByAlgorithmForFloating(stats);
+      if (codec instanceof AdaptiveFloatingCodec) {
+        AdaptiveFloatingCodec adaptiveCodec = (AdaptiveFloatingCodec) codec;
+        ColumnPageEncoderMeta meta =
+            new ColumnPageEncoderMeta(spec, adaptiveCodec.getTargetDataType(), stats, compressor);
+        return codec.createDecoder(meta);
+      } else if (codec instanceof DirectCompressCodec) {
+        ColumnPageEncoderMeta meta =
+            new ColumnPageEncoderMeta(spec, DataType.getDataType(metadata.getType()), stats,
+                compressor);
+        return codec.createDecoder(meta);
+      } else {
+        throw new RuntimeException("internal error");
+      }
+    } else if (dataType == DataTypes.DECIMAL || dataType == DataTypes.BYTE_ARRAY) {
+      // no dictionary dimension
+      return new DirectCompressCodec(stats.getDataType())
+          .createDecoder(new ColumnPageEncoderMeta(spec, stats.getDataType(), stats, compressor));
+    } else if (dataType == DataTypes.LEGACY_LONG) {
+      // In case of older versions like in V1 format it has special datatype to handle
+      AdaptiveIntegralCodec adaptiveCodec =
+          new AdaptiveIntegralCodec(DataTypes.LONG, DataTypes.LONG, stats);
+      ColumnPageEncoderMeta meta =
+          new ColumnPageEncoderMeta(spec, adaptiveCodec.getTargetDataType(), stats, compressor);
+      return adaptiveCodec.createDecoder(meta);
+    } else {
+      throw new RuntimeException("unsupported data type: " + stats.getDataType());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
index 383670a..96f7b16 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
@@ -34,10 +34,9 @@ import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.format.Encoding;
 
-import static org.apache.carbondata.core.metadata.datatype.DataType.DECIMAL;
-
 /**
  * Codec for integer (byte, short, int, long) data type and floating data type (in case of
  * scale is 0).
@@ -52,29 +51,22 @@ public class AdaptiveDeltaIntegralCodec extends AdaptiveCodec {
   public AdaptiveDeltaIntegralCodec(DataType srcDataType, DataType targetDataType,
       SimpleStatsResult stats) {
     super(srcDataType, targetDataType, stats);
-    switch (srcDataType) {
-      case BYTE:
-        this.max = (byte) stats.getMax();
-        break;
-      case SHORT:
-        this.max = (short) stats.getMax();
-        break;
-      case INT:
-        this.max = (int) stats.getMax();
-        break;
-      case LONG:
-        this.max = (long) stats.getMax();
-        break;
-      case DOUBLE:
-        this.max = (long) (double) stats.getMax();
-        break;
-      case DECIMAL:
-        this.max = ((BigDecimal) stats.getMax()).unscaledValue().longValue();
-        break;
-      default:
-        // this codec is for integer type only
-        throw new UnsupportedOperationException(
-            "unsupported data type for Delta compress: " + srcDataType);
+    if (srcDataType == DataTypes.BYTE) {
+      this.max = (byte) stats.getMax();
+    } else if (srcDataType == DataTypes.SHORT) {
+      this.max = (short) stats.getMax();
+    } else if (srcDataType == DataTypes.INT) {
+      this.max = (int) stats.getMax();
+    } else if (srcDataType == DataTypes.LONG) {
+      this.max = (long) stats.getMax();
+    } else if (srcDataType == DataTypes.DOUBLE) {
+      this.max = (long) (double) stats.getMax();
+    } else if (srcDataType == DataTypes.DECIMAL) {
+      this.max = ((BigDecimal) stats.getMax()).unscaledValue().longValue();
+    } else {
+      // this codec is for integer type only
+      throw new UnsupportedOperationException(
+          "unsupported data type for Delta compress: " + srcDataType);
     }
   }
 
@@ -122,12 +114,10 @@ public class AdaptiveDeltaIntegralCodec extends AdaptiveCodec {
       @Override public ColumnPage decode(byte[] input, int offset, int length)
           throws MemoryException, IOException {
         ColumnPage page = null;
-        switch (meta.getSchemaDataType()) {
-          case DECIMAL:
-            page = ColumnPage.decompressDecimalPage(meta, input, offset, length);
-            break;
-          default:
-            page = ColumnPage.decompress(meta, input, offset, length);
+        if (meta.getSchemaDataType() == DataTypes.DECIMAL) {
+          page = ColumnPage.decompressDecimalPage(meta, input, offset, length);
+        } else {
+          page = ColumnPage.decompress(meta, input, offset, length);
         }
         return LazyColumnPage.newPage(page, converter);
       }
@@ -137,115 +127,87 @@ public class AdaptiveDeltaIntegralCodec extends AdaptiveCodec {
   private ColumnPageValueConverter converter = new ColumnPageValueConverter() {
     @Override
     public void encode(int rowId, byte value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte)(max - value));
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (max - value));
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 
     @Override
     public void encode(int rowId, short value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte)(max - value));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short)(max - value));
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (max - value));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (max - value));
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 
     @Override
     public void encode(int rowId, int value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte)(max - value));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short)(max - value));
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int)(max - value));
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int)(max - value));
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (max - value));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (max - value));
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) (max - value));
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 
     @Override
     public void encode(int rowId, long value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte)(max - value));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short)(max - value));
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int)(max - value));
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int)(max - value));
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, max - value);
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (max - value));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (max - value));
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, max - value);
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 
     @Override
     public void encode(int rowId, float value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte)(max - value));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short)(max - value));
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int)(max - value));
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int)(max - value));
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, (long)(max - value));
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (max - value));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (max - value));
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, (long) (max - value));
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 
     @Override
     public void encode(int rowId, double value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte)(max - value));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short)(max - value));
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int)(max - value));
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int)(max - value));
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, (long)(max - value));
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (max - value));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (max - value));
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) (max - value));
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, (long) (max - value));
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java
index c238245..fb3e248 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java
@@ -34,6 +34,7 @@ import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.format.Encoding;
 
 /**
@@ -135,50 +136,37 @@ public class AdaptiveFloatingCodec extends AdaptiveCodec {
 
     @Override
     public void encode(int rowId, float value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) (value * factor));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short) (value * factor));
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int) (value * factor));
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int) (value * factor));
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, (long) (value * factor));
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (value * factor));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (value * factor));
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) (value * factor));
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) (value * factor));
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, (long) (value * factor));
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 
     @Override
     public void encode(int rowId, double value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) (value * factor));
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short) (value * factor));
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int) (value * factor));
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int) (value * factor));
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, (long) (value * factor));
-          break;
-        case DOUBLE:
-          encodedPage.putDouble(rowId, value);
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) (value * factor));
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) (value * factor));
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) (value * factor));
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) (value * factor));
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, (long) (value * factor));
+      } else if (targetDataType == DataTypes.DOUBLE) {
+        encodedPage.putDouble(rowId, value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
index bbc28a6..907649d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.format.Encoding;
 
 /**
@@ -93,12 +94,10 @@ public class AdaptiveIntegralCodec extends AdaptiveCodec {
       public ColumnPage decode(byte[] input, int offset, int length)
           throws MemoryException, IOException {
         ColumnPage page = null;
-        switch (meta.getSchemaDataType()) {
-          case DECIMAL:
-            page = ColumnPage.decompressDecimalPage(meta, input, offset, length);
-            break;
-          default:
-            page = ColumnPage.decompress(meta, input, offset, length);
+        if (meta.getSchemaDataType() == DataTypes.DECIMAL) {
+          page = ColumnPage.decompressDecimalPage(meta, input, offset, length);
+        } else {
+          page = ColumnPage.decompress(meta, input, offset, length);
         }
         return LazyColumnPage.newPage(page, converter);
       }
@@ -109,106 +108,85 @@ public class AdaptiveIntegralCodec extends AdaptiveCodec {
   private ColumnPageValueConverter converter = new ColumnPageValueConverter() {
     @Override
     public void encode(int rowId, byte value) {
-      switch (targetDataType) {
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 
     @Override
     public void encode(int rowId, short value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) value);
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) value);
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 
     @Override
     public void encode(int rowId, int value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) value);
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short) value);
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, value);
-          break;
-        case INT:
-          encodedPage.putInt(rowId, value);
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) value);
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) value);
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, value);
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 
     @Override
     public void encode(int rowId, long value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) value);
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short) value);
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int) value);
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int) value);
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, (long) value);
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) value);
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) value);
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) value);
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) value);
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, (long) value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 
     @Override
     public void encode(int rowId, float value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) value);
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short) value);
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int) value);
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int) value);
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) value);
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) value);
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) value);
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 
     @Override
     public void encode(int rowId, double value) {
-      switch (targetDataType) {
-        case BYTE:
-          encodedPage.putByte(rowId, (byte) value);
-          break;
-        case SHORT:
-          encodedPage.putShort(rowId, (short) value);
-          break;
-        case SHORT_INT:
-          encodedPage.putShortInt(rowId, (int) value);
-          break;
-        case INT:
-          encodedPage.putInt(rowId, (int) value);
-          break;
-        case LONG:
-          encodedPage.putLong(rowId, (long) value);
-          break;
-        default:
-          throw new RuntimeException("internal error: " + debugInfo());
+      if (targetDataType == DataTypes.BYTE) {
+        encodedPage.putByte(rowId, (byte) value);
+      } else if (targetDataType == DataTypes.SHORT) {
+        encodedPage.putShort(rowId, (short) value);
+      } else if (targetDataType == DataTypes.SHORT_INT) {
+        encodedPage.putShortInt(rowId, (int) value);
+      } else if (targetDataType == DataTypes.INT) {
+        encodedPage.putInt(rowId, (int) value);
+      } else if (targetDataType == DataTypes.LONG) {
+        encodedPage.putLong(rowId, (long) value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
index 13879b9..b3d282e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
@@ -34,6 +34,7 @@ import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
 import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.format.Encoding;
 
 /**
@@ -102,7 +103,7 @@ public class DirectCompressCodec implements ColumnPageCodec {
     @Override
     public ColumnPage decode(byte[] input, int offset, int length) throws MemoryException {
       ColumnPage decodedPage;
-      if (dataType == DataType.DECIMAL) {
+      if (dataType == DataTypes.DECIMAL) {
         decodedPage = ColumnPage.decompressDecimalPage(meta, input, offset, length);
       } else {
         decodedPage = ColumnPage.decompress(meta, input, offset, length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/rle/RLECodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/rle/RLECodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/rle/RLECodec.java
index 419b589..809bac0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/rle/RLECodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/rle/RLECodec.java
@@ -34,6 +34,7 @@ import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
 import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.format.Encoding;
 
 /**
@@ -70,14 +71,9 @@ public class RLECodec implements ColumnPageCodec {
 
   // This codec supports integral type only
   private void validateDataType(DataType dataType) {
-    switch (dataType) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-        break;
-      default:
-        throw new UnsupportedOperationException(dataType + " is not supported for RLE");
+    if (! (dataType == DataTypes.BYTE || dataType == DataTypes.SHORT || dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG)) {
+      throw new UnsupportedOperationException(dataType + " is not supported for RLE");
     }
   }
 
@@ -117,34 +113,29 @@ public class RLECodec implements ColumnPageCodec {
     protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException {
       validateDataType(input.getDataType());
       this.dataType = input.getDataType();
-      switch (dataType) {
-        case BYTE:
-          byte[] bytePage = input.getBytePage();
-          for (int i = 0; i < bytePage.length; i++) {
-            putValue(bytePage[i]);
-          }
-          break;
-        case SHORT:
-          short[] shortPage = input.getShortPage();
-          for (int i = 0; i < shortPage.length; i++) {
-            putValue(shortPage[i]);
-          }
-          break;
-        case INT:
-          int[] intPage = input.getIntPage();
-          for (int i = 0; i < intPage.length; i++) {
-            putValue(intPage[i]);
-          }
-          break;
-        case LONG:
-          long[] longPage = input.getLongPage();
-          for (int i = 0; i < longPage.length; i++) {
-            putValue(longPage[i]);
-          }
-          break;
-        default:
-          throw new UnsupportedOperationException(input.getDataType() +
-              " does not support RLE encoding");
+      if (dataType == DataTypes.BYTE) {
+        byte[] bytePage = input.getBytePage();
+        for (int i = 0; i < bytePage.length; i++) {
+          putValue(bytePage[i]);
+        }
+      } else if (dataType == DataTypes.SHORT) {
+        short[] shortPage = input.getShortPage();
+        for (int i = 0; i < shortPage.length; i++) {
+          putValue(shortPage[i]);
+        }
+      } else if (dataType == DataTypes.INT) {
+        int[] intPage = input.getIntPage();
+        for (int i = 0; i < intPage.length; i++) {
+          putValue(intPage[i]);
+        }
+      } else if (dataType == DataTypes.LONG) {
+        long[] longPage = input.getLongPage();
+        for (int i = 0; i < longPage.length; i++) {
+          putValue(longPage[i]);
+        }
+      } else {
+        throw new UnsupportedOperationException(input.getDataType() +
+            " does not support RLE encoding");
       }
       return collectResult();
     }
@@ -200,21 +191,16 @@ public class RLECodec implements ColumnPageCodec {
     }
 
     private void writeRunValue(Object value) throws IOException {
-      switch (dataType) {
-        case BYTE:
-          stream.writeByte((byte) value);
-          break;
-        case SHORT:
-          stream.writeShort((short) value);
-          break;
-        case INT:
-          stream.writeInt((int) value);
-          break;
-        case LONG:
-          stream.writeLong((long) value);
-          break;
-        default:
-          throw new RuntimeException("internal error");
+      if (dataType == DataTypes.BYTE) {
+        stream.writeByte((byte) value);
+      } else if (dataType == DataTypes.SHORT) {
+        stream.writeShort((short) value);
+      } else if (dataType == DataTypes.INT) {
+        stream.writeInt((int) value);
+      } else if (dataType == DataTypes.LONG) {
+        stream.writeLong((long) value);
+      } else {
+        throw new RuntimeException("internal error");
       }
     }
 
@@ -307,21 +293,16 @@ public class RLECodec implements ColumnPageCodec {
       DataType dataType = columnSpec.getSchemaDataType();
       DataInputStream in = new DataInputStream(new ByteArrayInputStream(input, offset, length));
       ColumnPage resultPage = ColumnPage.newPage(columnSpec, dataType, pageSize);
-      switch (dataType) {
-        case BYTE:
-          decodeBytePage(in, resultPage);
-          break;
-        case SHORT:
-          decodeShortPage(in, resultPage);
-          break;
-        case INT:
-          decodeIntPage(in, resultPage);
-          break;
-        case LONG:
-          decodeLongPage(in, resultPage);
-          break;
-        default:
-          throw new RuntimeException("unsupported datatype:" + dataType);
+      if (dataType == DataTypes.BYTE) {
+        decodeBytePage(in, resultPage);
+      } else if (dataType == DataTypes.SHORT) {
+        decodeShortPage(in, resultPage);
+      } else if (dataType == DataTypes.INT) {
+        decodeIntPage(in, resultPage);
+      } else if (dataType == DataTypes.LONG) {
+        decodeLongPage(in, resultPage);
+      } else {
+        throw new RuntimeException("unsupported datatype:" + dataType);
       }
       return resultPage;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
index 1b26a60..20e10b8 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.core.datastore.page.statistics;
 import java.math.BigDecimal;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 
 public class LVStringStatsCollector implements ColumnPageStatsCollector {
@@ -115,7 +116,7 @@ public class LVStringStatsCollector implements ColumnPageStatsCollector {
       }
 
       @Override public DataType getDataType() {
-        return DataType.STRING;
+        return DataTypes.STRING;
       }
 
       @Override public int getScale() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
index 9490b93..304d998 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
@@ -22,6 +22,7 @@ import java.math.BigDecimal;
 import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 /** statics for primitive column page */
 public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, SimpleStatsResult {
@@ -43,10 +44,7 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
   // this is for encode flow
   public static PrimitivePageStatsCollector newInstance(DataType dataType,
       int scale, int precision) {
-    switch (dataType) {
-      default:
-        return new PrimitivePageStatsCollector(dataType, scale, precision);
-    }
+    return new PrimitivePageStatsCollector(dataType, scale, precision);
   }
 
   // this is for decode flow, create stats from encoder meta in carbondata file
@@ -54,38 +52,32 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
     PrimitivePageStatsCollector instance = new PrimitivePageStatsCollector(meta.getSchemaDataType(),
         meta.getScale(), meta.getPrecision());
     // set min max from meta
-    switch (meta.getSchemaDataType()) {
-      case BYTE:
-        instance.minByte = (byte) meta.getMinValue();
-        instance.maxByte = (byte) meta.getMaxValue();
-        break;
-      case SHORT:
-        instance.minShort = (short) meta.getMinValue();
-        instance.maxShort = (short) meta.getMaxValue();
-        break;
-      case INT:
-        instance.minInt = (int) meta.getMinValue();
-        instance.maxInt = (int) meta.getMaxValue();
-        break;
-      case LONG:
-        instance.minLong = (long) meta.getMinValue();
-        instance.maxLong = (long) meta.getMaxValue();
-        break;
-      case DOUBLE:
-        instance.minDouble = (double) meta.getMinValue();
-        instance.maxDouble = (double) meta.getMaxValue();
-        instance.decimal = meta.getDecimal();
-        break;
-      case DECIMAL:
-        instance.minDecimal = (BigDecimal) meta.getMinValue();
-        instance.maxDecimal = (BigDecimal) meta.getMaxValue();
-        instance.decimal = meta.getDecimal();
-        instance.scale = meta.getScale();
-        instance.precision = meta.getPrecision();
-        break;
-      default:
-        throw new UnsupportedOperationException(
-            "unsupported data type for stats collection: " + meta.getSchemaDataType());
+    DataType dataType = meta.getSchemaDataType();
+    if (dataType == DataTypes.BYTE) {
+      instance.minByte = (byte) meta.getMinValue();
+      instance.maxByte = (byte) meta.getMaxValue();
+    } else if (dataType == DataTypes.SHORT) {
+      instance.minShort = (short) meta.getMinValue();
+      instance.maxShort = (short) meta.getMaxValue();
+    } else if (dataType == DataTypes.INT) {
+      instance.minInt = (int) meta.getMinValue();
+      instance.maxInt = (int) meta.getMaxValue();
+    } else if (dataType == DataTypes.LONG) {
+      instance.minLong = (long) meta.getMinValue();
+      instance.maxLong = (long) meta.getMaxValue();
+    } else if (dataType == DataTypes.DOUBLE) {
+      instance.minDouble = (double) meta.getMinValue();
+      instance.maxDouble = (double) meta.getMaxValue();
+      instance.decimal = meta.getDecimal();
+    } else if (dataType == DataTypes.DECIMAL) {
+      instance.minDecimal = (BigDecimal) meta.getMinValue();
+      instance.maxDecimal = (BigDecimal) meta.getMaxValue();
+      instance.decimal = meta.getDecimal();
+      instance.scale = meta.getScale();
+      instance.precision = meta.getPrecision();
+    } else {
+      throw new UnsupportedOperationException(
+          "unsupported data type for stats collection: " + meta.getSchemaDataType());
     }
     return instance;
   }
@@ -94,112 +86,90 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
     PrimitivePageStatsCollector instance =
         new PrimitivePageStatsCollector(DataType.getDataType(meta.getType()), -1, -1);
     // set min max from meta
-    switch (DataType.getDataType(meta.getType())) {
-      case BYTE:
-        instance.minByte = (byte) meta.getMinValue();
-        instance.maxByte = (byte) meta.getMaxValue();
-        break;
-      case SHORT:
-        instance.minShort = (short) meta.getMinValue();
-        instance.maxShort = (short) meta.getMaxValue();
-        break;
-      case INT:
-        instance.minInt = (int) meta.getMinValue();
-        instance.maxInt = (int) meta.getMaxValue();
-        break;
-      case LEGACY_LONG:
-      case LONG:
-        instance.minLong = (long) meta.getMinValue();
-        instance.maxLong = (long) meta.getMaxValue();
-        break;
-      case DOUBLE:
-        instance.minDouble = (double) meta.getMinValue();
-        instance.maxDouble = (double) meta.getMaxValue();
-        instance.decimal = meta.getDecimal();
-        break;
-      case DECIMAL:
-        instance.minDecimal = (BigDecimal) meta.getMinValue();
-        instance.maxDecimal = (BigDecimal) meta.getMaxValue();
-        instance.decimal = meta.getDecimal();
-        instance.scale = -1;
-        instance.precision = -1;
-        break;
-      default:
-        throw new UnsupportedOperationException(
-            "unsupported data type for Stats collection: " + meta.getType());
+    DataType dataType = DataType.getDataType(meta.getType());
+    if (dataType == DataTypes.BYTE) {
+      instance.minByte = (byte) meta.getMinValue();
+      instance.maxByte = (byte) meta.getMaxValue();
+    } else if (dataType == DataTypes.SHORT) {
+      instance.minShort = (short) meta.getMinValue();
+      instance.maxShort = (short) meta.getMaxValue();
+    } else if (dataType == DataTypes.INT) {
+      instance.minInt = (int) meta.getMinValue();
+      instance.maxInt = (int) meta.getMaxValue();
+    } else if (dataType == DataTypes.LEGACY_LONG || dataType == DataTypes.LONG) {
+      instance.minLong = (long) meta.getMinValue();
+      instance.maxLong = (long) meta.getMaxValue();
+    } else if (dataType == DataTypes.DOUBLE) {
+      instance.minDouble = (double) meta.getMinValue();
+      instance.maxDouble = (double) meta.getMaxValue();
+      instance.decimal = meta.getDecimal();
+    } else if (dataType == DataTypes.DECIMAL) {
+      instance.minDecimal = (BigDecimal) meta.getMinValue();
+      instance.maxDecimal = (BigDecimal) meta.getMaxValue();
+      instance.decimal = meta.getDecimal();
+      instance.scale = -1;
+      instance.precision = -1;
+    } else {
+      throw new UnsupportedOperationException(
+          "unsupported data type for Stats collection: " + meta.getType());
     }
     return instance;
   }
 
   private PrimitivePageStatsCollector(DataType dataType, int scale, int precision) {
     this.dataType = dataType;
-    switch (dataType) {
-      case BYTE:
-        minByte = Byte.MAX_VALUE;
-        maxByte = Byte.MIN_VALUE;
-        break;
-      case SHORT:
-        minShort = Short.MAX_VALUE;
-        maxShort = Short.MIN_VALUE;
-        break;
-      case INT:
-        minInt = Integer.MAX_VALUE;
-        maxInt = Integer.MIN_VALUE;
-        break;
-      case LEGACY_LONG:
-      case LONG:
-        minLong = Long.MAX_VALUE;
-        maxLong = Long.MIN_VALUE;
-        break;
-      case DOUBLE:
-        minDouble = Double.POSITIVE_INFINITY;
-        maxDouble = Double.NEGATIVE_INFINITY;
-        decimal = 0;
-        break;
-      case DECIMAL:
-        this.zeroDecimal = BigDecimal.ZERO;
-        decimal = scale;
-        this.scale = scale;
-        this.precision = precision;
-        break;
-      default:
-        throw new UnsupportedOperationException(
-            "unsupported data type for Stats collection: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      minByte = Byte.MAX_VALUE;
+      maxByte = Byte.MIN_VALUE;
+    } else if (dataType == DataTypes.SHORT) {
+      minShort = Short.MAX_VALUE;
+      maxShort = Short.MIN_VALUE;
+    } else if (dataType == DataTypes.INT) {
+      minInt = Integer.MAX_VALUE;
+      maxInt = Integer.MIN_VALUE;
+    } else if (dataType == DataTypes.LEGACY_LONG || dataType == DataTypes.LONG) {
+      minLong = Long.MAX_VALUE;
+      maxLong = Long.MIN_VALUE;
+    } else if (dataType == DataTypes.DOUBLE) {
+      minDouble = Double.POSITIVE_INFINITY;
+      maxDouble = Double.NEGATIVE_INFINITY;
+      decimal = 0;
+    } else if (dataType == DataTypes.DECIMAL) {
+      this.zeroDecimal = BigDecimal.ZERO;
+      decimal = scale;
+      this.scale = scale;
+      this.precision = precision;
+    } else {
+      throw new UnsupportedOperationException(
+          "unsupported data type for Stats collection: " + dataType);
     }
   }
 
   @Override
   public void updateNull(int rowId) {
     long value = 0;
-    switch (dataType) {
-      case BYTE:
-        update((byte) value);
-        break;
-      case SHORT:
-        update((short) value);
-        break;
-      case INT:
-        update((int) value);
-        break;
-      case LONG:
-        update(value);
-        break;
-      case DOUBLE:
-        update(0d);
-        break;
-      case DECIMAL:
-        if (isFirst) {
-          maxDecimal = zeroDecimal;
-          minDecimal = zeroDecimal;
-          isFirst = false;
-        } else {
-          maxDecimal = (maxDecimal.compareTo(zeroDecimal) > 0) ? maxDecimal : zeroDecimal;
-          minDecimal = (minDecimal.compareTo(zeroDecimal) < 0) ? minDecimal : zeroDecimal;
-        }
-        break;
-      default:
-        throw new UnsupportedOperationException(
-            "unsupported data type for Stats collection: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      update((byte) value);
+    } else if (dataType == DataTypes.SHORT) {
+      update((short) value);
+    } else if (dataType == DataTypes.INT) {
+      update((int) value);
+    } else if (dataType == DataTypes.LONG) {
+      update(value);
+    } else if (dataType == DataTypes.DOUBLE) {
+      update(0d);
+    } else if (dataType == DataTypes.DECIMAL) {
+      if (isFirst) {
+        maxDecimal = zeroDecimal;
+        minDecimal = zeroDecimal;
+        isFirst = false;
+      } else {
+        maxDecimal = (maxDecimal.compareTo(zeroDecimal) > 0) ? maxDecimal : zeroDecimal;
+        minDecimal = (minDecimal.compareTo(zeroDecimal) < 0) ? minDecimal : zeroDecimal;
+      }
+    } else {
+      throw new UnsupportedOperationException(
+          "unsupported data type for Stats collection: " + dataType);
     }
   }
 
@@ -300,55 +270,52 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
 
   @Override
   public String toString() {
-    switch (dataType) {
-      case BYTE:
-        return String.format("min: %s, max: %s, decimal: %s ", minByte, maxByte, decimal);
-      case SHORT:
-        return String.format("min: %s, max: %s, decimal: %s ", minShort, maxShort, decimal);
-      case INT:
-        return String.format("min: %s, max: %s, decimal: %s ", minInt, maxInt, decimal);
-      case LONG:
-        return String.format("min: %s, max: %s, decimal: %s ", minLong, maxLong, decimal);
-      case DOUBLE:
-        return String.format("min: %s, max: %s, decimal: %s ", minDouble, maxDouble, decimal);
+    if (dataType == DataTypes.BYTE) {
+      return String.format("min: %s, max: %s, decimal: %s ", minByte, maxByte, decimal);
+    } else if (dataType == DataTypes.SHORT) {
+      return String.format("min: %s, max: %s, decimal: %s ", minShort, maxShort, decimal);
+    } else if (dataType == DataTypes.INT) {
+      return String.format("min: %s, max: %s, decimal: %s ", minInt, maxInt, decimal);
+    } else if (dataType == DataTypes.LONG) {
+      return String.format("min: %s, max: %s, decimal: %s ", minLong, maxLong, decimal);
+    } else if (dataType == DataTypes.DOUBLE) {
+      return String.format("min: %s, max: %s, decimal: %s ", minDouble, maxDouble, decimal);
     }
     return super.toString();
   }
 
   @Override
   public Object getMin() {
-    switch (dataType) {
-      case BYTE:
-        return minByte;
-      case SHORT:
-        return minShort;
-      case INT:
-        return minInt;
-      case LONG:
-        return minLong;
-      case DOUBLE:
-        return minDouble;
-      case DECIMAL:
-        return minDecimal;
+    if (dataType == DataTypes.BYTE) {
+      return minByte;
+    } else if (dataType == DataTypes.SHORT) {
+      return minShort;
+    } else if (dataType == DataTypes.INT) {
+      return minInt;
+    } else if (dataType == DataTypes.LONG) {
+      return minLong;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return minDouble;
+    } else if (dataType == DataTypes.DECIMAL) {
+      return minDecimal;
     }
     return null;
   }
 
   @Override
   public Object getMax() {
-    switch (dataType) {
-      case BYTE:
-        return maxByte;
-      case SHORT:
-        return maxShort;
-      case INT:
-        return maxInt;
-      case LONG:
-        return maxLong;
-      case DOUBLE:
-        return maxDouble;
-      case DECIMAL:
-        return maxDecimal;
+    if (dataType == DataTypes.BYTE) {
+      return maxByte;
+    } else if (dataType == DataTypes.SHORT) {
+      return maxShort;
+    } else if (dataType == DataTypes.INT) {
+      return maxInt;
+    } else if (dataType == DataTypes.LONG) {
+      return maxLong;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return maxDouble;
+    } else if (dataType == DataTypes.DECIMAL) {
+      return maxDecimal;
     }
     return null;
   }
@@ -363,11 +330,13 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
     return dataType;
   }
 
-  @Override public int getScale() {
+  @Override
+  public int getScale() {
     return scale;
   }
 
-  @Override public int getPrecision() {
+  @Override
+  public int getPrecision() {
     return precision;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index 5d17426..bf1678a 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -22,6 +22,8 @@ import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
 import static org.apache.carbondata.core.memory.CarbonUnsafe.BYTE_ARRAY_OFFSET;
@@ -102,52 +104,45 @@ public class UnsafeMemoryDMStore {
   private void addToUnsafe(DataMapSchema schema, DataMapRow row, int index) {
     switch (schema.getSchemaType()) {
       case FIXED:
-        switch (schema.getDataType()) {
-          case BYTE:
-            getUnsafe()
-                .putByte(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
-                    row.getByte(index));
-            runningLength += row.getSizeInBytes(index);
-            break;
-          case SHORT:
-            getUnsafe()
-                .putShort(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
-                    row.getShort(index));
-            runningLength += row.getSizeInBytes(index);
-            break;
-          case INT:
-            getUnsafe()
-                .putInt(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
-                    row.getInt(index));
-            runningLength += row.getSizeInBytes(index);
-            break;
-          case LONG:
-            getUnsafe()
-                .putLong(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
-                    row.getLong(index));
-            runningLength += row.getSizeInBytes(index);
-            break;
-          case FLOAT:
-            getUnsafe()
-                .putFloat(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
-                    row.getFloat(index));
-            runningLength += row.getSizeInBytes(index);
-            break;
-          case DOUBLE:
-            getUnsafe()
-                .putDouble(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
-                    row.getDouble(index));
-            runningLength += row.getSizeInBytes(index);
-            break;
-          case BYTE_ARRAY:
-            byte[] data = row.getByteArray(index);
-            getUnsafe().copyMemory(data, BYTE_ARRAY_OFFSET, memoryBlock.getBaseObject(),
-                memoryBlock.getBaseOffset() + runningLength, data.length);
-            runningLength += row.getSizeInBytes(index);
-            break;
-          default:
-            throw new UnsupportedOperationException(
-                "unsupported data type for unsafe storage: " + schema.getDataType());
+        DataType dataType = schema.getDataType();
+        if (dataType == DataTypes.BYTE) {
+          getUnsafe()
+              .putByte(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
+                  row.getByte(index));
+          runningLength += row.getSizeInBytes(index);
+        } else if (dataType == DataTypes.SHORT) {
+          getUnsafe()
+              .putShort(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
+                  row.getShort(index));
+          runningLength += row.getSizeInBytes(index);
+        } else if (dataType == DataTypes.INT) {
+          getUnsafe()
+              .putInt(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
+                  row.getInt(index));
+          runningLength += row.getSizeInBytes(index);
+        } else if (dataType == DataTypes.LONG) {
+          getUnsafe()
+              .putLong(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
+                  row.getLong(index));
+          runningLength += row.getSizeInBytes(index);
+        } else if (dataType == DataTypes.FLOAT) {
+          getUnsafe()
+              .putFloat(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
+                  row.getFloat(index));
+          runningLength += row.getSizeInBytes(index);
+        } else if (dataType == DataTypes.DOUBLE) {
+          getUnsafe()
+              .putDouble(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset() + runningLength,
+                  row.getDouble(index));
+          runningLength += row.getSizeInBytes(index);
+        } else if (dataType == DataTypes.BYTE_ARRAY) {
+          byte[] data = row.getByteArray(index);
+          getUnsafe().copyMemory(data, BYTE_ARRAY_OFFSET, memoryBlock.getBaseObject(),
+              memoryBlock.getBaseOffset() + runningLength, data.length);
+          runningLength += row.getSizeInBytes(index);
+        } else {
+          throw new UnsupportedOperationException(
+              "unsupported data type for unsafe storage: " + schema.getDataType());
         }
         break;
       case VARIABLE:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index 66d07dc..0d7bb71 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -51,6 +51,7 @@ import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
@@ -187,30 +188,25 @@ public class BlockletDataMap implements DataMap, Cacheable {
       ByteBuffer buffer = ByteBuffer.allocate(8);
       for (int i = 0; i < measures.size(); i++) {
         buffer.rewind();
-        switch (measures.get(i).getDataType()) {
-          case BYTE:
-            buffer.putLong(Byte.MIN_VALUE);
-            updatedValues[minValues.length + i] = buffer.array().clone();
-            break;
-          case SHORT:
-            buffer.putLong(Short.MIN_VALUE);
-            updatedValues[minValues.length + i] = buffer.array().clone();
-            break;
-          case INT:
-            buffer.putLong(Integer.MIN_VALUE);
-            updatedValues[minValues.length + i] = buffer.array().clone();
-            break;
-          case LONG:
-            buffer.putLong(Long.MIN_VALUE);
-            updatedValues[minValues.length + i] = buffer.array().clone();
-            break;
-          case DECIMAL:
-            updatedValues[minValues.length + i] =
-                DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MIN_VALUE));
-            break;
-          default:
-            buffer.putDouble(Double.MIN_VALUE);
-            updatedValues[minValues.length + i] = buffer.array().clone();
+        DataType dataType = measures.get(i).getDataType();
+        if (dataType == DataTypes.BYTE) {
+          buffer.putLong(Byte.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.SHORT) {
+          buffer.putLong(Short.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.INT) {
+          buffer.putLong(Integer.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.LONG) {
+          buffer.putLong(Long.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.DECIMAL) {
+          updatedValues[minValues.length + i] =
+              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MIN_VALUE));
+        } else {
+          buffer.putDouble(Double.MIN_VALUE);
+          updatedValues[minValues.length + i] = buffer.array().clone();
         }
       }
     }
@@ -230,30 +226,25 @@ public class BlockletDataMap implements DataMap, Cacheable {
       ByteBuffer buffer = ByteBuffer.allocate(8);
       for (int i = 0; i < measures.size(); i++) {
         buffer.rewind();
-        switch (measures.get(i).getDataType()) {
-          case BYTE:
-            buffer.putLong(Byte.MAX_VALUE);
-            updatedValues[maxValues.length + i] = buffer.array().clone();
-            break;
-          case SHORT:
-            buffer.putLong(Short.MAX_VALUE);
-            updatedValues[maxValues.length + i] = buffer.array().clone();
-            break;
-          case INT:
-            buffer.putLong(Integer.MAX_VALUE);
-            updatedValues[maxValues.length + i] = buffer.array().clone();
-            break;
-          case LONG:
-            buffer.putLong(Long.MAX_VALUE);
-            updatedValues[maxValues.length + i] = buffer.array().clone();
-            break;
-          case DECIMAL:
-            updatedValues[maxValues.length + i] =
-                DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MAX_VALUE));
-            break;
-          default:
-            buffer.putDouble(Double.MAX_VALUE);
-            updatedValues[maxValues.length + i] = buffer.array().clone();
+        DataType dataType = measures.get(i).getDataType();
+        if (dataType == DataTypes.BYTE) {
+          buffer.putLong(Byte.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.SHORT) {
+          buffer.putLong(Short.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.INT) {
+          buffer.putLong(Integer.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.LONG) {
+          buffer.putLong(Long.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
+        } else if (dataType == DataTypes.DECIMAL) {
+          updatedValues[maxValues.length + i] =
+              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MAX_VALUE));
+        } else {
+          buffer.putDouble(Double.MAX_VALUE);
+          updatedValues[maxValues.length + i] = buffer.array().clone();
         }
       }
     }
@@ -276,39 +267,39 @@ public class BlockletDataMap implements DataMap, Cacheable {
     List<DataMapSchema> indexSchemas = new ArrayList<>();
 
     // Index key
-    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataType.BYTE_ARRAY));
+    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
     int[] minMaxLen = segmentProperties.getColumnsValueSize();
     // do it 2 times, one for min and one for max.
     for (int k = 0; k < 2; k++) {
       DataMapSchema[] mapSchemas = new DataMapSchema[minMaxLen.length];
       for (int i = 0; i < minMaxLen.length; i++) {
         if (minMaxLen[i] <= 0) {
-          mapSchemas[i] = new DataMapSchema.VariableDataMapSchema(DataType.BYTE_ARRAY);
+          mapSchemas[i] = new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY);
         } else {
-          mapSchemas[i] = new DataMapSchema.FixedDataMapSchema(DataType.BYTE_ARRAY, minMaxLen[i]);
+          mapSchemas[i] = new DataMapSchema.FixedDataMapSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
         }
       }
-      DataMapSchema mapSchema = new DataMapSchema.StructDataMapSchema(DataType.STRUCT, mapSchemas);
+      DataMapSchema mapSchema = new DataMapSchema.StructDataMapSchema(DataTypes.STRUCT, mapSchemas);
       indexSchemas.add(mapSchema);
     }
 
     // for number of rows.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataType.INT));
+    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.INT));
 
     // for table block path
-    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataType.BYTE_ARRAY));
+    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
 
     // for number of pages.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataType.SHORT));
+    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.SHORT));
 
     // for version number.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataType.SHORT));
+    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.SHORT));
 
     // for schema updated time.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataType.LONG));
+    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.LONG));
 
     //for blocklet info
-    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataType.BYTE_ARRAY));
+    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
 
     unsafeMemoryDMStore =
         new UnsafeMemoryDMStore(indexSchemas.toArray(new DataMapSchema[indexSchemas.size()]));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
index 32d15d3..bc55e74 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
@@ -17,7 +17,7 @@
 package org.apache.carbondata.core.indexstore.row;
 
 import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 /**
  * Data map row.
@@ -44,7 +44,7 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setByteArray(byte[] byteArray, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.BYTE_ARRAY);
+    assert (schemas[ordinal].getDataType() == DataTypes.BYTE_ARRAY);
     data[ordinal] = byteArray;
   }
 
@@ -53,12 +53,12 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setInt(int value, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.INT);
+    assert (schemas[ordinal].getDataType() == DataTypes.INT);
     data[ordinal] = value;
   }
 
   @Override public void setByte(byte value, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.BYTE);
+    assert (schemas[ordinal].getDataType() == DataTypes.BYTE);
     data[ordinal] = value;
   }
 
@@ -67,7 +67,7 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setShort(short value, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.SHORT);
+    assert (schemas[ordinal].getDataType() == DataTypes.SHORT);
     data[ordinal] = value;
   }
 
@@ -76,7 +76,7 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setLong(long value, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.LONG);
+    assert (schemas[ordinal].getDataType() == DataTypes.LONG);
     data[ordinal] = value;
   }
 
@@ -85,7 +85,7 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setFloat(float value, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.FLOAT);
+    assert (schemas[ordinal].getDataType() == DataTypes.FLOAT);
     data[ordinal] = value;
   }
 
@@ -94,12 +94,12 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setDouble(double value, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.DOUBLE);
+    assert (schemas[ordinal].getDataType() == DataTypes.DOUBLE);
     data[ordinal] = value;
   }
 
   @Override public void setRow(DataMapRow row, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataType.STRUCT);
+    assert (schemas[ordinal].getDataType() == DataTypes.STRUCT);
     data[ordinal] = row;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/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
index e20ebb2..9ce6748 100644
--- 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
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.keygenerator.directdictionary;
 import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.DateDirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampDirectDictionaryGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
@@ -42,15 +43,10 @@ public final class DirectDictionaryKeyGeneratorFactory {
   public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType,
       String dateFormat) {
     DirectDictionaryGenerator directDictionaryGenerator = null;
-    switch (dataType) {
-      case DATE:
-        directDictionaryGenerator = new DateDirectDictionaryGenerator(dateFormat);
-        break;
-      case TIMESTAMP:
-        directDictionaryGenerator = new TimeStampDirectDictionaryGenerator(dateFormat);
-        break;
-      default:
-
+    if (dataType == DataTypes.DATE) {
+      directDictionaryGenerator = new DateDirectDictionaryGenerator(dateFormat);
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      directDictionaryGenerator = new TimeStampDirectDictionaryGenerator(dateFormat);
     }
     return directDictionaryGenerator;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/DateDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/DateDirectDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/DateDirectDictionaryGenerator.java
index 5a6e03d..cd32e97 100644
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/DateDirectDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/DateDirectDictionaryGenerator.java
@@ -26,7 +26,7 @@ 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.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 /**
  * The class provides the method to generate dictionary key and getting the actual value from
@@ -53,11 +53,6 @@ public class DateDirectDictionaryGenerator implements DirectDictionaryGenerator
     initialize();
   }
 
-  public DateDirectDictionaryGenerator() {
-    this(CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-  }
-
   /**
    * The method take member String as input and converts
    * and returns the dictionary key
@@ -159,6 +154,6 @@ public class DateDirectDictionaryGenerator implements DirectDictionaryGenerator
   }
 
   @Override public DataType getReturnType() {
-    return DataType.INT;
+    return DataTypes.INT;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/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
index e0f5d41..d218e99 100644
--- 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
@@ -25,16 +25,13 @@ 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.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 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;
+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
@@ -226,7 +223,7 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
   }
 
   @Override public DataType getReturnType() {
-    return DataType.LONG;
+    return DataTypes.LONG;
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 5862933..2203b3b 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
+import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
-import javax.annotation.concurrent.GuardedBy;
 
 /**
  * Code ported from Apache Spark {org.apache.spark.unsafe.memory} package


[08/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index bc877b7..6b2cb90 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.BucketingInfo;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
@@ -122,33 +123,32 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
    * @return
    */
   private org.apache.carbondata.format.DataType fromWrapperToExternalDataType(DataType dataType) {
-
     if (null == dataType) {
       return null;
     }
-    switch (dataType) {
-      case STRING:
-        return org.apache.carbondata.format.DataType.STRING;
-      case INT:
-        return org.apache.carbondata.format.DataType.INT;
-      case SHORT:
-        return org.apache.carbondata.format.DataType.SHORT;
-      case LONG:
-        return org.apache.carbondata.format.DataType.LONG;
-      case DOUBLE:
-        return org.apache.carbondata.format.DataType.DOUBLE;
-      case DECIMAL:
-        return org.apache.carbondata.format.DataType.DECIMAL;
-      case DATE:
-        return org.apache.carbondata.format.DataType.DATE;
-      case TIMESTAMP:
-        return org.apache.carbondata.format.DataType.TIMESTAMP;
-      case ARRAY:
-        return org.apache.carbondata.format.DataType.ARRAY;
-      case STRUCT:
-        return org.apache.carbondata.format.DataType.STRUCT;
-      default:
-        return org.apache.carbondata.format.DataType.STRING;
+    // data type object maybe created by GSON, use id to compare the type instead of object address
+    if (dataType.getId() == DataTypes.STRING.getId()) {
+      return org.apache.carbondata.format.DataType.STRING;
+    } else if (dataType.getId() == DataTypes.INT.getId()) {
+      return org.apache.carbondata.format.DataType.INT;
+    } else if (dataType.getId() == DataTypes.SHORT.getId()) {
+      return org.apache.carbondata.format.DataType.SHORT;
+    } else if (dataType.getId() == DataTypes.LONG.getId()) {
+      return org.apache.carbondata.format.DataType.LONG;
+    } else if (dataType.getId() == DataTypes.DOUBLE.getId()) {
+      return org.apache.carbondata.format.DataType.DOUBLE;
+    } else if (dataType.getId() == DataTypes.DECIMAL.getId()) {
+      return org.apache.carbondata.format.DataType.DECIMAL;
+    } else if (dataType.getId() == DataTypes.DATE.getId()) {
+      return org.apache.carbondata.format.DataType.DATE;
+    } else if (dataType.getId() == DataTypes.TIMESTAMP.getId()) {
+      return org.apache.carbondata.format.DataType.TIMESTAMP;
+    } else if (dataType.getId() == DataTypes.ARRAY.getId()) {
+      return org.apache.carbondata.format.DataType.ARRAY;
+    } else if (dataType.getId() == DataTypes.STRUCT.getId()) {
+      return org.apache.carbondata.format.DataType.STRUCT;
+    } else {
+      return org.apache.carbondata.format.DataType.STRING;
     }
   }
 
@@ -362,27 +362,27 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     }
     switch (dataType) {
       case STRING:
-        return DataType.STRING;
+        return DataTypes.STRING;
       case INT:
-        return DataType.INT;
+        return DataTypes.INT;
       case SHORT:
-        return DataType.SHORT;
+        return DataTypes.SHORT;
       case LONG:
-        return DataType.LONG;
+        return DataTypes.LONG;
       case DOUBLE:
-        return DataType.DOUBLE;
+        return DataTypes.DOUBLE;
       case DECIMAL:
-        return DataType.DECIMAL;
+        return DataTypes.DECIMAL;
       case TIMESTAMP:
-        return DataType.TIMESTAMP;
+        return DataTypes.TIMESTAMP;
       case DATE:
-        return DataType.DATE;
+        return DataTypes.DATE;
       case ARRAY:
-        return DataType.ARRAY;
+        return DataTypes.ARRAY;
       case STRUCT:
-        return DataType.STRUCT;
+        return DataTypes.STRUCT;
       default:
-        return DataType.STRING;
+        return DataTypes.STRING;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
new file mode 100644
index 0000000..241d0f6
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
@@ -0,0 +1,37 @@
+/*
+ * 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.datatype;
+
+public class ArrayType extends DataType {
+
+  public static final DataType ARRAY = new ArrayType(DataTypes.ARRAY_TYPE_ID, 9, "ARRAY", -1);
+
+  private ArrayType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.ARRAY;
+  }
+
+  @Override
+  public boolean isComplexType() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
new file mode 100644
index 0000000..36cb84f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
@@ -0,0 +1,33 @@
+/*
+ * 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.datatype;
+
+public class BooleanType extends DataType {
+
+  public static final DataType BOOLEAN =
+      new BooleanType(DataTypes.BOOLEAN_TYPE_ID, 1, "BOOLEAN", 1);
+
+  private BooleanType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.BOOLEAN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
new file mode 100644
index 0000000..1f2872d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
@@ -0,0 +1,33 @@
+/*
+ * 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.datatype;
+
+public class ByteArrayType extends DataType {
+
+  public static final DataType BYTE_ARRAY =
+      new ByteArrayType(DataTypes.BYTE_ARRAY_TYPE_ID, 13, "BYTE_ARRAY", -1);
+
+  private ByteArrayType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.BYTE_ARRAY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
new file mode 100644
index 0000000..11138d2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class ByteType extends DataType {
+
+  public static final DataType BYTE = new ByteType(DataTypes.BYTE_TYPE_ID, 12, "BYTE", 1);
+
+  private ByteType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.BYTE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
index a37f265..b3dd1bc 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
@@ -17,46 +17,22 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public enum DataType {
-
-  STRING(0, "STRING", -1),
-  DATE(1, "DATE", -1),
-  TIMESTAMP(2, "TIMESTAMP", -1),
-  BOOLEAN(1, "BOOLEAN", 1),
-  SHORT(2, "SMALLINT", 2),
-  INT(3, "INT", 4),
-  FLOAT(4, "FLOAT", 4),
-  LONG(5, "BIGINT", 8),
-  DOUBLE(6, "DOUBLE", 8),
-  NULL(7, "NULL", 1),
-  DECIMAL(8, "DECIMAL", -1),
-  ARRAY(9, "ARRAY", -1),
-  STRUCT(10, "STRUCT", -1),
-  MAP(11, "MAP", -1),
-  BYTE(12, "BYTE", 1),
-  // internal use only, for variable length data type
-  BYTE_ARRAY(13, "BYTE_ARRAY", -1),
-  // internal use only, for value compression from integer/long to 3 bytes value
-  SHORT_INT(14, "SHORT_INT", 3),
-  // Only for internal use for backward compatability. It is only used for V1 version
-  LEGACY_LONG(15, "LEGACYBIGINT", 8);
+import java.io.Serializable;
 
-  public static final char DOUBLE_MEASURE_CHAR = 'n';
-  public static final char STRING_CHAR = 's';
-  public static final char TIMESTAMP_CHAR = 't';
-  public static final char DATE_CHAR = 'x';
-  public static final char BYTE_ARRAY_CHAR = 'y';
-  public static final char BYTE_VALUE_MEASURE_CHAR = 'c';
-  public static final char BIG_DECIMAL_MEASURE_CHAR = 'b';
-  public static final char BIG_INT_MEASURE_CHAR = 'd';
+public class DataType implements Serializable {
+
+  private static final long serialVersionUID = 19371726L;
 
+  // id is used for comparison and serialization/deserialization
+  private int id;
   private int precedenceOrder;
   private String name;
 
   // size of the value of this data type, negative value means variable length
   private int sizeInBytes;
 
-  DataType(int precedenceOrder, String name, int sizeInBytes) {
+  DataType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    this.id = id;
     this.precedenceOrder = precedenceOrder;
     this.name = name;
     this.sizeInBytes = sizeInBytes;
@@ -70,96 +46,70 @@ public enum DataType {
     return name;
   }
 
-  public boolean isComplexType() {
-    return precedenceOrder >= 9 && precedenceOrder <= 11;
-  }
-
   public int getSizeInBytes() {
     return sizeInBytes;
   }
 
   public int getSizeBits() {
-    if (this == SHORT_INT) {
-      throw new UnsupportedOperationException("Should not call this from datatype " + SHORT_INT);
-    }
     return (int) (Math.log(getSizeInBytes()) / Math.log(2));
   }
 
-  public static DataType valueOf(int ordinal) {
-    if (ordinal == STRING.ordinal()) {
-      return STRING;
-    } else if (ordinal == DATE.ordinal()) {
-      return DATE;
-    } else if (ordinal == TIMESTAMP.ordinal()) {
-      return TIMESTAMP;
-    } else if (ordinal == BOOLEAN.ordinal()) {
-      return BOOLEAN;
-    } else if (ordinal == SHORT.ordinal()) {
-      return SHORT;
-    } else if (ordinal == INT.ordinal()) {
-      return INT;
-    } else if (ordinal == FLOAT.ordinal()) {
-      return FLOAT;
-    } else if (ordinal == LONG.ordinal()) {
-      return LONG;
-    } else if (ordinal == DOUBLE.ordinal()) {
-      return DOUBLE;
-    } else if (ordinal == NULL.ordinal()) {
-      return NULL;
-    } else if (ordinal == DECIMAL.ordinal()) {
-      return DECIMAL;
-    } else if (ordinal == ARRAY.ordinal()) {
-      return ARRAY;
-    } else if (ordinal == STRUCT.ordinal()) {
-      return STRUCT;
-    } else if (ordinal == MAP.ordinal()) {
-      return MAP;
-    } else if (ordinal == BYTE.ordinal()) {
-      return BYTE;
-    } else if (ordinal == BYTE_ARRAY.ordinal()) {
-      return BYTE_ARRAY;
-    } else if (ordinal == SHORT_INT.ordinal()) {
-      return SHORT_INT;
-    } else {
-      throw new RuntimeException("create DataType with invalid ordinal: " + ordinal);
-    }
+  public int getId() {
+    return id;
   }
 
-  public static char convertType(DataType type) {
-    switch (type) {
-      case BYTE:
-      case SHORT:
-      case SHORT_INT:
-      case INT:
-      case LONG:
-        return BIG_INT_MEASURE_CHAR;
-      case DOUBLE:
-        return DOUBLE_MEASURE_CHAR;
-      case DECIMAL:
-        return BIG_DECIMAL_MEASURE_CHAR;
-      case STRING:
-        return STRING_CHAR;
-      case TIMESTAMP:
-        return TIMESTAMP_CHAR;
-      case DATE:
-        return DATE_CHAR;
-      case BYTE_ARRAY:
-        return BYTE_ARRAY_CHAR;
-      default:
-        throw new RuntimeException("Unexpected type: " + type);
+  public boolean isComplexType() {
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  public static final char DOUBLE_MEASURE_CHAR = 'n';
+  public static final char STRING_CHAR = 's';
+  public static final char TIMESTAMP_CHAR = 't';
+  public static final char DATE_CHAR = 'x';
+  public static final char BYTE_ARRAY_CHAR = 'y';
+  public static final char BYTE_VALUE_MEASURE_CHAR = 'c';
+  public static final char BIG_DECIMAL_MEASURE_CHAR = 'b';
+  public static final char BIG_INT_MEASURE_CHAR = 'd';
+
+  public static char convertType(DataType dataType) {
+    if (dataType == DataTypes.BYTE ||
+        dataType == DataTypes.SHORT ||
+        dataType == DataTypes.SHORT_INT ||
+        dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG) {
+      return BIG_INT_MEASURE_CHAR;
+    } else if (dataType == DataTypes.DOUBLE) {
+      return DOUBLE_MEASURE_CHAR;
+    } else if (dataType == DataTypes.DECIMAL) {
+      return BIG_DECIMAL_MEASURE_CHAR;
+    } else if (dataType == DataTypes.STRING) {
+      return STRING_CHAR;
+    } else if (dataType == DataTypes.TIMESTAMP) {
+      return TIMESTAMP_CHAR;
+    } else if (dataType == DataTypes.DATE) {
+      return DATE_CHAR;
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
+      return BYTE_ARRAY_CHAR;
+    } else {
+      throw new RuntimeException("Unexpected type: " + dataType);
     }
   }
 
   public static DataType getDataType(char type) {
     switch (type) {
       case BIG_INT_MEASURE_CHAR:
-        return DataType.LONG;
+        return DataTypes.LONG;
       case DOUBLE_MEASURE_CHAR:
-        return DataType.DOUBLE;
+        return DataTypes.DOUBLE;
       case BIG_DECIMAL_MEASURE_CHAR:
-        return DataType.DECIMAL;
+        return DataTypes.DECIMAL;
       case 'l':
-        return DataType.LEGACY_LONG;
+        return DataTypes.LEGACY_LONG;
       default:
         throw new RuntimeException("Unexpected type: " + type);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java
new file mode 100644
index 0000000..178f06a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.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.metadata.datatype;
+
+/**
+ * Holds all singleton object for all data type used in carbon
+ */
+public class DataTypes {
+
+  // singleton for each data type
+  public static final DataType STRING = StringType.STRING;
+  public static final DataType DATE = DateType.DATE;
+  public static final DataType TIMESTAMP = TimestampType.TIMESTAMP;
+  public static final DataType BOOLEAN = BooleanType.BOOLEAN;
+  public static final DataType SHORT = ShortType.SHORT;
+  public static final DataType INT = IntType.INT;
+  public static final DataType FLOAT = FloatType.FLOAT;
+  public static final DataType LONG = LongType.LONG;
+  public static final DataType DOUBLE = DoubleType.DOUBLE;
+  public static final DataType NULL = NullType.NULL;
+  public static final DataType BYTE = ByteType.BYTE;
+
+  // internal use only, for variable length data type
+  public static final DataType BYTE_ARRAY = ByteArrayType.BYTE_ARRAY;
+
+  // internal use only, for value compression from integer/long to 3 bytes value
+  public static final DataType SHORT_INT = ShortIntType.SHORT_INT;
+
+  // Only for internal use for backward compatability. It is only used for V1 version
+  public static final DataType LEGACY_LONG = LegacyLongType.LEGACY_LONG;
+
+  public static final DataType DECIMAL = DecimalType.DECIMAL;
+  public static final DataType ARRAY = ArrayType.ARRAY;
+  public static final DataType STRUCT = StructType.STRUCT;
+  public static final DataType MAP = MapType.MAP;
+
+  // these IDs are used within this package only
+  static final int STRING_TYPE_ID = 0;
+  static final int DATE_TYPE_ID = 1;
+  static final int TIMESTAMP_TYPE_ID = 2;
+  static final int BOOLEAN_TYPE_ID = 3;
+  static final int SHORT_TYPE_ID = 4;
+  static final int INT_TYPE_ID = 5;
+  static final int FLOAT_TYPE_ID = 6;
+  static final int LONG_TYPE_ID = 7;
+  static final int DOUBLE_TYPE_ID = 8;
+  static final int NULL_TYPE_ID = 9;
+  static final int BYTE_TYPE_ID = 10;
+  static final int BYTE_ARRAY_TYPE_ID = 11;
+  static final int SHORT_INT_TYPE_ID = 12;
+  static final int LEGACY_LONG_TYPE_ID = 13;
+  static final int DECIMAL_TYPE_ID = 20;
+  static final int ARRAY_TYPE_ID = 21;
+  static final int STRUCT_TYPE_ID = 22;
+  static final int MAP_TYPE_ID = 23;
+
+  /**
+   * create a DataType instance from uniqueId of the DataType
+   */
+  public static DataType valueOf(int id) {
+    if (id == STRING.getId()) {
+      return STRING;
+    } else if (id == DATE.getId()) {
+      return DATE;
+    } else if (id == TIMESTAMP.getId()) {
+      return TIMESTAMP;
+    } else if (id == BOOLEAN.getId()) {
+      return BOOLEAN;
+    } else if (id == BYTE.getId()) {
+      return BYTE;
+    } else if (id == SHORT.getId()) {
+      return SHORT;
+    } else if (id == SHORT_INT.getId()) {
+      return SHORT_INT;
+    } else if (id == INT.getId()) {
+      return INT;
+    } else if (id == LONG.getId()) {
+      return LONG;
+    } else if (id == LEGACY_LONG.getId()) {
+      return LEGACY_LONG;
+    } else if (id == FLOAT.getId()) {
+      return FLOAT;
+    } else if (id == DOUBLE.getId()) {
+      return DOUBLE;
+    } else if (id == NULL.getId()) {
+      return NULL;
+    } else if (id == DECIMAL.getId()) {
+      return DECIMAL;
+    } else if (id == ARRAY.getId()) {
+      return ARRAY;
+    } else if (id == STRUCT.getId()) {
+      return STRUCT;
+    } else if (id == MAP.getId()) {
+      return MAP;
+    } else if (id == BYTE_ARRAY.getId()) {
+      return BYTE_ARRAY;
+    } else {
+      throw new RuntimeException("create DataType with invalid id: " + id);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
new file mode 100644
index 0000000..2b930c3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class DateType extends DataType {
+
+  public static final DataType DATE = new DateType(DataTypes.DATE_TYPE_ID, 1, "DATE", -1);
+
+  private DateType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.DATE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
new file mode 100644
index 0000000..0c78e50
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
@@ -0,0 +1,33 @@
+/*
+ * 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.datatype;
+
+public class DecimalType extends DataType {
+
+  public static final DataType DECIMAL =
+      new DecimalType(DataTypes.DECIMAL_TYPE_ID, 8, "DECIMAL", -1);
+
+  private DecimalType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.DECIMAL;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
new file mode 100644
index 0000000..59a4cc2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class DoubleType extends DataType {
+
+  public static final DataType DOUBLE = new DoubleType(DataTypes.DOUBLE_TYPE_ID, 6, "DOUBLE", 8);
+
+  private DoubleType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.DOUBLE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
new file mode 100644
index 0000000..1c0bfb6
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
@@ -0,0 +1,31 @@
+/*
+ * 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.datatype;
+
+public class FloatType extends DataType {
+  public static final DataType FLOAT = new FloatType(DataTypes.FLOAT_TYPE_ID, 4, "FLOAT", 4);
+
+  private FloatType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.FLOAT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
new file mode 100644
index 0000000..4324d9e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class IntType extends DataType {
+
+  public static final DataType INT = new IntType(DataTypes.INT_TYPE_ID, 3, "INT", 4);
+
+  private IntType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.INT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LegacyLongType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LegacyLongType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LegacyLongType.java
new file mode 100644
index 0000000..610f2cc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LegacyLongType.java
@@ -0,0 +1,33 @@
+/*
+ * 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.datatype;
+
+class LegacyLongType extends DataType {
+
+  static final DataType LEGACY_LONG =
+      new LegacyLongType(DataTypes.LEGACY_LONG_TYPE_ID, 15, "LEGACYBIGINT", 8);
+
+  private LegacyLongType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.LEGACY_LONG;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
new file mode 100644
index 0000000..3d05fd9
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class LongType extends DataType {
+
+  public static final DataType LONG = new LongType(DataTypes.LONG_TYPE_ID, 5, "LONG", 8);
+
+  private LongType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.LONG;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
new file mode 100644
index 0000000..a360243
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
@@ -0,0 +1,37 @@
+/*
+ * 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.datatype;
+
+public class MapType extends DataType {
+
+  public static final DataType MAP = new MapType(DataTypes.MAP_TYPE_ID, 11, "MAP", -1);
+
+  private MapType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.MAP;
+  }
+
+  @Override
+  public boolean isComplexType() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
new file mode 100644
index 0000000..43ffe41
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class NullType extends DataType {
+
+  public static final DataType NULL = new NullType(DataTypes.NULL_TYPE_ID, 7, "NULL", 1);
+
+  private NullType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.NULL;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
new file mode 100644
index 0000000..5d57748
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
@@ -0,0 +1,38 @@
+/*
+ * 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.datatype;
+
+public class ShortIntType extends DataType {
+
+  public static final DataType SHORT_INT =
+      new ShortIntType(DataTypes.SHORT_INT_TYPE_ID, 14, "SHORT_INT", 3);
+
+  private ShortIntType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  @Override
+  public int getSizeBits() {
+    throw new UnsupportedOperationException("Should not call this from SHORT_INT type");
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.SHORT_INT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
new file mode 100644
index 0000000..0085227
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class ShortType extends DataType {
+
+  public static final DataType SHORT = new ShortType(DataTypes.SHORT_TYPE_ID, 2, "SHORT", 2);
+
+  private ShortType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.SHORT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
new file mode 100644
index 0000000..5c95480
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
@@ -0,0 +1,31 @@
+/*
+ * 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.datatype;
+
+public class StringType extends DataType {
+  public static final DataType STRING = new StringType(DataTypes.STRING_TYPE_ID, 0, "STRING", -1);
+
+  private StringType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.STRING;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
new file mode 100644
index 0000000..081f952
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
@@ -0,0 +1,37 @@
+/*
+ * 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.datatype;
+
+public class StructType extends DataType {
+
+  public static final DataType STRUCT = new StructType(DataTypes.STRUCT_TYPE_ID, 10, "STRUCT", -1);
+
+  private StructType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return DataTypes.STRUCT;
+  }
+
+  @Override
+  public boolean isComplexType() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
new file mode 100644
index 0000000..287c457
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.datatype;
+
+public class TimestampType extends DataType {
+  public static final TimestampType TIMESTAMP =
+      new TimestampType(DataTypes.TIMESTAMP_TYPE_ID, 2, "TIMESTAMP", -1);
+
+  private TimestampType(int id, int precedenceOrder, String name, int sizeInBytes) {
+    super(id, precedenceOrder, name, sizeInBytes);
+  }
+
+  // this function is needed to ensure singleton pattern while supporting java serialization
+  private Object readResolve() {
+    return TIMESTAMP;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 20f3224..a6738a3 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -124,8 +124,6 @@ public class CarbonTable implements Serializable {
    */
   private int numberOfNoDictSortColumns;
 
-  private int lastDimensionColumnOrdinal;
-
   private CarbonTable() {
     this.tableDimensionsMap = new HashMap<String, List<CarbonDimension>>();
     this.tableImplicitDimensionsMap = new HashMap<String, List<CarbonDimension>>();
@@ -258,7 +256,6 @@ public class CarbonTable implements Serializable {
                  columnSchema.getSchemaOrdinal()));
       }
     }
-    lastDimensionColumnOrdinal = dimensionOrdinal;
     fillVisibleDimensions(tableSchema.getTableName());
     fillVisibleMeasures(tableSchema.getTableName());
     addImplicitDimension(dimensionOrdinal, implicitDimensions);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
index 7cca993..13e7d49 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
@@ -128,7 +128,7 @@ public class CarbonColumn implements Serializable {
    * false.
    */
   public Boolean isComplex() {
-    return columnSchema.isComplex();
+    return columnSchema.getDataType().isComplexType();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonImplicitDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonImplicitDimension.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonImplicitDimension.java
index 0fb3860..9e0cc22 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonImplicitDimension.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonImplicitDimension.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.UUID;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 
 /**
@@ -114,7 +115,7 @@ public class CarbonImplicitDimension extends CarbonDimension {
    * @return the dataType
    */
   @Override public DataType getDataType() {
-    return DataType.STRING;
+    return DataTypes.STRING;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
index 3680d53..bd246a4 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.WritableUtil;
@@ -349,18 +350,6 @@ public class ColumnSchema implements Serializable, Writable {
   }
 
   /**
-   * @return if DataType is ARRAY or STRUCT, this method return true, else
-   * false.
-   */
-  public Boolean isComplex() {
-    if (DataType.ARRAY.equals(this.getDataType()) || DataType.STRUCT.equals(this.getDataType())) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  /**
    * @param columnProperties
    */
   public void setColumnProperties(Map<String, String> columnProperties) {
@@ -421,7 +410,7 @@ public class ColumnSchema implements Serializable, Writable {
 
   @Override
   public void write(DataOutput out) throws IOException {
-    out.writeShort(dataType.ordinal());
+    out.writeShort(dataType.getId());
     out.writeUTF(columnName);
     out.writeUTF(columnUniqueId);
     out.writeUTF(columnReferenceId);
@@ -454,16 +443,16 @@ public class ColumnSchema implements Serializable, Writable {
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    int ordinal = in.readShort();
-    this.dataType = DataType.valueOf(ordinal);
+    int id = in.readShort();
+    this.dataType = DataTypes.valueOf(id);
     this.columnName = in.readUTF();
     this.columnUniqueId = in.readUTF();
     this.columnReferenceId = in.readUTF();
     int encodingListSize = in.readShort();
     this.encodingList = new ArrayList<>(encodingListSize);
     for (int i = 0; i < encodingListSize; i++) {
-      ordinal = in.readShort();
-      encodingList.add(Encoding.valueOf(ordinal));
+      id = in.readShort();
+      encodingList.add(Encoding.valueOf(id));
     }
     this.isDimensionColumn = in.readBoolean();
     this.scale = in.readInt();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
index 84995b9..1fa42dc 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
@@ -23,6 +23,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
@@ -77,7 +78,7 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
       } else {
         // if not then get the default value and use that value in aggregation
         Object defaultValue = measureInfo.getDefaultValues()[i];
-        if (null != defaultValue && measureInfo.getMeasureDataTypes()[i] == DataType.DECIMAL) {
+        if (null != defaultValue && measureInfo.getMeasureDataTypes()[i] == DataTypes.DECIMAL) {
           // convert data type as per the computing engine
           defaultValue = DataTypeUtil.getDataTypeConverter().convertToDecimal(defaultValue);
         }
@@ -89,24 +90,23 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
   protected Object getMeasureData(ColumnPage dataChunk, int index,
       CarbonMeasure carbonMeasure) {
     if (!dataChunk.getNullBits().get(index)) {
-      switch (carbonMeasure.getDataType()) {
-        case SHORT:
-          return (short)dataChunk.getLong(index);
-        case INT:
-          return (int)dataChunk.getLong(index);
-        case LONG:
-          return dataChunk.getLong(index);
-        case DECIMAL:
-          BigDecimal bigDecimalMsrValue =
-              dataChunk.getDecimal(index);
-          if (null != bigDecimalMsrValue && carbonMeasure.getScale() > bigDecimalMsrValue.scale()) {
-            bigDecimalMsrValue =
-                bigDecimalMsrValue.setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
-          }
-          // convert data type as per the computing engine
-          return DataTypeUtil.getDataTypeConverter().convertToDecimal(bigDecimalMsrValue);
-        default:
-          return dataChunk.getDouble(index);
+      DataType dataType = carbonMeasure.getDataType();
+      if (dataType == DataTypes.SHORT) {
+        return (short) dataChunk.getLong(index);
+      } else if (dataType == DataTypes.INT) {
+        return (int) dataChunk.getLong(index);
+      } else if (dataType == DataTypes.LONG) {
+        return dataChunk.getLong(index);
+      } else if (dataType == DataTypes.DECIMAL) {
+        BigDecimal bigDecimalMsrValue = dataChunk.getDecimal(index);
+        if (null != bigDecimalMsrValue && carbonMeasure.getScale() > bigDecimalMsrValue.scale()) {
+          bigDecimalMsrValue =
+              bigDecimalMsrValue.setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
+        }
+        // convert data type as per the computing engine
+        return DataTypeUtil.getDataTypeConverter().convertToDecimal(bigDecimalMsrValue);
+      } else {
+        return dataChunk.getDouble(index);
       }
     }
     return null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
index 5522ff2..cf6e6d6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
@@ -128,10 +128,10 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
           row[order[i]] = DataTypeUtil.getDataBasedOnDataType(
               scannedResult.getBlockletId() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult
                   .getCurrentPageCounter() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult
-                  .getCurrentRowId(), DataType.STRING);
+                  .getCurrentRowId(), DataTypes.STRING);
         } else {
           row[order[i]] =
-              DataTypeUtil.getDataBasedOnDataType(scannedResult.getBlockletId(), DataType.STRING);
+              DataTypeUtil.getDataBasedOnDataType(scannedResult.getBlockletId(), DataTypes.STRING);
         }
       } else {
         row[order[i]] = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
index 45275a5..b08a1d4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
@@ -239,7 +239,7 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
           Object defaultValue = dimensionInfo.getDefaultValues()[i];
           if (null != defaultValue) {
             newColumnDefaultValue = ((UTF8String) defaultValue).getBytes();
-          } else if (actualQueryDimensions[i].getDimension().getDataType() == DataType.STRING) {
+          } else if (actualQueryDimensions[i].getDimension().getDataType() == DataTypes.STRING) {
             newColumnDefaultValue =
                 UTF8String.fromString(CarbonCommonConstants.MEMBER_DEFAULT_VAL).getBytes();
           } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
index 65b9a17..cbde2e1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
@@ -20,6 +20,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -60,8 +61,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
         // add a dummy column vector result collector object
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         columnVectorInfo.dimension = queryDimensions[i];
-        if (queryDimensions[i].getDimension().getDataType().equals(DataType.TIMESTAMP)
-            || queryDimensions[i].getDimension().getDataType().equals(DataType.DATE)) {
+        if (queryDimensions[i].getDimension().getDataType().equals(DataTypes.TIMESTAMP)
+            || queryDimensions[i].getDimension().getDataType().equals(DataTypes.DATE)) {
           columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
               .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
         }
@@ -177,7 +178,7 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
   private void fillDirectDictionaryData(CarbonColumnVector vector,
       ColumnVectorInfo columnVectorInfo, Object defaultValue) {
     if (null != defaultValue) {
-      if (columnVectorInfo.directDictionaryGenerator.getReturnType().equals(DataType.INT)) {
+      if (columnVectorInfo.directDictionaryGenerator.getReturnType().equals(DataTypes.INT)) {
         vector.putInts(columnVectorInfo.vectorOffset, columnVectorInfo.size, (int) defaultValue);
       } else {
         vector.putLongs(columnVectorInfo.vectorOffset, columnVectorInfo.size, (long) defaultValue);
@@ -197,18 +198,14 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
   private void fillNoDictionaryData(CarbonColumnVector vector, ColumnVectorInfo columnVectorInfo,
       Object defaultValue) {
     if (null != defaultValue) {
-      switch (columnVectorInfo.dimension.getDimension().getDataType()) {
-        case INT:
-          vector.putInts(columnVectorInfo.vectorOffset, columnVectorInfo.size, (int) defaultValue);
-          break;
-        case LONG:
-        case TIMESTAMP:
-          vector
-              .putLongs(columnVectorInfo.vectorOffset, columnVectorInfo.size, (long) defaultValue);
-          break;
-        default:
-          vector.putBytes(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-              ((UTF8String) defaultValue).getBytes());
+      DataType dataType = columnVectorInfo.dimension.getDimension().getDataType();
+      if (dataType == DataTypes.INT) {
+        vector.putInts(columnVectorInfo.vectorOffset, columnVectorInfo.size, (int) defaultValue);
+      } else if (dataType == DataTypes.LONG || dataType == DataTypes.TIMESTAMP) {
+        vector.putLongs(columnVectorInfo.vectorOffset, columnVectorInfo.size, (long) defaultValue);
+      } else {
+        vector.putBytes(columnVectorInfo.vectorOffset, columnVectorInfo.size,
+            ((UTF8String) defaultValue).getBytes());
       }
     } else {
       vector.putNulls(columnVectorInfo.vectorOffset, columnVectorInfo.size);
@@ -229,26 +226,22 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
         if (null == defaultValue) {
           vector.putNulls(columnVectorInfo.vectorOffset, columnVectorInfo.size);
         } else {
-          switch (measureInfo.getMeasureDataTypes()[i]) {
-            case SHORT:
-              vector.putShorts(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-                  (short) defaultValue);
-              break;
-            case INT:
-              vector.putInts(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-                  (int) defaultValue);
-              break;
-            case LONG:
-              vector.putLongs(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-                  (long) defaultValue);
-              break;
-            case DECIMAL:
-              vector.putDecimals(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-                  (Decimal) defaultValue, measure.getPrecision());
-              break;
-            default:
-              vector.putDoubles(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-                  (double) defaultValue);
+          DataType dataType = measureInfo.getMeasureDataTypes()[i];
+          if (dataType == DataTypes.SHORT) {
+            vector.putShorts(columnVectorInfo.vectorOffset, columnVectorInfo.size,
+                (short) defaultValue);
+          } else if (dataType == DataTypes.INT) {
+            vector
+                .putInts(columnVectorInfo.vectorOffset, columnVectorInfo.size, (int) defaultValue);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLongs(columnVectorInfo.vectorOffset, columnVectorInfo.size,
+                (long) defaultValue);
+          } else if (dataType == DataTypes.DECIMAL) {
+            vector.putDecimals(columnVectorInfo.vectorOffset, columnVectorInfo.size,
+                (Decimal) defaultValue, measure.getPrecision());
+          } else {
+            vector.putDoubles(columnVectorInfo.vectorOffset, columnVectorInfo.size,
+                (double) defaultValue);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
index 39227a3..1fcccca 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
@@ -25,8 +25,9 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 
-import org.apache.spark.sql.catalyst.util.*;
-import org.apache.spark.sql.types.*;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.types.ArrayType;
+import org.apache.spark.sql.types.DataType;
 
 public class ArrayQueryType extends ComplexQueryType implements GenericQueryType {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
index 56c265b..978a652 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
@@ -96,21 +96,20 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
   }
 
   @Override public DataType getSchemaType() {
-    switch (dataType) {
-      case INT:
-        return IntegerType$.MODULE$;
-      case DOUBLE:
-        return DoubleType$.MODULE$;
-      case LONG:
-        return LongType$.MODULE$;
-      case BOOLEAN:
-        return BooleanType$.MODULE$;
-      case TIMESTAMP:
-        return TimestampType$.MODULE$;
-      case DATE:
-        return DateType$.MODULE$;
-      default:
-        return IntegerType$.MODULE$;
+    if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT) {
+      return IntegerType$.MODULE$;
+    } else if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE) {
+      return DoubleType$.MODULE$;
+    } else if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
+      return LongType$.MODULE$;
+    } else if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN) {
+      return BooleanType$.MODULE$;
+    } else if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.TIMESTAMP) {
+      return TimestampType$.MODULE$;
+    } else if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DATE) {
+      return DateType$.MODULE$;
+    } else {
+      return IntegerType$.MODULE$;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index 25c827b..b2c1a2b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -49,6 +49,7 @@ import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -478,7 +479,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     List<Integer> parentBlockIndexList = new ArrayList<Integer>();
     for (QueryDimension queryDimension : queryDimensions) {
       if (CarbonUtil.hasDataType(queryDimension.getDimension().getDataType(),
-          new DataType[] { DataType.ARRAY, DataType.STRUCT, DataType.MAP })) {
+          new DataType[] { DataTypes.ARRAY, DataTypes.STRUCT, DataTypes.MAP })) {
         parentBlockIndexList.add(queryDimension.getDimension().getOrdinal());
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index ee09793..59387b3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -43,6 +43,7 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -778,10 +779,18 @@ public class QueryUtil {
       CarbonDimension dimension, Map<Integer, GenericQueryType> complexTypeMap,
       int[] eachComplexColumnValueSize, Map<String, Dictionary> columnIdToDictionaryMap) {
     int parentBlockIndex = dimensionToBlockIndexMap.get(dimension.getOrdinal());
-    GenericQueryType parentQueryType = dimension.getDataType().equals(DataType.ARRAY) ?
-        new ArrayQueryType(dimension.getColName(), dimension.getColName(), parentBlockIndex) :
-        new StructQueryType(dimension.getColName(), dimension.getColName(),
-            dimensionToBlockIndexMap.get(dimension.getOrdinal()));
+    GenericQueryType parentQueryType;
+    if (dimension.getDataType() == DataTypes.ARRAY) {
+      parentQueryType =
+          new ArrayQueryType(dimension.getColName(), dimension.getColName(), parentBlockIndex);
+    } else if (dimension.getDataType() == DataTypes.STRUCT) {
+      parentQueryType =
+          new StructQueryType(dimension.getColName(), dimension.getColName(),
+              dimensionToBlockIndexMap.get(dimension.getOrdinal()));
+    } else {
+      throw new UnsupportedOperationException(dimension.getDataType().getName() +
+          " is not supported");
+    }
     complexTypeMap.put(dimension.getOrdinal(), parentQueryType);
     fillChildrenDetails(eachComplexColumnValueSize, columnIdToDictionaryMap, parentBlockIndex,
             dimension, parentQueryType);
@@ -791,29 +800,27 @@ public class QueryUtil {
       Map<String, Dictionary> columnIdToDictionaryMap, int parentBlockIndex,
       CarbonDimension dimension, GenericQueryType parentQueryType) {
     for (int i = 0; i < dimension.getNumberOfChild(); i++) {
-      switch (dimension.getListOfChildDimensions().get(i).getDataType()) {
-        case ARRAY:
-          parentQueryType.addChildren(
-              new ArrayQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                  dimension.getColName(), ++parentBlockIndex));
-          break;
-        case STRUCT:
-          parentQueryType.addChildren(
-              new StructQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                  dimension.getColName(), ++parentBlockIndex));
-          break;
-        default:
-          boolean isDirectDictionary = CarbonUtil
-              .hasEncoding(dimension.getListOfChildDimensions().get(i).getEncoder(),
-                  Encoding.DIRECT_DICTIONARY);
-          parentQueryType.addChildren(
-              new PrimitiveQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                  dimension.getColName(), ++parentBlockIndex,
-                  dimension.getListOfChildDimensions().get(i).getDataType(),
-                  eachComplexColumnValueSize[dimension.getListOfChildDimensions().get(i)
-                      .getComplexTypeOrdinal()], columnIdToDictionaryMap
-                  .get(dimension.getListOfChildDimensions().get(i).getColumnId()),
-                  isDirectDictionary));
+      DataType dataType = dimension.getListOfChildDimensions().get(i).getDataType();
+      if (dataType == DataTypes.ARRAY) {
+        parentQueryType.addChildren(
+            new ArrayQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
+                dimension.getColName(), ++parentBlockIndex));
+      } else if (dataType == DataTypes.STRUCT) {
+        parentQueryType.addChildren(
+            new StructQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
+                dimension.getColName(), ++parentBlockIndex));
+      } else {
+        boolean isDirectDictionary = CarbonUtil
+            .hasEncoding(dimension.getListOfChildDimensions().get(i).getEncoder(),
+                Encoding.DIRECT_DICTIONARY);
+        parentQueryType.addChildren(
+            new PrimitiveQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
+                dimension.getColName(), ++parentBlockIndex,
+                dimension.getListOfChildDimensions().get(i).getDataType(),
+                eachComplexColumnValueSize[dimension.getListOfChildDimensions().get(i)
+                    .getComplexTypeOrdinal()], columnIdToDictionaryMap
+                .get(dimension.getListOfChildDimensions().get(i).getColumnId()),
+                isDirectDictionary));
       }
       if (dimension.getListOfChildDimensions().get(i).getNumberOfChild() > 0) {
         parentBlockIndex = fillChildrenDetails(eachComplexColumnValueSize, columnIdToDictionaryMap,


[10/10] carbondata git commit: [CARBONDATA-1539] Change data type from enum to class

Posted by ra...@apache.org.
[CARBONDATA-1539] Change data type from enum to class

DataType should be java class instead of enum, to hold more information for decimal and complex type. And it is needed to decouple carbon core and spark.

No logic is changed in this PR.

This closes #1402


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/956833e5
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/956833e5
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/956833e5

Branch: refs/heads/master
Commit: 956833e5525742616d8a0e6b885132acd7291b76
Parents: 75e0bd4
Author: Jacky Li <ja...@qq.com>
Authored: Sun Oct 8 22:06:01 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Oct 13 14:56:00 2017 +0530

----------------------------------------------------------------------
 .../cache/dictionary/ColumnDictionaryInfo.java  |  60 +-
 .../core/constants/CarbonCommonConstants.java   |   4 +-
 .../core/datamap/dev/DataMapFactory.java        |   1 -
 .../carbondata/core/datastore/TableSpec.java    |   7 +-
 .../core/datastore/block/SegmentProperties.java |   5 +-
 .../impl/FixedLengthDimensionDataChunk.java     |  40 +-
 .../core/datastore/impl/FileFactory.java        |   6 +-
 .../core/datastore/page/ColumnPage.java         | 349 +++++-----
 .../core/datastore/page/LazyColumnPage.java     |  89 +--
 .../datastore/page/SafeDecimalColumnPage.java   |  33 +-
 .../datastore/page/SafeFixLengthColumnPage.java |  62 +-
 .../datastore/page/UnsafeDecimalColumnPage.java |  81 ++-
 .../page/UnsafeFixLengthColumnPage.java         | 123 ++--
 .../datastore/page/VarLengthColumnPageBase.java |  15 +-
 .../page/encoding/ColumnPageEncoder.java        |   6 +-
 .../page/encoding/ColumnPageEncoderMeta.java    | 243 ++++---
 .../page/encoding/DefaultEncodingFactory.java   | 122 ++--
 .../page/encoding/EncodingFactory.java          | 109 ++--
 .../adaptive/AdaptiveDeltaIntegralCodec.java    | 192 +++---
 .../adaptive/AdaptiveFloatingCodec.java         |  66 +-
 .../adaptive/AdaptiveIntegralCodec.java         | 140 ++--
 .../encoding/compress/DirectCompressCodec.java  |   3 +-
 .../datastore/page/encoding/rle/RLECodec.java   | 113 ++--
 .../page/statistics/LVStringStatsCollector.java |   3 +-
 .../statistics/PrimitivePageStatsCollector.java | 307 ++++-----
 .../core/indexstore/UnsafeMemoryDMStore.java    |  87 ++-
 .../blockletindex/BlockletDataMap.java          | 107 ++--
 .../core/indexstore/row/DataMapRowImpl.java     |  18 +-
 .../DirectDictionaryKeyGeneratorFactory.java    |  14 +-
 .../DateDirectDictionaryGenerator.java          |   9 +-
 .../TimeStampDirectDictionaryGenerator.java     |  15 +-
 .../core/memory/HeapMemoryAllocator.java        |   2 +-
 .../ThriftWrapperSchemaConverterImpl.java       |  70 +-
 .../core/metadata/datatype/ArrayType.java       |  37 ++
 .../core/metadata/datatype/BooleanType.java     |  33 +
 .../core/metadata/datatype/ByteArrayType.java   |  33 +
 .../core/metadata/datatype/ByteType.java        |  32 +
 .../core/metadata/datatype/DataType.java        | 156 ++---
 .../core/metadata/datatype/DataTypes.java       | 117 ++++
 .../core/metadata/datatype/DateType.java        |  32 +
 .../core/metadata/datatype/DecimalType.java     |  33 +
 .../core/metadata/datatype/DoubleType.java      |  32 +
 .../core/metadata/datatype/FloatType.java       |  31 +
 .../core/metadata/datatype/IntType.java         |  32 +
 .../core/metadata/datatype/LegacyLongType.java  |  33 +
 .../core/metadata/datatype/LongType.java        |  32 +
 .../core/metadata/datatype/MapType.java         |  37 ++
 .../core/metadata/datatype/NullType.java        |  32 +
 .../core/metadata/datatype/ShortIntType.java    |  38 ++
 .../core/metadata/datatype/ShortType.java       |  32 +
 .../core/metadata/datatype/StringType.java      |  31 +
 .../core/metadata/datatype/StructType.java      |  37 ++
 .../core/metadata/datatype/TimestampType.java   |  32 +
 .../core/metadata/schema/table/CarbonTable.java |   3 -
 .../schema/table/column/CarbonColumn.java       |   2 +-
 .../table/column/CarbonImplicitDimension.java   |   3 +-
 .../schema/table/column/ColumnSchema.java       |  23 +-
 .../impl/AbstractScannedResultCollector.java    |  38 +-
 .../impl/DictionaryBasedResultCollector.java    |   6 +-
 .../RestructureBasedRawResultCollector.java     |   4 +-
 .../RestructureBasedVectorResultCollector.java  |  63 +-
 .../core/scan/complextypes/ArrayQueryType.java  |   5 +-
 .../scan/complextypes/PrimitiveQueryType.java   |  29 +-
 .../executor/impl/AbstractQueryExecutor.java    |   3 +-
 .../core/scan/executor/util/QueryUtil.java      |  61 +-
 .../scan/executor/util/RestructureUtil.java     | 122 ++--
 .../core/scan/expression/ExpressionResult.java  | 572 ++++++++---------
 .../expression/RangeExpressionEvaluator.java    |   5 +-
 .../conditional/EqualToExpression.java          |  51 +-
 .../GreaterThanEqualToExpression.java           |  49 +-
 .../conditional/GreaterThanExpression.java      |  49 +-
 .../expression/conditional/InExpression.java    |  49 +-
 .../conditional/LessThanEqualToExpression.java  |  49 +-
 .../conditional/LessThanExpression.java         |  49 +-
 .../conditional/NotEqualsExpression.java        |  51 +-
 .../expression/conditional/NotInExpression.java |  53 +-
 .../scan/expression/logical/AndExpression.java  |  14 +-
 .../expression/logical/FalseExpression.java     |   4 +-
 .../scan/expression/logical/OrExpression.java   |  15 +-
 .../expression/logical/RangeExpression.java     |  14 +-
 .../scan/expression/logical/TrueExpression.java |   4 +-
 .../scan/filter/FilterExpressionProcessor.java  |  14 +-
 .../carbondata/core/scan/filter/FilterUtil.java | 151 ++---
 .../executer/ExcludeFilterExecuterImpl.java     |  22 +-
 .../executer/IncludeFilterExecuterImpl.java     |  22 +-
 .../executer/RangeValueFilterExecuterImpl.java  |   4 +-
 .../executer/RestructureEvaluatorImpl.java      |   4 +-
 .../executer/RowLevelFilterExecuterImpl.java    |  96 ++-
 ...velRangeLessThanEqualFilterExecuterImpl.java |   3 +-
 .../RowLevelRangeLessThanFiterExecuterImpl.java |   3 +-
 .../resolver/ConditionalFilterResolverImpl.java |  10 +-
 .../RowLevelRangeFilterResolverImpl.java        |   4 +-
 .../visitor/CustomTypeDictionaryVisitor.java    |   3 +-
 .../core/scan/partition/PartitionUtil.java      |  76 ++-
 .../core/scan/partition/RangePartitioner.java   |   3 +-
 .../vector/MeasureDataVectorProcessor.java      |  22 +-
 .../util/AbstractDataFileFooterConverter.java   |  23 +-
 .../core/util/CarbonMetadataUtil.java           |  48 +-
 .../apache/carbondata/core/util/CarbonUtil.java | 109 ++--
 .../carbondata/core/util/DataTypeUtil.java      | 641 +++++++++----------
 .../core/util/comparator/Comparator.java        |  54 +-
 .../sortindex/CarbonDictionarySortModel.java    | 109 ++--
 .../dictionary/AbstractDictionaryCacheTest.java |   5 +-
 .../dictionary/ColumnDictionaryInfoTest.java    |  45 +-
 .../DictionaryCacheLoaderImplTest.java          |   6 +-
 .../DictionaryColumnUniqueIdentifierTest.java   |  26 +-
 .../cache/dictionary/ForwardDictionaryTest.java |   3 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   3 +-
 .../core/carbon/ColumnIdentifierTest.java       |  11 +-
 .../datastore/block/SegmentPropertiesTest.java  |  28 +-
 .../block/SegmentPropertiesTestUtil.java        |  25 +-
 .../datastore/page/encoding/RLECodecSuite.java  |  11 +-
 ...ctDictionaryKeyGeneratorFactoryUnitTest.java |   5 +-
 .../core/metadata/CarbonMetadataTest.java       |   5 +-
 .../ThriftWrapperSchemaConverterImplTest.java   |  33 +-
 .../metadata/schema/table/CarbonTableTest.java  |   5 +-
 .../table/CarbonTableWithComplexTypesTest.java  |  16 +-
 .../impl/RawBasedResultCollectorTest.java       |   2 +-
 .../complextypes/PrimitiveQueryTypeTest.java    |  21 +-
 .../scan/executor/util/RestructureUtilTest.java |  18 +-
 .../scan/expression/ColumnExpressionTest.java   |   5 +-
 .../scan/expression/ExpressionResultTest.java   | 141 ++--
 .../scan/expression/LiteralExpressionTest.java  |   8 +-
 .../conditional/EqualToExpressionUnitTest.java  |  31 +-
 .../GreaterThanEqualToExpressionUnitTest.java   |  45 +-
 .../GreaterThanExpressionUnitTest.java          |  41 +-
 .../conditional/InExpressionUnitTest.java       |  47 +-
 .../LessThanEqualToExpressionUnitTest.java      |  41 +-
 .../conditional/LessThanExpressionUnitTest.java |  41 +-
 .../conditional/ListExpressionUnitTest.java     |   5 +-
 .../NotEqualsExpressionUnitTest.java            |  49 +-
 .../conditional/NotInExpressionUnitTest.java    |  47 +-
 .../expression/logical/AndExpressionTest.java   |   9 +-
 .../expression/logical/FalseExpressionTest.java |   6 +-
 .../expression/logical/OrExpressionTest.java    |   9 +-
 .../core/scan/filter/FilterUtilTest.java        |  59 +-
 .../core/util/CarbonMetadataUtilTest.java       |   2 +-
 .../carbondata/core/util/CarbonUtilTest.java    |  15 +-
 .../core/util/DataFileFooterConverterTest.java  |  37 +-
 .../carbondata/core/util/DataTypeUtilTest.java  |  70 +-
 .../core/util/RangeFilterProcessorTest.java     |  97 +--
 .../CarbonDictionarySortInfoPreparatorTest.java |   5 +-
 .../CarbonDictionarySortModelTest.java          |  73 +--
 .../hadoop/ft/CarbonInputFormat_FT.java         |   5 +-
 .../hadoop/ft/CarbonInputMapperTest.java        |   5 +-
 .../test/util/ObjectSerializationUtilTest.java  |   9 +-
 .../hadoop/test/util/StoreCreator.java          |  15 +-
 .../hive/CarbonDictionaryDecodeReadSupport.java | 109 +---
 .../carbondata/presto/CarbonTypeUtil.java       |  32 +-
 .../presto/CarbonVectorizedRecordReader.java    |  31 +-
 .../carbondata/presto/CarbondataMetadata.java   |  51 +-
 .../carbondata/presto/PrestoFilterUtil.java     |  25 +-
 .../CarbonDictionaryDecodeReadSupport.scala     |   7 +-
 .../presto/util/CarbonDataStoreCreator.scala    |  22 +-
 .../testsuite/datamap/DataMapWriterSuite.scala  |   4 +-
 .../partition/TestDDLForPartitionTable.scala    |   9 +-
 ...ForPartitionTableWithDefaultProperties.scala |  10 +-
 .../apache/carbondata/spark/rdd/CarbonRDD.scala |   4 +-
 .../spark/rdd/CarbonScanPartitionRDD.scala      |   4 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |  46 +-
 .../carbondata/spark/util/CommonUtil.scala      |   6 +-
 .../spark/util/DataTypeConverterUtil.scala      |  92 +--
 .../spark/util/GlobalDictionaryUtil.scala       |   8 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   6 +-
 .../command/carbonTableSchemaCommon.scala       |  14 +-
 .../readsupport/SparkRowReadSupportImpl.java    |  28 +-
 .../spark/CarbonDataFrameWriter.scala           |   6 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   4 +-
 .../carbondata/spark/util/CarbonSparkUtil.scala |   2 +-
 .../spark/sql/CarbonDatasourceRelation.scala    |  18 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  23 +-
 .../org/apache/spark/sql/CarbonSparkUtil.scala  |   2 +-
 .../spark/sql/optimizer/CarbonFilters.scala     |  16 +-
 .../VectorizedCarbonRecordReader.java           |  31 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   4 +-
 .../carbondata/spark/util/CarbonSparkUtil.scala |   2 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |   2 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  56 +-
 .../apache/spark/sql/hive/CarbonRelation.scala  |  36 +-
 .../spark/sql/optimizer/CarbonFilters.scala     |  24 +-
 .../processing/datatypes/StructDataType.java    |   4 +-
 .../converter/impl/FieldEncoderFactory.java     |  62 +-
 .../impl/NonDictionaryFieldConverterImpl.java   |   5 +-
 .../loading/parser/CarbonParserFactory.java     |  49 +-
 .../partition/impl/HashPartitionerImpl.java     |  23 +-
 .../sort/unsafe/UnsafeCarbonRowPage.java        | 187 +++---
 .../holder/UnsafeSortTempFileChunkHolder.java   |  40 +-
 .../merger/UnsafeIntermediateFileMerger.java    |  46 +-
 .../merger/CompactionResultSortProcessor.java   |  16 +-
 .../sort/sortdata/IntermediateFileMerger.java   |  36 +-
 .../processing/sort/sortdata/SortDataRows.java  |  38 +-
 .../sort/sortdata/SortTempFileChunkHolder.java  |  38 +-
 .../store/CarbonFactDataHandlerColumnar.java    |   3 +-
 .../carbondata/processing/store/TablePage.java  |  11 +-
 .../util/CarbonDataProcessorUtil.java           |  28 +-
 .../carbondata/processing/StoreCreator.java     |  15 +-
 196 files changed, 4448 insertions(+), 4256 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
index bc748c6..223812e 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
@@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
@@ -193,7 +194,7 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
       int surrogateKey = sortedSurrogates.get(mid);
       byte[] dictionaryValue = getDictionaryBytesFromSurrogate(surrogateKey);
       int cmp = -1;
-      if (this.getDataType() != DataType.STRING) {
+      if (this.getDataType() != DataTypes.STRING) {
         cmp = compareFilterKeyWithDictionaryKey(
             new String(dictionaryValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
             filterKey, this.getDataType());
@@ -240,7 +241,7 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
         //fortify fix
         if (null == dictionaryValue) {
           cmp = -1;
-        } else if (this.getDataType() != DataType.STRING) {
+        } else if (this.getDataType() != DataTypes.STRING) {
           cmp = compareFilterKeyWithDictionaryKey(
               new String(dictionaryValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
               filterKey, this.getDataType());
@@ -270,35 +271,32 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
   private int compareFilterKeyWithDictionaryKey(String dictionaryVal, String memberVal,
       DataType dataType) {
     try {
-      switch (dataType) {
-        case SHORT:
-          return Short.compare((Short.parseShort(dictionaryVal)), (Short.parseShort(memberVal)));
-        case INT:
-          return Integer.compare((Integer.parseInt(dictionaryVal)), (Integer.parseInt(memberVal)));
-        case DOUBLE:
-          return DataTypeUtil
-              .compareDoubleWithNan((Double.parseDouble(dictionaryVal)),
-                  (Double.parseDouble(memberVal)));
-        case LONG:
-          return Long.compare((Long.parseLong(dictionaryVal)), (Long.parseLong(memberVal)));
-        case BOOLEAN:
-          return Boolean
-              .compare((Boolean.parseBoolean(dictionaryVal)), (Boolean.parseBoolean(memberVal)));
-        case DATE:
-        case TIMESTAMP:
-          String format = CarbonUtil.getFormatFromProperty(dataType);
-          SimpleDateFormat parser = new SimpleDateFormat(format);
-          Date dateToStr;
-          Date dictionaryDate;
-          dateToStr = parser.parse(memberVal);
-          dictionaryDate = parser.parse(dictionaryVal);
-          return dictionaryDate.compareTo(dateToStr);
-        case DECIMAL:
-          java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
-          java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
-          return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
-        default:
-          return -1;
+      if (dataType == DataTypes.SHORT) {
+        return Short.compare((Short.parseShort(dictionaryVal)), (Short.parseShort(memberVal)));
+      } else if (dataType == DataTypes.INT) {
+        return Integer.compare((Integer.parseInt(dictionaryVal)), (Integer.parseInt(memberVal)));
+      } else if (dataType == DataTypes.DOUBLE) {
+        return DataTypeUtil.compareDoubleWithNan(
+            (Double.parseDouble(dictionaryVal)), (Double.parseDouble(memberVal)));
+      } else if (dataType == DataTypes.LONG) {
+        return Long.compare((Long.parseLong(dictionaryVal)), (Long.parseLong(memberVal)));
+      } else if (dataType == DataTypes.BOOLEAN) {
+        return Boolean.compare(
+            (Boolean.parseBoolean(dictionaryVal)), (Boolean.parseBoolean(memberVal)));
+      } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
+        String format = CarbonUtil.getFormatFromProperty(dataType);
+        SimpleDateFormat parser = new SimpleDateFormat(format);
+        Date dateToStr;
+        Date dictionaryDate;
+        dateToStr = parser.parse(memberVal);
+        dictionaryDate = parser.parse(dictionaryVal);
+        return dictionaryDate.compareTo(dateToStr);
+      } else if (dataType == DataTypes.DECIMAL) {
+        java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
+        java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
+        return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
+      } else {
+        return -1;
       }
     } catch (Exception e) {
       //In all data types excluding String data type the null member will be the highest

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 51b92cc..0510d7f 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -632,8 +632,8 @@ public final class CarbonCommonConstants {
   public static final String SHORT = "Short";
   public static final String NUMERIC = "Numeric";
   public static final String TIMESTAMP = "Timestamp";
-  public static final String ARRAY = "ARRAY";
-  public static final String STRUCT = "STRUCT";
+  public static final String ARRAY = "array";
+  public static final String STRUCT = "struct";
   public static final String FROM = "from";
   /**
    * FACT_UPDATE_EXTENSION.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index 9796a77..62cf813 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -21,7 +21,6 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.events.ChangeEvent;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
index b0b62db..9f29e27 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -62,7 +63,7 @@ public class TableSpec {
         if (dimension.isComplex()) {
           DimensionSpec spec = new DimensionSpec(ColumnType.COMPLEX, dimension);
           dimensionSpec[dimIndex++] = spec;
-        } else if (dimension.getDataType() == DataType.TIMESTAMP && !dimension
+        } else if (dimension.getDataType() == DataTypes.TIMESTAMP && !dimension
             .isDirectDictionaryEncoding()) {
           DimensionSpec spec = new DimensionSpec(ColumnType.PLAIN_VALUE, dimension);
           dimensionSpec[dimIndex++] = spec;
@@ -167,7 +168,7 @@ public class TableSpec {
     @Override
     public void write(DataOutput out) throws IOException {
       out.writeUTF(fieldName);
-      out.writeByte(schemaDataType.ordinal());
+      out.writeByte(schemaDataType.getId());
       out.writeByte(columnType.ordinal());
       out.writeInt(scale);
       out.writeInt(precision);
@@ -176,7 +177,7 @@ public class TableSpec {
     @Override
     public void readFields(DataInput in) throws IOException {
       this.fieldName = in.readUTF();
-      this.schemaDataType = DataType.valueOf(in.readByte());
+      this.schemaDataType = DataTypes.valueOf(in.readByte());
       this.columnType = ColumnType.valueOf(in.readByte());
       this.scale = in.readInt();
       this.precision = in.readInt();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
index a742a5b..2ac200f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.apache.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
 import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -378,7 +379,7 @@ public class SegmentProperties {
         // as complex type will be stored at last so once complex type started all the dimension
         // will be added to complex type
         else if (isComplexDimensionStarted || CarbonUtil.hasDataType(columnSchema.getDataType(),
-            new DataType[] { DataType.ARRAY, DataType.STRUCT })) {
+            new DataType[] { DataTypes.ARRAY, DataTypes.STRUCT })) {
           cardinalityIndexForComplexDimensionColumn.add(tableOrdinal);
           carbonDimension =
               new CarbonDimension(columnSchema, dimensonOrdinal++, -1, -1, ++complexTypeOrdinal);
@@ -668,7 +669,7 @@ public class SegmentProperties {
     int k = eachDimColumnValueSize.length + eachComplexDimColumnValueSize.length;
     for (int i = 0; i < measures.size(); i++) {
       DataType dataType = measures.get(i).getDataType();
-      if (dataType.equals(DataType.DECIMAL)) {
+      if (dataType.equals(DataTypes.DECIMAL)) {
         dimensionValueSize[k++] = -1;
       } else {
         dimensionValueSize[k++] = 8;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
index 97fd226..6629d31 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.chunk.impl;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
 import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
@@ -103,17 +105,14 @@ public class FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk {
         if (valueFromSurrogate == null) {
           vector.putNull(vectorOffset++);
         } else {
-          switch (columnVectorInfo.directDictionaryGenerator.getReturnType()) {
-            case INT:
-              vector.putInt(vectorOffset++, (int) valueFromSurrogate);
-              break;
-            case LONG:
-              vector.putLong(vectorOffset++, (long) valueFromSurrogate);
-              break;
-            default:
-              throw new IllegalArgumentException(
-                  "unsupported data type: " + columnVectorInfo.directDictionaryGenerator
-                      .getReturnType());
+          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
+          if (dataType == DataTypes.INT) {
+            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
+          } else {
+            throw new IllegalArgumentException("unsupported data type: " +
+                columnVectorInfo.directDictionaryGenerator.getReturnType());
           }
         }
       }
@@ -147,17 +146,14 @@ public class FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk {
         if (valueFromSurrogate == null) {
           vector.putNull(vectorOffset++);
         } else {
-          switch (columnVectorInfo.directDictionaryGenerator.getReturnType()) {
-            case INT:
-              vector.putInt(vectorOffset++, (int) valueFromSurrogate);
-              break;
-            case LONG:
-              vector.putLong(vectorOffset++, (long) valueFromSurrogate);
-              break;
-            default:
-              throw new IllegalArgumentException(
-                  "unsupported data type: " + columnVectorInfo.directDictionaryGenerator
-                      .getReturnType());
+          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
+          if (dataType == DataTypes.INT) {
+            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
+          } else {
+            throw new IllegalArgumentException("unsupported data type: " +
+                columnVectorInfo.directDictionaryGenerator.getReturnType());
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index 3191200..97f0b3f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -30,7 +30,11 @@ import java.util.zip.GZIPInputStream;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.filesystem.*;
+import org.apache.carbondata.core.datastore.filesystem.AlluxioCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.HDFSCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.ViewFSCarbonFile;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
index 6c534d6..012413b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
@@ -30,17 +30,18 @@ import org.apache.carbondata.core.datastore.page.statistics.ColumnPageStatsColle
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 
-import static org.apache.carbondata.core.metadata.datatype.DataType.BYTE;
-import static org.apache.carbondata.core.metadata.datatype.DataType.BYTE_ARRAY;
-import static org.apache.carbondata.core.metadata.datatype.DataType.DECIMAL;
-import static org.apache.carbondata.core.metadata.datatype.DataType.DOUBLE;
-import static org.apache.carbondata.core.metadata.datatype.DataType.FLOAT;
-import static org.apache.carbondata.core.metadata.datatype.DataType.INT;
-import static org.apache.carbondata.core.metadata.datatype.DataType.LONG;
-import static org.apache.carbondata.core.metadata.datatype.DataType.SHORT;
-import static org.apache.carbondata.core.metadata.datatype.DataType.SHORT_INT;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE_ARRAY;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.FLOAT;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.INT;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.LONG;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT_INT;
 
 public abstract class ColumnPage {
 
@@ -183,59 +184,42 @@ public abstract class ColumnPage {
       int pageSize) throws MemoryException {
     ColumnPage instance;
     if (unsafe) {
-      switch (dataType) {
-        case BYTE:
-        case SHORT:
-        case SHORT_INT:
-        case INT:
-        case LONG:
-        case FLOAT:
-        case DOUBLE:
-          instance = new UnsafeFixLengthColumnPage(columnSpec, dataType, pageSize);
-          break;
-        case DECIMAL:
-          instance = new UnsafeDecimalColumnPage(columnSpec, dataType, pageSize);
-          break;
-        case STRING:
-        case BYTE_ARRAY:
-          instance =
-              new UnsafeVarLengthColumnPage(columnSpec, dataType, pageSize);
-          break;
-        default:
-          throw new RuntimeException("Unsupported data dataType: " + dataType);
+      if (dataType == DataTypes.BYTE ||
+          dataType == DataTypes.SHORT ||
+          dataType == DataTypes.SHORT_INT ||
+          dataType == DataTypes.INT ||
+          dataType == DataTypes.LONG ||
+          dataType == DataTypes.FLOAT ||
+          dataType == DataTypes.DOUBLE) {
+        instance = new UnsafeFixLengthColumnPage(columnSpec, dataType, pageSize);
+      } else if (dataType == DataTypes.DECIMAL) {
+        instance = new UnsafeDecimalColumnPage(columnSpec, dataType, pageSize);
+      } else if (dataType == DataTypes.STRING || dataType == DataTypes.BYTE_ARRAY) {
+        instance = new UnsafeVarLengthColumnPage(columnSpec, dataType, pageSize);
+      } else {
+        throw new RuntimeException("Unsupported data dataType: " + dataType);
       }
     } else {
-      switch (dataType) {
-        case BYTE:
-          instance = newBytePage(columnSpec, new byte[pageSize]);
-          break;
-        case SHORT:
-          instance = newShortPage(columnSpec, new short[pageSize]);
-          break;
-        case SHORT_INT:
-          instance = newShortIntPage(columnSpec, new byte[pageSize * 3]);
-          break;
-        case INT:
-          instance = newIntPage(columnSpec, new int[pageSize]);
-          break;
-        case LONG:
-          instance = newLongPage(columnSpec, new long[pageSize]);
-          break;
-        case FLOAT:
-          instance = newFloatPage(columnSpec, new float[pageSize]);
-          break;
-        case DOUBLE:
-          instance = newDoublePage(columnSpec, new double[pageSize]);
-          break;
-        case DECIMAL:
-          instance = newDecimalPage(columnSpec, new byte[pageSize][]);
-          break;
-        case STRING:
-        case BYTE_ARRAY:
-          instance = new SafeVarLengthColumnPage(columnSpec, dataType, pageSize);
-          break;
-        default:
-          throw new RuntimeException("Unsupported data dataType: " + dataType);
+      if (dataType == DataTypes.BYTE) {
+        instance = newBytePage(columnSpec, new byte[pageSize]);
+      } else if (dataType == DataTypes.SHORT) {
+        instance = newShortPage(columnSpec, new short[pageSize]);
+      } else if (dataType == DataTypes.SHORT_INT) {
+        instance = newShortIntPage(columnSpec, new byte[pageSize * 3]);
+      } else if (dataType == DataTypes.INT) {
+        instance = newIntPage(columnSpec, new int[pageSize]);
+      } else if (dataType == DataTypes.LONG) {
+        instance = newLongPage(columnSpec, new long[pageSize]);
+      } else if (dataType == DataTypes.FLOAT) {
+        instance = newFloatPage(columnSpec, new float[pageSize]);
+      } else if (dataType == DataTypes.DOUBLE) {
+        instance = newDoublePage(columnSpec, new double[pageSize]);
+      } else if (dataType == DataTypes.DECIMAL) {
+        instance = newDecimalPage(columnSpec, new byte[pageSize][]);
+      } else if (dataType == DataTypes.STRING || dataType == DataTypes.BYTE_ARRAY) {
+        instance = new SafeVarLengthColumnPage(columnSpec, dataType, pageSize);
+      } else {
+        throw new RuntimeException("Unsupported data dataType: " + dataType);
       }
     }
     return instance;
@@ -360,38 +344,29 @@ public abstract class ColumnPage {
       nullBitSet.set(rowId);
       return;
     }
-    switch (dataType) {
-      case BYTE:
-        putByte(rowId, (byte) value);
-        statsCollector.update((byte) value);
-        break;
-      case SHORT:
-        putShort(rowId, (short) value);
-        statsCollector.update((short) value);
-        break;
-      case INT:
-        putInt(rowId, (int) value);
-        statsCollector.update((int) value);
-        break;
-      case LONG:
-        putLong(rowId, (long) value);
-        statsCollector.update((long) value);
-        break;
-      case DOUBLE:
-        putDouble(rowId, (double) value);
-        statsCollector.update((double) value);
-        break;
-      case DECIMAL:
-        putDecimal(rowId, (BigDecimal) value);
-        statsCollector.update((BigDecimal) value);
-        break;
-      case STRING:
-      case BYTE_ARRAY:
-        putBytes(rowId, (byte[]) value);
-        statsCollector.update((byte[]) value);
-        break;
-      default:
-        throw new RuntimeException("unsupported data type: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      putByte(rowId, (byte) value);
+      statsCollector.update((byte) value);
+    } else if (dataType == DataTypes.SHORT) {
+      putShort(rowId, (short) value);
+      statsCollector.update((short) value);
+    } else if (dataType == DataTypes.INT) {
+      putInt(rowId, (int) value);
+      statsCollector.update((int) value);
+    } else if (dataType == DataTypes.LONG) {
+      putLong(rowId, (long) value);
+      statsCollector.update((long) value);
+    } else if (dataType == DataTypes.DOUBLE) {
+      putDouble(rowId, (double) value);
+      statsCollector.update((double) value);
+    } else if (dataType == DataTypes.DECIMAL) {
+      putDecimal(rowId, (BigDecimal) value);
+      statsCollector.update((BigDecimal) value);
+    } else if (dataType == DataTypes.STRING || dataType == DataTypes.BYTE_ARRAY) {
+      putBytes(rowId, (byte[]) value);
+      statsCollector.update((byte[]) value);
+    } else {
+      throw new RuntimeException("unsupported data type: " + dataType);
     }
   }
 
@@ -445,27 +420,20 @@ public abstract class ColumnPage {
    * Set null at rowId
    */
   private void putNull(int rowId) {
-    switch (dataType) {
-      case BYTE:
-        putByte(rowId, (byte) 0);
-        break;
-      case SHORT:
-        putShort(rowId, (short) 0);
-        break;
-      case INT:
-        putInt(rowId, 0);
-        break;
-      case LONG:
-        putLong(rowId, 0L);
-        break;
-      case DOUBLE:
-        putDouble(rowId, 0.0);
-        break;
-      case DECIMAL:
-        putDecimal(rowId, BigDecimal.ZERO);
-        break;
-      default:
-        throw new IllegalArgumentException("unsupported data type: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      putByte(rowId, (byte) 0);
+    } else if (dataType == DataTypes.SHORT) {
+      putShort(rowId, (short) 0);
+    } else if (dataType == DataTypes.INT) {
+      putInt(rowId, 0);
+    } else if (dataType == DataTypes.LONG) {
+      putLong(rowId, 0L);
+    } else if (dataType == DataTypes.DOUBLE) {
+      putDouble(rowId, 0.0);
+    } else if (dataType == DataTypes.DECIMAL) {
+      putDecimal(rowId, BigDecimal.ZERO);
+    } else {
+      throw new IllegalArgumentException("unsupported data type: " + dataType);
     }
   }
 
@@ -573,27 +541,26 @@ public abstract class ColumnPage {
    * Compress page data using specified compressor
    */
   public byte[] compress(Compressor compressor) throws MemoryException, IOException {
-    switch (dataType) {
-      case BYTE:
-        return compressor.compressByte(getBytePage());
-      case SHORT:
-        return compressor.compressShort(getShortPage());
-      case SHORT_INT:
-        return compressor.compressByte(getShortIntPage());
-      case INT:
-        return compressor.compressInt(getIntPage());
-      case LONG:
-        return compressor.compressLong(getLongPage());
-      case FLOAT:
-        return compressor.compressFloat(getFloatPage());
-      case DOUBLE:
-        return compressor.compressDouble(getDoublePage());
-      case DECIMAL:
-        return compressor.compressByte(getDecimalPage());
-      case BYTE_ARRAY:
-        return compressor.compressByte(getLVFlattenedBytePage());
-      default:
-        throw new UnsupportedOperationException("unsupport compress column page: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      return compressor.compressByte(getBytePage());
+    } else if (dataType == DataTypes.SHORT) {
+      return compressor.compressShort(getShortPage());
+    } else if (dataType == DataTypes.SHORT_INT) {
+      return compressor.compressByte(getShortIntPage());
+    } else if (dataType == DataTypes.INT) {
+      return compressor.compressInt(getIntPage());
+    } else if (dataType == DataTypes.LONG) {
+      return compressor.compressLong(getLongPage());
+    } else if (dataType == DataTypes.FLOAT) {
+      return compressor.compressFloat(getFloatPage());
+    } else if (dataType == DataTypes.DOUBLE) {
+      return compressor.compressDouble(getDoublePage());
+    } else if (dataType == DataTypes.DECIMAL) {
+      return compressor.compressByte(getDecimalPage());
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
+      return compressor.compressByte(getLVFlattenedBytePage());
+    } else {
+      throw new UnsupportedOperationException("unsupport compress column page: " + dataType);
     }
   }
 
@@ -606,34 +573,34 @@ public abstract class ColumnPage {
       throws MemoryException {
     Compressor compressor = CompressorFactory.getInstance().getCompressor(meta.getCompressorName());
     TableSpec.ColumnSpec columnSpec = meta.getColumnSpec();
-    switch (meta.getStoreDataType()) {
-      case BYTE:
-        byte[] byteData = compressor.unCompressByte(compressedData, offset, length);
-        return newBytePage(columnSpec, byteData);
-      case SHORT:
-        short[] shortData = compressor.unCompressShort(compressedData, offset, length);
-        return newShortPage(columnSpec, shortData);
-      case SHORT_INT:
-        byte[] shortIntData = compressor.unCompressByte(compressedData, offset, length);
-        return newShortIntPage(columnSpec, shortIntData);
-      case INT:
-        int[] intData = compressor.unCompressInt(compressedData, offset, length);
-        return newIntPage(columnSpec, intData);
-      case LONG:
-        long[] longData = compressor.unCompressLong(compressedData, offset, length);
-        return newLongPage(columnSpec, longData);
-      case FLOAT:
-        float[] floatData = compressor.unCompressFloat(compressedData, offset, length);
-        return newFloatPage(columnSpec, floatData);
-      case DOUBLE:
-        double[] doubleData = compressor.unCompressDouble(compressedData, offset, length);
-        return newDoublePage(columnSpec, doubleData);
-      case BYTE_ARRAY:
-        byte[] lvVarBytes = compressor.unCompressByte(compressedData, offset, length);
-        return newLVBytesPage(columnSpec, lvVarBytes);
-      default:
-        throw new UnsupportedOperationException("unsupport uncompress column page: " +
-            meta.getStoreDataType());
+    DataType storeDataType = meta.getStoreDataType();
+    if (storeDataType == DataTypes.BYTE) {
+      byte[] byteData = compressor.unCompressByte(compressedData, offset, length);
+      return newBytePage(columnSpec, byteData);
+    } else if (storeDataType == DataTypes.SHORT) {
+      short[] shortData = compressor.unCompressShort(compressedData, offset, length);
+      return newShortPage(columnSpec, shortData);
+    } else if (storeDataType == DataTypes.SHORT_INT) {
+      byte[] shortIntData = compressor.unCompressByte(compressedData, offset, length);
+      return newShortIntPage(columnSpec, shortIntData);
+    } else if (storeDataType == DataTypes.INT) {
+      int[] intData = compressor.unCompressInt(compressedData, offset, length);
+      return newIntPage(columnSpec, intData);
+    } else if (storeDataType == DataTypes.LONG) {
+      long[] longData = compressor.unCompressLong(compressedData, offset, length);
+      return newLongPage(columnSpec, longData);
+    } else if (storeDataType == DataTypes.FLOAT) {
+      float[] floatData = compressor.unCompressFloat(compressedData, offset, length);
+      return newFloatPage(columnSpec, floatData);
+    } else if (storeDataType == DataTypes.DOUBLE) {
+      double[] doubleData = compressor.unCompressDouble(compressedData, offset, length);
+      return newDoublePage(columnSpec, doubleData);
+    } else if (storeDataType == DataTypes.BYTE_ARRAY) {
+      byte[] lvVarBytes = compressor.unCompressByte(compressedData, offset, length);
+      return newLVBytesPage(columnSpec, lvVarBytes);
+    } else {
+      throw new UnsupportedOperationException("unsupport uncompress column page: " +
+          meta.getStoreDataType());
     }
   }
 
@@ -645,35 +612,35 @@ public abstract class ColumnPage {
     Compressor compressor = CompressorFactory.getInstance().getCompressor(meta.getCompressorName());
     TableSpec.ColumnSpec columnSpec = meta.getColumnSpec();
     ColumnPage decimalPage = null;
-    switch (meta.getStoreDataType()) {
-      case BYTE:
-        byte[] byteData = compressor.unCompressByte(compressedData, offset, length);
-        decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), byteData.length);
-        decimalPage.setBytePage(byteData);
-        return decimalPage;
-      case SHORT:
-        short[] shortData = compressor.unCompressShort(compressedData, offset, length);
-        decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), shortData.length);
-        decimalPage.setShortPage(shortData);
-        return decimalPage;
-      case SHORT_INT:
-        byte[] shortIntData = compressor.unCompressByte(compressedData, offset, length);
-        decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), shortIntData.length);
-        decimalPage.setShortIntPage(shortIntData);
-        return decimalPage;
-      case INT:
-        int[] intData = compressor.unCompressInt(compressedData, offset, length);
-        decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), intData.length);
-        decimalPage.setIntPage(intData);
-        return decimalPage;
-      case LONG:
-        long[] longData = compressor.unCompressLong(compressedData, offset, length);
-        decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), longData.length);
-        decimalPage.setLongPage(longData);
-        return decimalPage;
-      default:
-        byte[] lvEncodedBytes = compressor.unCompressByte(compressedData, offset, length);
-        return newDecimalPage(columnSpec, lvEncodedBytes);
+    DataType storeDataType = meta.getStoreDataType();
+    if (storeDataType == DataTypes.BYTE) {
+      byte[] byteData = compressor.unCompressByte(compressedData, offset, length);
+      decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), byteData.length);
+      decimalPage.setBytePage(byteData);
+      return decimalPage;
+    } else if (storeDataType == DataTypes.SHORT) {
+      short[] shortData = compressor.unCompressShort(compressedData, offset, length);
+      decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), shortData.length);
+      decimalPage.setShortPage(shortData);
+      return decimalPage;
+    } else if (storeDataType == DataTypes.SHORT_INT) {
+      byte[] shortIntData = compressor.unCompressByte(compressedData, offset, length);
+      decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), shortIntData.length);
+      decimalPage.setShortIntPage(shortIntData);
+      return decimalPage;
+    }  else if (storeDataType == DataTypes.INT) {
+      int[] intData = compressor.unCompressInt(compressedData, offset, length);
+      decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), intData.length);
+      decimalPage.setIntPage(intData);
+      return decimalPage;
+    } else if (storeDataType == DataTypes.LONG) {
+      long[] longData = compressor.unCompressLong(compressedData, offset, length);
+      decimalPage = createDecimalPage(columnSpec, meta.getStoreDataType(), longData.length);
+      decimalPage.setLongPage(longData);
+      return decimalPage;
+    } else {
+      byte[] lvEncodedBytes = compressor.unCompressByte(compressedData, offset, length);
+      return newDecimalPage(columnSpec, lvEncodedBytes);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
index 4bdb252..cebb3c0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.page;
 
 import java.math.BigDecimal;
 
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.DecimalConverterFactory;
 
 /**
@@ -50,41 +52,41 @@ public class LazyColumnPage extends ColumnPage {
 
   @Override
   public long getLong(int rowId) {
-    switch (columnPage.getDataType()) {
-      case BYTE:
-        return converter.decodeLong(columnPage.getByte(rowId));
-      case SHORT:
-        return converter.decodeLong(columnPage.getShort(rowId));
-      case SHORT_INT:
-        return converter.decodeLong(columnPage.getShortInt(rowId));
-      case INT:
-        return converter.decodeLong(columnPage.getInt(rowId));
-      case LONG:
-        return columnPage.getLong(rowId);
-      default:
-        throw new RuntimeException("internal error: " + this.toString());
+    DataType dataType = columnPage.getDataType();
+    if (dataType == DataTypes.BYTE) {
+      return converter.decodeLong(columnPage.getByte(rowId));
+    } else if (dataType == DataTypes.SHORT) {
+      return converter.decodeLong(columnPage.getShort(rowId));
+    } else if (dataType == DataTypes.SHORT_INT) {
+      return converter.decodeLong(columnPage.getShortInt(rowId));
+    } else if (dataType == DataTypes.INT) {
+      return converter.decodeLong(columnPage.getInt(rowId));
+    } else if (dataType == DataTypes.LONG) {
+      return columnPage.getLong(rowId);
+    } else {
+      throw new RuntimeException("internal error: " + this.toString());
     }
   }
 
   @Override
   public double getDouble(int rowId) {
-    switch (columnPage.getDataType()) {
-      case BYTE:
-        return converter.decodeDouble(columnPage.getByte(rowId));
-      case SHORT:
-        return converter.decodeDouble(columnPage.getShort(rowId));
-      case SHORT_INT:
-        return converter.decodeDouble(columnPage.getShortInt(rowId));
-      case INT:
-        return converter.decodeDouble(columnPage.getInt(rowId));
-      case LONG:
-        return converter.decodeDouble(columnPage.getLong(rowId));
-      case FLOAT:
-        return converter.decodeDouble(columnPage.getFloat(rowId));
-      case DOUBLE:
-        return columnPage.getDouble(rowId);
-      default:
-        throw new RuntimeException("internal error: " + this.toString());
+    DataType dataType = columnPage.getDataType();
+    if (dataType == DataTypes.BYTE) {
+      return converter.decodeDouble(columnPage.getByte(rowId));
+    } else if (dataType == DataTypes.SHORT) {
+      return converter.decodeDouble(columnPage.getShort(rowId));
+    } else if (dataType == DataTypes.SHORT_INT) {
+      return converter.decodeDouble(columnPage.getShortInt(rowId));
+    } else if (dataType == DataTypes.INT) {
+      return converter.decodeDouble(columnPage.getInt(rowId));
+    } else if (dataType == DataTypes.LONG) {
+      return converter.decodeDouble(columnPage.getLong(rowId));
+    } else if (dataType == DataTypes.FLOAT) {
+      return converter.decodeDouble(columnPage.getFloat(rowId));
+    } else if (dataType == DataTypes.DOUBLE) {
+      return columnPage.getDouble(rowId);
+    } else {
+      throw new RuntimeException("internal error: " + this.toString());
     }
   }
 
@@ -97,20 +99,19 @@ public class LazyColumnPage extends ColumnPage {
   public BigDecimal getDecimal(int rowId) {
     DecimalConverterFactory.DecimalConverter decimalConverter =
         ((DecimalColumnPage) columnPage).getDecimalConverter();
-    switch (columnPage.getDataType()) {
-      case BYTE:
-        return decimalConverter.getDecimal(converter.decodeLong(columnPage.getByte(rowId)));
-      case SHORT:
-        return decimalConverter.getDecimal(converter.decodeLong(columnPage.getShort(rowId)));
-      case SHORT_INT:
-        return decimalConverter.getDecimal(converter.decodeLong(columnPage.getShortInt(rowId)));
-      case INT:
-        return decimalConverter.getDecimal(converter.decodeLong(columnPage.getInt(rowId)));
-      case LONG:
-      case DECIMAL:
-        return columnPage.getDecimal(rowId);
-      default:
-        throw new RuntimeException("internal error: " + this.toString());
+    DataType dataType = columnPage.getDataType();
+    if (dataType == DataTypes.BYTE) {
+      return decimalConverter.getDecimal(converter.decodeLong(columnPage.getByte(rowId)));
+    } else if (dataType == DataTypes.SHORT) {
+      return decimalConverter.getDecimal(converter.decodeLong(columnPage.getShort(rowId)));
+    } else if (dataType == DataTypes.SHORT_INT) {
+      return decimalConverter.getDecimal(converter.decodeLong(columnPage.getShortInt(rowId)));
+    } else if (dataType == DataTypes.INT) {
+      return decimalConverter.getDecimal(converter.decodeLong(columnPage.getInt(rowId)));
+    } else if (dataType == DataTypes.LONG || dataType == DataTypes.DECIMAL) {
+      return columnPage.getDecimal(rowId);
+    } else {
+      throw new RuntimeException("internal error: " + this.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeDecimalColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeDecimalColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeDecimalColumnPage.java
index 01d3d87..cb2d3bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeDecimalColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeDecimalColumnPage.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 
 /**
@@ -173,25 +174,19 @@ public class SafeDecimalColumnPage extends DecimalColumnPage {
   @Override
   public BigDecimal getDecimal(int rowId) {
     long value;
-    switch (dataType) {
-      case BYTE:
-        value = getByte(rowId);
-        break;
-      case SHORT:
-        value = getShort(rowId);
-        break;
-      case SHORT_INT:
-        value = getShortInt(rowId);
-        break;
-      case INT:
-        value = getInt(rowId);
-        break;
-      case LONG:
-        value = getLong(rowId);
-        break;
-      default:
-        byte[] bytes = byteArrayData[rowId];
-        return decimalConverter.getDecimal(bytes);
+    if (dataType == DataTypes.BYTE) {
+      value = getByte(rowId);
+    } else if (dataType == DataTypes.SHORT) {
+      value = getShort(rowId);
+    } else if (dataType == DataTypes.SHORT_INT) {
+      value = getShortInt(rowId);
+    } else if (dataType == DataTypes.INT) {
+      value = getInt(rowId);
+    } else if (dataType == DataTypes.LONG) {
+      value = getLong(rowId);
+    } else {
+      byte[] bytes = byteArrayData[rowId];
+      return decimalConverter.getDecimal(bytes);
     }
     return decimalConverter.getDecimal(value);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeFixLengthColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeFixLengthColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeFixLengthColumnPage.java
index 33d306d..5f848c0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeFixLengthColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/SafeFixLengthColumnPage.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 
 /**
@@ -318,40 +319,33 @@ public class SafeFixLengthColumnPage extends ColumnPage {
    */
   @Override
   public void convertValue(ColumnPageValueConverter codec) {
-    switch (dataType) {
-      case BYTE:
-        for (int i = 0; i < pageSize; i++) {
-          codec.encode(i, byteData[i]);
-        }
-        break;
-      case SHORT:
-        for (int i = 0; i < pageSize; i++) {
-          codec.encode(i, shortData[i]);
-        }
-        break;
-      case INT:
-        for (int i = 0; i < pageSize; i++) {
-          codec.encode(i, intData[i]);
-        }
-        break;
-      case LONG:
-        for (int i = 0; i < pageSize; i++) {
-          codec.encode(i, longData[i]);
-        }
-        break;
-      case FLOAT:
-        for (int i = 0; i < pageSize; i++) {
-          codec.encode(i, floatData[i]);
-        }
-        break;
-      case DOUBLE:
-        for (int i = 0; i < pageSize; i++) {
-          codec.encode(i, doubleData[i]);
-        }
-        break;
-      default:
-        throw new UnsupportedOperationException("not support value conversion on " +
-            dataType + " page");
+    if (dataType == DataTypes.BYTE) {
+      for (int i = 0; i < pageSize; i++) {
+        codec.encode(i, byteData[i]);
+      }
+    } else if (dataType == DataTypes.SHORT) {
+      for (int i = 0; i < pageSize; i++) {
+        codec.encode(i, shortData[i]);
+      }
+    } else if (dataType == DataTypes.INT) {
+      for (int i = 0; i < pageSize; i++) {
+        codec.encode(i, intData[i]);
+      }
+    } else if (dataType == DataTypes.LONG) {
+      for (int i = 0; i < pageSize; i++) {
+        codec.encode(i, longData[i]);
+      }
+    } else if (dataType == DataTypes.FLOAT) {
+      for (int i = 0; i < pageSize; i++) {
+        codec.encode(i, floatData[i]);
+      }
+    } else if (dataType == DataTypes.DOUBLE) {
+      for (int i = 0; i < pageSize; i++) {
+        codec.encode(i, doubleData[i]);
+      }
+    } else {
+      throw new UnsupportedOperationException("not support value conversion on " +
+          dataType + " page");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
index 45fa7d8..b4f33b8 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 
 /**
@@ -46,29 +47,25 @@ public class UnsafeDecimalColumnPage extends DecimalColumnPage {
   }
 
   private void initMemory() throws MemoryException {
-    switch (dataType) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-        int size = pageSize << dataType.getSizeBits();
-        memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
-        baseAddress = memoryBlock.getBaseObject();
-        baseOffset = memoryBlock.getBaseOffset();
-        break;
-      case SHORT_INT:
-        size = pageSize * 3;
-        memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
-        baseAddress = memoryBlock.getBaseObject();
-        baseOffset = memoryBlock.getBaseOffset();
-        break;
-      case DECIMAL:
-        memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, (long) (capacity));
-        baseAddress = memoryBlock.getBaseObject();
-        baseOffset = memoryBlock.getBaseOffset();
-        break;
-      default:
-        throw new UnsupportedOperationException("invalid data type: " + dataType);
+    if (dataType == DataTypes.BYTE ||
+        dataType == DataTypes.SHORT ||
+        dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG) {
+      int size = pageSize << dataType.getSizeBits();
+      memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
+      baseAddress = memoryBlock.getBaseObject();
+      baseOffset = memoryBlock.getBaseOffset();
+    } else if (dataType == DataTypes.SHORT_INT) {
+      int size = pageSize * 3;
+      memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
+      baseAddress = memoryBlock.getBaseObject();
+      baseOffset = memoryBlock.getBaseOffset();
+    } else if (dataType == DataTypes.DECIMAL) {
+      memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, (long) (capacity));
+      baseAddress = memoryBlock.getBaseObject();
+      baseOffset = memoryBlock.getBaseOffset();
+    } else {
+      throw new UnsupportedOperationException("invalid data type: " + dataType);
     }
   }
 
@@ -236,28 +233,22 @@ public class UnsafeDecimalColumnPage extends DecimalColumnPage {
   @Override
   public BigDecimal getDecimal(int rowId) {
     long value;
-    switch (dataType) {
-      case BYTE:
-        value = getByte(rowId);
-        break;
-      case SHORT:
-        value = getShort(rowId);
-        break;
-      case SHORT_INT:
-        value = getShortInt(rowId);
-        break;
-      case INT:
-        value = getInt(rowId);
-        break;
-      case LONG:
-        value = getLong(rowId);
-        break;
-      default:
-        int length = rowOffset[rowId + 1] - rowOffset[rowId];
-        byte[] bytes = new byte[length];
-        CarbonUnsafe.getUnsafe().copyMemory(baseAddress, baseOffset + rowOffset[rowId], bytes,
-            CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
-        return decimalConverter.getDecimal(bytes);
+    if (dataType == DataTypes.BYTE) {
+      value = getByte(rowId);
+    } else if (dataType == DataTypes.SHORT) {
+      value = getShort(rowId);
+    } else if (dataType == DataTypes.SHORT_INT) {
+      value = getShortInt(rowId);
+    } else if (dataType == DataTypes.INT) {
+      value = getInt(rowId);
+    } else if (dataType == DataTypes.LONG) {
+      value = getLong(rowId);
+    } else {
+      int length = rowOffset[rowId + 1] - rowOffset[rowId];
+      byte[] bytes = new byte[length];
+      CarbonUnsafe.getUnsafe().copyMemory(baseAddress, baseOffset + rowOffset[rowId], bytes,
+          CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
+      return decimalConverter.getDecimal(bytes);
     }
     return decimalConverter.getDecimal(value);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
index 7b55889..055c27a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
@@ -27,10 +27,10 @@ import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
-import static org.apache.carbondata.core.metadata.datatype.DataType.BYTE;
 
 // This extension uses unsafe memory to store page data, for fix length data type only (byte,
 // short, integer, long, float, double)
@@ -46,37 +46,33 @@ public class UnsafeFixLengthColumnPage extends ColumnPage {
 
   private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
-  private static final int byteBits = BYTE.getSizeBits();
-  private static final int shortBits = DataType.SHORT.getSizeBits();
-  private static final int intBits = DataType.INT.getSizeBits();
-  private static final int longBits = DataType.LONG.getSizeBits();
-  private static final int floatBits = DataType.FLOAT.getSizeBits();
-  private static final int doubleBits = DataType.DOUBLE.getSizeBits();
+  private static final int byteBits = DataTypes.BYTE.getSizeBits();
+  private static final int shortBits = DataTypes.SHORT.getSizeBits();
+  private static final int intBits = DataTypes.INT.getSizeBits();
+  private static final int longBits = DataTypes.LONG.getSizeBits();
+  private static final int floatBits = DataTypes.FLOAT.getSizeBits();
+  private static final int doubleBits = DataTypes.DOUBLE.getSizeBits();
 
   UnsafeFixLengthColumnPage(TableSpec.ColumnSpec columnSpec, DataType dataType, int pageSize)
       throws MemoryException {
     super(columnSpec, dataType, pageSize);
-    switch (dataType) {
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-      case FLOAT:
-      case DOUBLE:
-        int size = pageSize << dataType.getSizeBits();
-        memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
-        baseAddress = memoryBlock.getBaseObject();
-        baseOffset = memoryBlock.getBaseOffset();
-        break;
-      case SHORT_INT:
-        size = pageSize * 3;
-        memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
-        baseAddress = memoryBlock.getBaseObject();
-        baseOffset = memoryBlock.getBaseOffset();
-        break;
-      case DECIMAL:
-      case STRING:
-        throw new UnsupportedOperationException("invalid data type: " + dataType);
+    if (dataType == DataTypes.BYTE ||
+        dataType == DataTypes.SHORT ||
+        dataType == DataTypes.INT ||
+        dataType == DataTypes.LONG ||
+        dataType == DataTypes.FLOAT ||
+        dataType == DataTypes.DOUBLE) {
+      int size = pageSize << dataType.getSizeBits();
+      memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
+      baseAddress = memoryBlock.getBaseObject();
+      baseOffset = memoryBlock.getBaseOffset();
+    } else if (dataType == DataTypes.SHORT_INT) {
+      int size = pageSize * 3;
+      memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
+      baseAddress = memoryBlock.getBaseObject();
+      baseOffset = memoryBlock.getBaseOffset();
+    } else if (dataType == DataTypes.DECIMAL || dataType == DataTypes.STRING) {
+      throw new UnsupportedOperationException("invalid data type: " + dataType);
     }
   }
 
@@ -330,45 +326,38 @@ public class UnsafeFixLengthColumnPage extends ColumnPage {
   @Override
   public void convertValue(ColumnPageValueConverter codec) {
     int pageSize = getPageSize();
-    switch (dataType) {
-      case BYTE:
-        for (int i = 0; i < pageSize; i++) {
-          long offset = i << byteBits;
-          codec.encode(i, CarbonUnsafe.getUnsafe().getByte(baseAddress, baseOffset + offset));
-        }
-        break;
-      case SHORT:
-        for (int i = 0; i < pageSize; i++) {
-          long offset = i << shortBits;
-          codec.encode(i, CarbonUnsafe.getUnsafe().getShort(baseAddress, baseOffset + offset));
-        }
-        break;
-      case INT:
-        for (int i = 0; i < pageSize; i++) {
-          long offset = i << intBits;
-          codec.encode(i, CarbonUnsafe.getUnsafe().getInt(baseAddress, baseOffset + offset));
-        }
-        break;
-      case LONG:
-        for (int i = 0; i < pageSize; i++) {
-          long offset = i << longBits;
-          codec.encode(i, CarbonUnsafe.getUnsafe().getLong(baseAddress, baseOffset + offset));
-        }
-        break;
-      case FLOAT:
-        for (int i = 0; i < pageSize; i++) {
-          long offset = i << floatBits;
-          codec.encode(i, CarbonUnsafe.getUnsafe().getFloat(baseAddress, baseOffset + offset));
-        }
-        break;
-      case DOUBLE:
-        for (int i = 0; i < pageSize; i++) {
-          long offset = i << doubleBits;
-          codec.encode(i, CarbonUnsafe.getUnsafe().getDouble(baseAddress, baseOffset + offset));
-        }
-        break;
-      default:
-        throw new UnsupportedOperationException("invalid data type: " + dataType);
+    if (dataType == DataTypes.BYTE) {
+      for (int i = 0; i < pageSize; i++) {
+        long offset = i << byteBits;
+        codec.encode(i, CarbonUnsafe.getUnsafe().getByte(baseAddress, baseOffset + offset));
+      }
+    } else if (dataType == DataTypes.SHORT) {
+      for (int i = 0; i < pageSize; i++) {
+        long offset = i << shortBits;
+        codec.encode(i, CarbonUnsafe.getUnsafe().getShort(baseAddress, baseOffset + offset));
+      }
+    } else if (dataType == DataTypes.INT) {
+      for (int i = 0; i < pageSize; i++) {
+        long offset = i << intBits;
+        codec.encode(i, CarbonUnsafe.getUnsafe().getInt(baseAddress, baseOffset + offset));
+      }
+    } else if (dataType == DataTypes.LONG) {
+      for (int i = 0; i < pageSize; i++) {
+        long offset = i << longBits;
+        codec.encode(i, CarbonUnsafe.getUnsafe().getLong(baseAddress, baseOffset + offset));
+      }
+    } else if (dataType == DataTypes.FLOAT) {
+      for (int i = 0; i < pageSize; i++) {
+        long offset = i << floatBits;
+        codec.encode(i, CarbonUnsafe.getUnsafe().getFloat(baseAddress, baseOffset + offset));
+      }
+    } else if (dataType == DataTypes.DOUBLE) {
+      for (int i = 0; i < pageSize; i++) {
+        long offset = i << doubleBits;
+        codec.encode(i, CarbonUnsafe.getUnsafe().getDouble(baseAddress, baseOffset + offset));
+      }
+    } else {
+      throw new UnsupportedOperationException("invalid data type: " + dataType);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
index 276b7ff..60c7112 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
@@ -27,19 +27,20 @@ import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.DecimalConverterFactory;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
-import static org.apache.carbondata.core.metadata.datatype.DataType.BYTE;
-import static org.apache.carbondata.core.metadata.datatype.DataType.DECIMAL;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE;
+import static org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL;
 
 public abstract class VarLengthColumnPageBase extends ColumnPage {
 
   static final int byteBits = BYTE.getSizeBits();
-  static final int shortBits = DataType.SHORT.getSizeBits();
-  static final int intBits = DataType.INT.getSizeBits();
-  static final int longBits = DataType.LONG.getSizeBits();
+  static final int shortBits = DataTypes.SHORT.getSizeBits();
+  static final int intBits = DataTypes.INT.getSizeBits();
+  static final int longBits = DataTypes.LONG.getSizeBits();
   // default size for each row, grows as needed
   static final int DEFAULT_ROW_SIZE = 8;
 
@@ -115,7 +116,7 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
             columnSpec.getScale());
     int size = decimalConverter.getSize();
     if (size < 0) {
-      return getLVBytesColumnPage(columnSpec, lvEncodedBytes, DataType.DECIMAL);
+      return getLVBytesColumnPage(columnSpec, lvEncodedBytes, DataTypes.DECIMAL);
     } else {
       // Here the size is always fixed.
       return getDecimalColumnPage(columnSpec, lvEncodedBytes, size);
@@ -127,7 +128,7 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
    */
   static ColumnPage newLVBytesColumnPage(TableSpec.ColumnSpec columnSpec, byte[] lvEncodedBytes)
       throws MemoryException {
-    return getLVBytesColumnPage(columnSpec, lvEncodedBytes, DataType.BYTE_ARRAY);
+    return getLVBytesColumnPage(columnSpec, lvEncodedBytes, DataTypes.BYTE_ARRAY);
   }
 
   private static ColumnPage getDecimalColumnPage(TableSpec.ColumnSpec columnSpec,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
index 3b5ae57..15e26e7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
@@ -33,7 +33,7 @@ import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.datastore.page.ComplexColumnPage;
 import org.apache.carbondata.core.datastore.page.encoding.compress.DirectCompressCodec;
 import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonMetadataUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.BlockletMinMaxIndex;
@@ -149,9 +149,9 @@ public abstract class ColumnPageEncoder {
   private static EncodedColumnPage encodeChildColumn(byte[][] data)
       throws IOException, MemoryException {
     TableSpec.ColumnSpec spec =
-        new TableSpec.ColumnSpec("complex_inner_column", DataType.BYTE_ARRAY, ColumnType.COMPLEX);
+        new TableSpec.ColumnSpec("complex_inner_column", DataTypes.BYTE_ARRAY, ColumnType.COMPLEX);
     ColumnPage page = ColumnPage.wrapByteArrayPage(spec, data);
-    ColumnPageEncoder encoder = new DirectCompressCodec(DataType.BYTE_ARRAY).createEncoder(null);
+    ColumnPageEncoder encoder = new DirectCompressCodec(DataTypes.BYTE_ARRAY).createEncoder(null);
     return encoder.encode(page);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/956833e5/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
index 3f8fca6..daccc32 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
@@ -87,7 +88,7 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
   @Override
   public void write(DataOutput out) throws IOException {
     columnSpec.write(out);
-    out.writeByte(storeDataType.ordinal());
+    out.writeByte(storeDataType.getId());
     out.writeInt(getDecimal());
     out.writeByte(getDataTypeSelected());
     writeMinMax(out);
@@ -98,7 +99,7 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
   public void readFields(DataInput in) throws IOException {
     columnSpec = new TableSpec.ColumnSpec();
     columnSpec.readFields(in);
-    storeDataType = DataType.valueOf(in.readByte());
+    storeDataType = DataTypes.valueOf(in.readByte());
     setDecimal(in.readInt());
     setDataTypeSelected(in.readByte());
     readMinMax(in);
@@ -106,109 +107,95 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
   }
 
   private void writeMinMax(DataOutput out) throws IOException {
-    switch (columnSpec.getSchemaDataType()) {
-      case BYTE:
-        out.writeByte((byte) getMaxValue());
-        out.writeByte((byte) getMinValue());
-        out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
-        break;
-      case SHORT:
-        out.writeShort((short) getMaxValue());
-        out.writeShort((short) getMinValue());
-        out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
-        break;
-      case INT:
-        out.writeInt((int) getMaxValue());
-        out.writeInt((int) getMinValue());
-        out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
-        break;
-      case LONG:
-        out.writeLong((Long) getMaxValue());
-        out.writeLong((Long) getMinValue());
-        out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
-        break;
-      case DOUBLE:
-        out.writeDouble((Double) getMaxValue());
-        out.writeDouble((Double) getMinValue());
-        out.writeDouble(0d); // unique value is obsoleted, maintain for compatibility
-        break;
-      case DECIMAL:
-        byte[] maxAsBytes = getMaxAsBytes(columnSpec.getSchemaDataType());
-        byte[] minAsBytes = getMinAsBytes(columnSpec.getSchemaDataType());
-        byte[] unique = DataTypeUtil.bigDecimalToByte(BigDecimal.ZERO);
-        out.writeShort((short) maxAsBytes.length);
-        out.write(maxAsBytes);
-        out.writeShort((short) minAsBytes.length);
-        out.write(minAsBytes);
-        // unique value is obsoleted, maintain for compatibility
-        out.writeShort((short) unique.length);
-        out.write(unique);
-        out.writeInt(scale);
-        out.writeInt(precision);
-        break;
-      case BYTE_ARRAY:
-        // for complex type, it will come here, ignoring stats for complex type
-        // TODO: support stats for complex type
-        break;
-      default:
-        throw new IllegalArgumentException("invalid data type: " + storeDataType);
+    DataType dataType = columnSpec.getSchemaDataType();
+    if (dataType == DataTypes.BYTE) {
+      out.writeByte((byte) getMaxValue());
+      out.writeByte((byte) getMinValue());
+      out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
+    } else if (dataType == DataTypes.SHORT) {
+      out.writeShort((short) getMaxValue());
+      out.writeShort((short) getMinValue());
+      out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
+    } else if (dataType == DataTypes.INT) {
+      out.writeInt((int) getMaxValue());
+      out.writeInt((int) getMinValue());
+      out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
+    } else if (dataType == DataTypes.LONG) {
+      out.writeLong((Long) getMaxValue());
+      out.writeLong((Long) getMinValue());
+      out.writeLong(0L); // unique value is obsoleted, maintain for compatibility
+    } else if (dataType == DataTypes.DOUBLE) {
+      out.writeDouble((Double) getMaxValue());
+      out.writeDouble((Double) getMinValue());
+      out.writeDouble(0d); // unique value is obsoleted, maintain for compatibility
+    } else if (dataType == DataTypes.DECIMAL) {
+      byte[] maxAsBytes = getMaxAsBytes(columnSpec.getSchemaDataType());
+      byte[] minAsBytes = getMinAsBytes(columnSpec.getSchemaDataType());
+      byte[] unique = DataTypeUtil.bigDecimalToByte(BigDecimal.ZERO);
+      out.writeShort((short) maxAsBytes.length);
+      out.write(maxAsBytes);
+      out.writeShort((short) minAsBytes.length);
+      out.write(minAsBytes);
+      // unique value is obsoleted, maintain for compatibility
+      out.writeShort((short) unique.length);
+      out.write(unique);
+      out.writeInt(scale);
+      out.writeInt(precision);
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
+      // for complex type, it will come here, ignoring stats for complex type
+      // TODO: support stats for complex type
+    } else {
+      throw new IllegalArgumentException("invalid data type: " + storeDataType);
     }
   }
 
   private void readMinMax(DataInput in) throws IOException {
-    switch (columnSpec.getSchemaDataType()) {
-      case BYTE:
-        this.setMaxValue(in.readByte());
-        this.setMinValue(in.readByte());
-        in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
-        break;
-      case SHORT:
-        this.setMaxValue(in.readShort());
-        this.setMinValue(in.readShort());
-        in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
-        break;
-      case INT:
-        this.setMaxValue(in.readInt());
-        this.setMinValue(in.readInt());
-        in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
-        break;
-      case LONG:
-        this.setMaxValue(in.readLong());
-        this.setMinValue(in.readLong());
-        in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
-        break;
-      case DOUBLE:
-        this.setMaxValue(in.readDouble());
-        this.setMinValue(in.readDouble());
-        in.readDouble(); // for non exist value which is obsoleted, it is backward compatibility;
-        break;
-      case DECIMAL:
-        byte[] max = new byte[in.readShort()];
-        in.readFully(max);
-        this.setMaxValue(DataTypeUtil.byteToBigDecimal(max));
-        byte[] min = new byte[in.readShort()];
-        in.readFully(min);
-        this.setMinValue(DataTypeUtil.byteToBigDecimal(min));
-        // unique value is obsoleted, maintain for compatiability
-        short uniqueLength = in.readShort();
-        in.readFully(new byte[uniqueLength]);
-        this.scale = in.readInt();
-        this.precision = in.readInt();
-        break;
-      case BYTE_ARRAY:
-        // for complex type, it will come here, ignoring stats for complex type
-        // TODO: support stats for complex type
-        break;
-      default:
-        throw new IllegalArgumentException("invalid data type: " + storeDataType);
+    DataType dataType = columnSpec.getSchemaDataType();
+    if (dataType == DataTypes.BYTE) {
+      this.setMaxValue(in.readByte());
+      this.setMinValue(in.readByte());
+      in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
+    } else if (dataType == DataTypes.SHORT) {
+      this.setMaxValue(in.readShort());
+      this.setMinValue(in.readShort());
+      in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
+    } else if (dataType == DataTypes.INT) {
+      this.setMaxValue(in.readInt());
+      this.setMinValue(in.readInt());
+      in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
+    } else if (dataType == DataTypes.LONG) {
+      this.setMaxValue(in.readLong());
+      this.setMinValue(in.readLong());
+      in.readLong();  // for non exist value which is obsoleted, it is backward compatibility;
+    } else if (dataType == DataTypes.DOUBLE) {
+      this.setMaxValue(in.readDouble());
+      this.setMinValue(in.readDouble());
+      in.readDouble(); // for non exist value which is obsoleted, it is backward compatibility;
+    } else if (dataType == DataTypes.DECIMAL) {
+      byte[] max = new byte[in.readShort()];
+      in.readFully(max);
+      this.setMaxValue(DataTypeUtil.byteToBigDecimal(max));
+      byte[] min = new byte[in.readShort()];
+      in.readFully(min);
+      this.setMinValue(DataTypeUtil.byteToBigDecimal(min));
+      // unique value is obsoleted, maintain for compatiability
+      short uniqueLength = in.readShort();
+      in.readFully(new byte[uniqueLength]);
+      this.scale = in.readInt();
+      this.precision = in.readInt();
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
+      // for complex type, it will come here, ignoring stats for complex type
+      // TODO: support stats for complex type
+    } else {
+      throw new IllegalArgumentException("invalid data type: " + storeDataType);
     }
   }
 
-  public byte[] getMaxAsBytes(DataType dataType) {
+  private byte[] getMaxAsBytes(DataType dataType) {
     return getValueAsBytes(getMaxValue(), dataType);
   }
 
-  public byte[] getMinAsBytes(DataType dataType) {
+  private byte[] getMinAsBytes(DataType dataType) {
     return getValueAsBytes(getMinValue(), dataType);
   }
 
@@ -217,40 +204,38 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
    */
   private byte[] getValueAsBytes(Object value, DataType dataType) {
     ByteBuffer b;
-    switch (dataType) {
-      case BYTE:
-        b = ByteBuffer.allocate(8);
-        b.putLong((byte) value);
-        b.flip();
-        return b.array();
-      case SHORT:
-        b = ByteBuffer.allocate(8);
-        b.putLong((short) value);
-        b.flip();
-        return b.array();
-      case INT:
-        b = ByteBuffer.allocate(8);
-        b.putLong((int) value);
-        b.flip();
-        return b.array();
-      case LONG:
-        b = ByteBuffer.allocate(8);
-        b.putLong((long) value);
-        b.flip();
-        return b.array();
-      case DOUBLE:
-        b = ByteBuffer.allocate(8);
-        b.putDouble((double) value);
-        b.flip();
-        return b.array();
-      case DECIMAL:
-        return DataTypeUtil.bigDecimalToByte((BigDecimal)value);
-      case STRING:
-      case TIMESTAMP:
-      case DATE:
-        return (byte[]) value;
-      default:
-        throw new IllegalArgumentException("Invalid data type: " + storeDataType);
+    if (dataType == DataTypes.BYTE_ARRAY) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((byte) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.SHORT) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((short) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.INT) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((int) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.LONG) {
+      b = ByteBuffer.allocate(8);
+      b.putLong((long) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.DOUBLE) {
+      b = ByteBuffer.allocate(8);
+      b.putDouble((double) value);
+      b.flip();
+      return b.array();
+    } else if (dataType == DataTypes.DECIMAL) {
+      return DataTypeUtil.bigDecimalToByte((BigDecimal) value);
+    } else if (dataType == DataTypes.STRING || dataType == DataTypes.TIMESTAMP
+        || dataType == DataTypes.DATE) {
+      return (byte[]) value;
+    } else {
+      throw new IllegalArgumentException("Invalid data type: " + storeDataType);
     }
   }