You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by zh...@apache.org on 2019/12/09 14:51:33 UTC

[carbondata] branch master updated: [CARBONDATA-3606] Remove custom partition feature #3498

This is an automated email from the ASF dual-hosted git repository.

zhangzc pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new 1a181d1  [CARBONDATA-3606] Remove custom partition feature #3498
1a181d1 is described below

commit 1a181d19fea55eb128f8fbe104dadb772b53a601
Author: Jacky Li <ja...@qq.com>
AuthorDate: Fri Dec 6 14:45:35 2019 +0800

    [CARBONDATA-3606] Remove custom partition feature #3498
    
    Custom partition feature is deprecated, use Hive standard partition instead.
    
    This closes #3498
---
 .../exceptions/DeprecatedFeatureException.java     |   12 +-
 .../core/constants/CarbonCommonConstants.java      |    3 -
 .../ThriftWrapperSchemaConverterImpl.java          |   36 +-
 .../core/metadata/schema/PartitionInfo.java        |   89 +-
 .../scan/filter/FilterExpressionProcessor.java     |  152 ---
 .../core/scan/filter/FilterProcessor.java          |    7 -
 .../core/scan/filter/partition/AndFilterImpl.java  |   44 -
 .../scan/filter/partition/EqualToFilterImpl.java   |   60 -
 .../core/scan/filter/partition/InFilterImpl.java   |   61 -
 .../partition/KeepAllPartitionFilterImpl.java      |   34 -
 .../core/scan/filter/partition/OrFilterImpl.java   |   44 -
 .../scan/filter/partition/PartitionFilterIntf.java |   36 -
 .../scan/filter/partition/PartitionFilterUtil.java |  209 ----
 .../partition/PruneAllPartitionFilterImpl.java     |   34 -
 .../scan/filter/partition/RangeFilterImpl.java     |   88 --
 .../core/scan/partition/HashPartitioner.java       |   43 -
 .../core/scan/partition/ListPartitioner.java       |   79 --
 .../core/scan/partition/PartitionUtil.java         |  129 --
 .../core/scan/partition/RangePartitioner.java      |   91 --
 .../apache/carbondata/core/util/CarbonUtil.java    |   10 -
 docs/ddl-of-carbondata.md                          |   93 --
 .../examples/CarbonPartitionExample.scala          |  215 ----
 .../apache/carbondata/examplesCI/RunExamples.scala |    4 -
 format/src/main/thrift/schema.thrift               |    1 +
 .../hadoop/api/CarbonFileInputFormat.java          |   16 +-
 .../carbondata/hadoop/api/CarbonInputFormat.java   |   40 +-
 .../hadoop/api/CarbonTableInputFormat.java         |  138 +--
 .../impl/DictionaryDecodeReadSupport.java          |    5 -
 .../sdv/generated/BatchSortLoad3TestCase.scala     |    9 -
 .../cluster/sdv/generated/PartitionTestCase.scala  |  520 --------
 .../testsuite/binary/TestBinaryDataType.scala      |   63 +-
 .../dataload/TestLoadDataWithCompression.scala     |    4 +-
 .../TestCreateDDLForComplexMapType.scala           |    2 +-
 .../createTable/TestCreateTableLike.scala          |   14 -
 .../createTable/TestCreateTableWithSortScope.scala |    8 +-
 .../TestNonTransactionalCarbonTable.scala          |    6 +-
 .../CompactionSupportGlobalSortBigFileTest.scala   |    2 +-
 .../CompactionSupportGlobalSortFunctionTest.scala  |    2 +-
 .../CompactionSupportGlobalSortParameterTest.scala |    2 +-
 .../dataload/TestGlobalSortDataLoad.scala          |   14 -
 ...adDataWithMalformedCarbonCommandException.scala |    4 +-
 .../TestAllDataTypeForPartitionTable.scala         | 1238 --------------------
 .../TestCompactionForPartitionTable.scala          |   82 --
 .../partition/TestDDLForPartitionTable.scala       |  430 -------
 ...DDLForPartitionTableWithDefaultProperties.scala |  206 ----
 .../TestDataLoadingForPartitionTable.scala         |  409 -------
 .../partition/TestQueryForPartitionTable.scala     |  287 -----
 .../testsuite/partition/TestShowPartitions.scala   |  133 +--
 .../partition/TestUpdateForPartitionTable.scala    |   27 +-
 .../spark/partition/DropPartitionCallable.java     |   40 -
 .../spark/partition/SplitPartitionCallable.java    |   42 -
 .../apache/carbondata/spark/PartitionFactory.scala |   65 -
 .../spark/rdd/AlterTableLoadPartitionRDD.scala     |  106 --
 .../carbondata/spark/rdd/CarbonMergerRDD.scala     |   13 +-
 .../spark/rdd/CarbonScanPartitionRDD.scala         |  271 -----
 .../spark/rdd/CarbonSparkPartition.scala           |    1 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala           |    3 +-
 .../carbondata/spark/rdd/PartitionDropper.scala    |  118 --
 .../carbondata/spark/rdd/PartitionSplitter.scala   |   93 --
 .../carbondata/spark/util/CarbonScalaUtil.scala    |    4 +-
 .../apache/carbondata/spark/util/CommonUtil.scala  |  186 +--
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala    |   74 +-
 .../org/apache/spark/util/PartitionUtils.scala     |  265 +----
 .../spark/rdd/CarbonDataRDDFactory.scala           |  168 +--
 .../mutation/CarbonProjectForUpdateCommand.scala   |    8 -
 .../CarbonAlterTableDropPartitionCommand.scala     |  273 -----
 .../CarbonAlterTableSplitPartitionCommand.scala    |  323 -----
 .../CarbonShowCarbonPartitionsCommand.scala        |   56 -
 .../table/CarbonDescribeFormattedCommand.scala     |   11 +-
 .../spark/sql/execution/strategy/DDLStrategy.scala |   21 +-
 .../spark/sql/parser/CarbonSpark2SqlParser.scala   |   40 +-
 .../sql/parser/CarbonSparkSqlParserUtil.scala      |   22 +-
 .../partition/TestAlterPartitionTable.scala        |  926 ---------------
 .../loading/model/CarbonLoadModelBuilder.java      |   22 +-
 .../CarbonRowDataWriterProcessorStepImpl.java      |    7 -
 .../processing/partition/DataPartitioner.java      |   35 -
 .../carbondata/processing/partition/Partition.java |   35 -
 .../spliter/AbstractCarbonQueryExecutor.java       |   90 --
 .../partition/spliter/CarbonSplitExecutor.java     |   73 --
 .../partition/spliter/RowResultProcessor.java      |  113 --
 .../store/writer/AbstractFactDataWriter.java       |   11 -
 .../writer/v3/CarbonFactDataWriterImplV3.java      |    8 -
 .../processing/util/CarbonDataProcessorUtil.java   |   44 -
 83 files changed, 103 insertions(+), 8699 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/Partitioner.java b/common/src/main/java/org/apache/carbondata/common/exceptions/DeprecatedFeatureException.java
similarity index 77%
rename from core/src/main/java/org/apache/carbondata/core/scan/partition/Partitioner.java
rename to common/src/main/java/org/apache/carbondata/common/exceptions/DeprecatedFeatureException.java
index 772a98e..d38a131 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/Partitioner.java
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/DeprecatedFeatureException.java
@@ -15,13 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.scan.partition;
+package org.apache.carbondata.common.exceptions;
 
-import java.io.Serializable;
+public class DeprecatedFeatureException extends RuntimeException {
 
-public interface Partitioner extends Serializable {
-
-  int numPartitions();
-
-  int getPartition(Object key);
+  public DeprecatedFeatureException(String featureName) {
+    super(featureName + " is deprecated in CarbonData 2.0");
+  }
 }
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 9b12251..7a4fa77 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
@@ -441,9 +441,6 @@ public final class CarbonCommonConstants {
   public static final String SORT_SCOPE = "sort_scope";
   public static final String RANGE_COLUMN = "range_column";
   public static final String PARTITION_TYPE = "partition_type";
-  public static final String NUM_PARTITIONS = "num_partitions";
-  public static final String RANGE_INFO = "range_info";
-  public static final String LIST_INFO = "list_info";
   public static final String COLUMN_PROPERTIES = "columnproperties";
   // table block size in MB
   public static final String TABLE_BLOCKSIZE = "table_blocksize";
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 c64ef09..9fd25d6 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
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -239,15 +240,10 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     }
     switch (wrapperPartitionType) {
       case HASH:
-        return org.apache.carbondata.format.PartitionType.HASH;
       case LIST:
-        return org.apache.carbondata.format.PartitionType.LIST;
       case RANGE:
-        return org.apache.carbondata.format.PartitionType.RANGE;
       case RANGE_INTERVAL:
-        return org.apache.carbondata.format.PartitionType.RANGE_INTERVAL;
-      case NATIVE_HIVE:
-        return org.apache.carbondata.format.PartitionType.NATIVE_HIVE;
+        throw new DeprecatedFeatureException("Custom Partition");
       default:
         return org.apache.carbondata.format.PartitionType.NATIVE_HIVE;
     }
@@ -260,16 +256,8 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     for (ColumnSchema wrapperColumnSchema : wrapperPartitionInfo.getColumnSchemaList()) {
       thriftColumnSchema.add(fromWrapperToExternalColumnSchema(wrapperColumnSchema));
     }
-    org.apache.carbondata.format.PartitionInfo externalPartitionInfo =
-        new org.apache.carbondata.format.PartitionInfo(thriftColumnSchema,
-            fromWrapperToExternalPartitionType(wrapperPartitionInfo.getPartitionType()));
-    externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
-    externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
-    externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
-    externalPartitionInfo.setMax_partition(wrapperPartitionInfo.getMaxPartitionId());
-    externalPartitionInfo.setPartition_ids(wrapperPartitionInfo
-        .getPartitionIds());
-    return externalPartitionInfo;
+    return new org.apache.carbondata.format.PartitionInfo(thriftColumnSchema,
+        fromWrapperToExternalPartitionType(wrapperPartitionInfo.getPartitionType()));
   }
 
   /* (non-Javadoc)
@@ -578,15 +566,10 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     }
     switch (externalPartitionType) {
       case HASH:
-        return PartitionType.HASH;
       case LIST:
-        return PartitionType.LIST;
       case RANGE:
-        return PartitionType.RANGE;
       case RANGE_INTERVAL:
-        return PartitionType.RANGE_INTERVAL;
-      case NATIVE_HIVE:
-        return PartitionType.NATIVE_HIVE;
+        throw new DeprecatedFeatureException("Custom Partition");
       default:
         return PartitionType.NATIVE_HIVE;
     }
@@ -599,15 +582,8 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
         externalPartitionInfo.getPartition_columns()) {
       wrapperColumnSchema.add(fromExternalToWrapperColumnSchema(columnSchema));
     }
-    PartitionInfo wrapperPartitionInfo = new PartitionInfo(wrapperColumnSchema,
+    return new PartitionInfo(wrapperColumnSchema,
         fromExternalToWrapperPartitionType(externalPartitionInfo.getPartition_type()));
-    wrapperPartitionInfo.setListInfo(externalPartitionInfo.getList_info());
-    wrapperPartitionInfo.setRangeInfo(externalPartitionInfo.getRange_info());
-    wrapperPartitionInfo.setNumPartitions(externalPartitionInfo.getNum_partitions());
-    wrapperPartitionInfo.setPartitionIds(externalPartitionInfo
-        .getPartition_ids());
-    wrapperPartitionInfo.setMaxPartitionId(externalPartitionInfo.getMax_partition());
-    return wrapperPartitionInfo;
   }
 
   /* (non-Javadoc)
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
index cd69b83..7854267 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
@@ -42,27 +42,32 @@ public class PartitionInfo implements Serializable, Writable {
   /**
    * range information defined for range partition table
    */
+  @Deprecated
   private List<String> rangeInfo;
 
   /**
-   * value list defined for list partition table
+   * list information defined for range partition table
    */
+  @Deprecated
   private List<List<String>> listInfo;
 
   /**
    * total count of partitions
    */
+  @Deprecated
   private int numPartitions;
 
   /**
    * current max partition id, increase only, will be used in alter table partition operation
    */
+  @Deprecated
   private int maxPartitionId;
 
   /**
    * record the partitionId in the logical ascending order
    * initiate when table created and changed when alter table
    */
+  @Deprecated
   private List<Integer> partitionIds;
 
   public PartitionInfo() {
@@ -75,36 +80,6 @@ public class PartitionInfo implements Serializable, Writable {
     this.partitionIds = new ArrayList<>();
   }
 
-  /**
-   * add partition means split default partition, add in last directly
-   */
-  public void  addPartition(int addPartitionCount) {
-    for (int i = 0; i < addPartitionCount; i++) {
-      partitionIds.add(++maxPartitionId);
-      numPartitions++;
-    }
-  }
-
-  /**
-   * e.g. original partition[0,1,2,3,4,5]
-   * split partition 2 to partition 6,7,8 (will not reuse 2)
-   * then sourcePartitionId is 2, newPartitionNumbers is 3
-   * @param sourcePartitionIndex
-   * @param newPartitionNumbers
-   */
-  public void splitPartition(int sourcePartitionIndex, int newPartitionNumbers) {
-    partitionIds.remove(sourcePartitionIndex);
-    for (int i = 0; i < newPartitionNumbers; i++) {
-      partitionIds.add(sourcePartitionIndex + i, ++maxPartitionId);
-    }
-    numPartitions = numPartitions - 1 + newPartitionNumbers;
-  }
-
-  public void dropPartition(int index) {
-    partitionIds.remove(index);
-    numPartitions--;
-  }
-
   public List<ColumnSchema> getColumnSchemaList() {
     return columnSchemaList;
   }
@@ -117,58 +92,6 @@ public class PartitionInfo implements Serializable, Writable {
     return partitionType;
   }
 
-  public void setRangeInfo(List<String> rangeInfo) {
-    this.rangeInfo = rangeInfo;
-  }
-
-  public List<String> getRangeInfo() {
-    return rangeInfo;
-  }
-
-  public void setListInfo(List<List<String>> listInfo) {
-    this.listInfo = listInfo;
-  }
-
-  public List<List<String>> getListInfo() {
-    return listInfo;
-  }
-
-  public void initialize(int partitionNum) {
-    for (int i = 0; i < partitionNum; i++) {
-      partitionIds.add(i);
-    }
-    maxPartitionId = partitionNum - 1;
-    numPartitions = partitionNum;
-  }
-
-  public void setNumPartitions(int numPartitions) {
-    this.numPartitions = numPartitions;
-  }
-
-  public int getNumPartitions() {
-    return numPartitions;
-  }
-
-  public int getMaxPartitionId() {
-    return maxPartitionId;
-  }
-
-  public void setMaxPartitionId(int maxPartitionId) {
-    this.maxPartitionId = maxPartitionId;
-  }
-
-  public List<Integer> getPartitionIds() {
-    return partitionIds;
-  }
-
-  public void setPartitionIds(List<Integer> partitionIdList) {
-    this.partitionIds = partitionIdList;
-  }
-
-  public int getPartitionId(int index) {
-    return partitionIds.get(index);
-  }
-
   @Override
   public void write(DataOutput output) throws IOException {
     output.writeInt(columnSchemaList.size());
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 1c2f51c..7b6b5a8 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
@@ -19,27 +19,19 @@ package org.apache.carbondata.core.scan.filter;
 
 import java.io.IOException;
 import java.util.BitSet;
-import java.util.List;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.scan.expression.BinaryExpression;
-import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
 import org.apache.carbondata.core.scan.expression.conditional.BinaryConditionalExpression;
 import org.apache.carbondata.core.scan.expression.conditional.ConditionalExpression;
-import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
 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.InExpression;
-import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
 import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression;
-import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
 import org.apache.carbondata.core.scan.expression.conditional.StartsWithExpression;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.expression.logical.AndExpression;
@@ -48,14 +40,6 @@ import org.apache.carbondata.core.scan.expression.logical.TrueExpression;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
 import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
-import org.apache.carbondata.core.scan.filter.partition.AndFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.EqualToFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.InFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.KeepAllPartitionFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.OrFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.PartitionFilterIntf;
-import org.apache.carbondata.core.scan.filter.partition.PruneAllPartitionFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.RangeFilterImpl;
 import org.apache.carbondata.core.scan.filter.resolver.ConditionalFilterResolverImpl;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.scan.filter.resolver.LogicalFilterResolverImpl;
@@ -63,8 +47,6 @@ import org.apache.carbondata.core.scan.filter.resolver.RowLevelFilterResolverImp
 import org.apache.carbondata.core.scan.filter.resolver.RowLevelRangeFilterResolverImpl;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.FalseConditionalResolverImpl;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.TrueConditionalResolverImpl;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
 
 import org.apache.log4j.Logger;
 
@@ -91,140 +73,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
   }
 
   /**
-   * Get the map of required partitions
-   * The value of "1" in BitSet represent the required partition
-   * @param expressionTree
-   * @param partitionInfo
-   * @return
-   */
-  @Override
-  public BitSet getFilteredPartitions(Expression expressionTree,
-      PartitionInfo partitionInfo) {
-    Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
-    return createPartitionFilterTree(expressionTree, partitionInfo).applyFilter(partitioner);
-  }
-
-  /**
-   * create partition filter by basing on pushed-down filter
-   * @param expressionTree
-   * @param partitionInfo
-   * @return
-   */
-  private PartitionFilterIntf createPartitionFilterTree(Expression expressionTree,
-      PartitionInfo partitionInfo) {
-    ExpressionType filterExpressionType = expressionTree.getFilterExpressionType();
-    String partitionColumnName = partitionInfo.getColumnSchemaList().get(0).getColumnName();
-    BinaryExpression currentExpression = null;
-    ColumnExpression left = null;
-    switch (filterExpressionType) {
-      case OR:
-        currentExpression = (BinaryExpression) expressionTree;
-        return new OrFilterImpl(
-            createPartitionFilterTree(currentExpression.getLeft(), partitionInfo),
-            createPartitionFilterTree(currentExpression.getRight(), partitionInfo));
-      case RANGE:
-      case AND:
-        currentExpression = (BinaryExpression) expressionTree;
-        return new AndFilterImpl(
-            createPartitionFilterTree(currentExpression.getLeft(), partitionInfo),
-            createPartitionFilterTree(currentExpression.getRight(), partitionInfo));
-      case EQUALS:
-        EqualToExpression equalTo = (EqualToExpression) expressionTree;
-        if (equalTo.getLeft() instanceof ColumnExpression &&
-            equalTo.getRight() instanceof LiteralExpression) {
-          left = (ColumnExpression) equalTo.getLeft();
-          if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
-            return new EqualToFilterImpl(equalTo, partitionInfo);
-          }
-        }
-        return new KeepAllPartitionFilterImpl();
-      case IN:
-        InExpression in = (InExpression) expressionTree;
-        if (in.getLeft() instanceof ColumnExpression &&
-            in.getRight() instanceof ListExpression) {
-          left = (ColumnExpression) in.getLeft();
-          if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
-            return new InFilterImpl(in, partitionInfo);
-          }
-        }
-        return new KeepAllPartitionFilterImpl();
-      case FALSE:
-        return new PruneAllPartitionFilterImpl();
-      case TRUE:
-        return new KeepAllPartitionFilterImpl();
-      case GREATERTHAN:
-        GreaterThanExpression greaterThan = (GreaterThanExpression) expressionTree;
-        if (greaterThan.getLeft() instanceof ColumnExpression &&
-            greaterThan.getRight() instanceof LiteralExpression) {
-          left = (ColumnExpression) greaterThan.getLeft();
-          if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
-            return new RangeFilterImpl((LiteralExpression) greaterThan.getRight(), true, false,
-                partitionInfo);
-          }
-        }
-        return new KeepAllPartitionFilterImpl();
-      case GREATERTHAN_EQUALTO:
-        GreaterThanEqualToExpression greaterThanEqualTo =
-            (GreaterThanEqualToExpression) expressionTree;
-        if (greaterThanEqualTo.getLeft() instanceof ColumnExpression &&
-            greaterThanEqualTo.getRight() instanceof LiteralExpression) {
-          left = (ColumnExpression) greaterThanEqualTo.getLeft();
-          if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
-            return new RangeFilterImpl((LiteralExpression) greaterThanEqualTo.getRight(), true,
-                true, partitionInfo);
-          }
-        }
-        return new KeepAllPartitionFilterImpl();
-      case LESSTHAN:
-        LessThanExpression lessThan = (LessThanExpression) expressionTree;
-        if (lessThan.getLeft() instanceof ColumnExpression &&
-            lessThan.getRight() instanceof LiteralExpression) {
-          left = (ColumnExpression) lessThan.getLeft();
-          if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
-            return new RangeFilterImpl((LiteralExpression) lessThan.getRight(), false, false,
-                partitionInfo);
-          }
-        }
-        return new KeepAllPartitionFilterImpl();
-      case LESSTHAN_EQUALTO:
-        LessThanEqualToExpression lessThanEqualTo = (LessThanEqualToExpression) expressionTree;
-        if (lessThanEqualTo.getLeft() instanceof ColumnExpression &&
-            lessThanEqualTo.getRight() instanceof LiteralExpression) {
-          left = (ColumnExpression) lessThanEqualTo.getLeft();
-          if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
-            return new RangeFilterImpl((LiteralExpression) lessThanEqualTo.getRight(), false, true,
-                partitionInfo);
-          }
-        }
-        return new KeepAllPartitionFilterImpl();
-      case NOT_IN:
-      case NOT_EQUALS:
-      default:
-        return new KeepAllPartitionFilterImpl();
-    }
-  }
-
-  /**
-   * Selects the blocks based on col max and min value.
-   *
-   * @param listOfDataBlocksToScan
-   * @param dataRefNode
-   */
-  private void addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter,
-      List<DataRefNode> listOfDataBlocksToScan, DataRefNode dataRefNode, boolean[] isMinMaxSet) {
-    if (null == dataRefNode.getColumnsMinValue() || null == dataRefNode.getColumnsMaxValue()) {
-      listOfDataBlocksToScan.add(dataRefNode);
-      return;
-    }
-    BitSet bitSet = filterExecuter
-        .isScanRequired(dataRefNode.getColumnsMaxValue(), dataRefNode.getColumnsMinValue(),
-            isMinMaxSet);
-    if (!bitSet.isEmpty()) {
-      listOfDataBlocksToScan.add(dataRefNode);
-    }
-  }
-
-  /**
    * API will return a filter resolver instance which will be used by
    * executers to evaluate or execute the filters.
    *
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
index 91cc0f8..a6f1dc6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
@@ -18,10 +18,8 @@
 package org.apache.carbondata.core.scan.filter;
 
 import java.io.IOException;
-import java.util.BitSet;
 
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -41,9 +39,4 @@ public interface FilterProcessor {
       AbsoluteTableIdentifier tableIdentifier)
       throws FilterUnsupportedException, IOException;
 
-  /**
-   * This API will get the map of required partitions.
-   * @return BitSet the value "1" represent the required partition.
-   */
-  BitSet getFilteredPartitions(Expression expressionTree, PartitionInfo partitionInfo);
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/AndFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/AndFilterImpl.java
deleted file mode 100644
index 1d756f4..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/AndFilterImpl.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of AND logical filter
- */
-public class AndFilterImpl implements PartitionFilterIntf {
-
-  protected PartitionFilterIntf left;
-  protected PartitionFilterIntf right;
-
-  public AndFilterImpl(PartitionFilterIntf left, PartitionFilterIntf right) {
-    this.left = left;
-    this.right = right;
-  }
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-    BitSet leftBitSet = left.applyFilter(partitioner);
-    BitSet rightBitSet = right.applyFilter(partitioner);
-    leftBitSet.and(rightBitSet);
-    return leftBitSet;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java
deleted file mode 100644
index 93657b2..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
-import org.apache.carbondata.core.scan.expression.LiteralExpression;
-import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-
-/**
- * the implement of EqualTo filter
- */
-public class EqualToFilterImpl implements PartitionFilterIntf {
-
-  private EqualToExpression equalTo;
-  private PartitionInfo partitionInfo;
-
-  public EqualToFilterImpl(EqualToExpression equalTo, PartitionInfo partitionInfo) {
-    this.equalTo = equalTo;
-    this.partitionInfo = partitionInfo;
-  }
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-    BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
-    if (equalTo.isNull) {
-      partitionMap.set(partitioner.getPartition(null));
-    } else {
-      LiteralExpression literal = (LiteralExpression) equalTo.getRight();
-      Object value = PartitionUtil.getDataBasedOnDataTypeForFilter(
-          literal.getLiteralExpValue().toString(),
-          partitionInfo.getColumnSchemaList().get(0).getDataType());
-      if (PartitionType.RANGE == partitionInfo.getPartitionType() && value instanceof String) {
-        value = ByteUtil.toBytes((String)value);
-      }
-      partitionMap.set(partitioner.getPartition(value));
-    }
-    return partitionMap;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java
deleted file mode 100644
index 1974ee1..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
-import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.LiteralExpression;
-import org.apache.carbondata.core.scan.expression.conditional.InExpression;
-import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-
-/**
- * the implement of In filter
- */
-public class InFilterImpl implements PartitionFilterIntf {
-
-  private InExpression in;
-  private PartitionInfo partitionInfo;
-
-  public InFilterImpl(InExpression in, PartitionInfo partitionInfo) {
-    this.in = in;
-    this.partitionInfo = partitionInfo;
-  }
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-    BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
-    ListExpression list = (ListExpression) in.getRight();
-    for (Expression expr : list.getChildren()) {
-      LiteralExpression literal = (LiteralExpression) expr;
-      Object value = PartitionUtil.getDataBasedOnDataTypeForFilter(
-          literal.getLiteralExpValue().toString(),
-          partitionInfo.getColumnSchemaList().get(0).getDataType());
-      if (PartitionType.RANGE == partitionInfo.getPartitionType() && value instanceof String) {
-        value = ByteUtil.toBytes((String)value);
-      }
-      partitionMap.set(partitioner.getPartition(value));
-    }
-    return partitionMap;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/KeepAllPartitionFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/KeepAllPartitionFilterImpl.java
deleted file mode 100644
index 8ad3be6..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/KeepAllPartitionFilterImpl.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of partition filter to keep all partitions
- */
-public class KeepAllPartitionFilterImpl implements PartitionFilterIntf {
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-    return PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), true);
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/OrFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/OrFilterImpl.java
deleted file mode 100644
index 804a92a..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/OrFilterImpl.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of OR logical filter
- */
-public class OrFilterImpl implements PartitionFilterIntf {
-
-  protected PartitionFilterIntf left;
-  protected PartitionFilterIntf right;
-
-  public OrFilterImpl(PartitionFilterIntf left, PartitionFilterIntf right) {
-    this.left = left;
-    this.right = right;
-  }
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-    BitSet leftBitSet = left.applyFilter(partitioner);
-    BitSet rightBitSet = right.applyFilter(partitioner);
-    leftBitSet.or(rightBitSet);
-    return leftBitSet;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterIntf.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterIntf.java
deleted file mode 100644
index fae07b4..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterIntf.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the interface of partition filter
- */
-public interface PartitionFilterIntf {
-
-  /**
-   * apply partition filter on Partitioner to get the map of required partitions
-   * @param partitioner
-   * @return
-   */
-  BitSet applyFilter(Partitioner partitioner);
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java
deleted file mode 100644
index 9671199..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.text.DateFormat;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.scan.partition.ListPartitioner;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.RangePartitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.core.util.comparator.Comparator;
-import org.apache.carbondata.core.util.comparator.SerializableComparator;
-
-public class PartitionFilterUtil {
-
-  /**
-   * get partition map of range filter on list partition table
-   * @param partitionInfo
-   * @param partitioner
-   * @param filterValue
-   * @param isGreaterThan
-   * @param isEqualTo
-   * @return
-   */
-  public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
-      ListPartitioner partitioner, Object filterValue,  boolean isGreaterThan, boolean isEqualTo,
-      DateFormat timestampFormatter, DateFormat dateFormatter) {
-
-    List<List<String>> listInfo = partitionInfo.getListInfo();
-    DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
-
-    SerializableComparator comparator =
-        Comparator.getComparator(partitionColumnDataType);
-
-    BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
-    // add default partition
-    partitionMap.set(0);
-    int partitions = listInfo.size();
-    if (isGreaterThan) {
-      if (isEqualTo) {
-        // GreaterThanEqualTo(>=)
-        outer1:
-        for (int i = 0; i < partitions; i++) {
-          for (String value : listInfo.get(i)) {
-            Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
-                timestampFormatter, dateFormatter);
-            if (listValue instanceof String) {
-              listValue = ByteUtil.toBytes((String)listValue);
-            }
-            if (comparator.compare(listValue, filterValue) >= 0) {
-              partitionMap.set(i + 1);
-              continue outer1;
-            }
-          }
-        }
-      } else {
-        // GreaterThan(>)
-        outer2:
-        for (int i = 0; i < partitions; i++) {
-          for (String value : listInfo.get(i)) {
-            Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
-                timestampFormatter, dateFormatter);
-            if (listValue instanceof String) {
-              listValue = ByteUtil.toBytes((String)listValue);
-            }
-            if (comparator.compare(listValue, filterValue) > 0) {
-              partitionMap.set(i + 1);
-              continue outer2;
-            }
-          }
-        }
-      }
-    } else {
-      if (isEqualTo) {
-        // LessThanEqualTo(<=)
-        outer3:
-        for (int i = 0; i < partitions; i++) {
-          for (String value : listInfo.get(i)) {
-            Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
-                timestampFormatter, dateFormatter);
-            if (listValue instanceof String) {
-              listValue = ByteUtil.toBytes((String)listValue);
-            }
-            if (comparator.compare(listValue, filterValue) <= 0) {
-              partitionMap.set(i + 1);
-              continue outer3;
-            }
-          }
-        }
-      } else {
-        // LessThan(<)
-        outer4:
-        for (int i = 0; i < partitions; i++) {
-          for (String value : listInfo.get(i)) {
-            Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
-                timestampFormatter, dateFormatter);
-            if (listValue instanceof String) {
-              listValue = ByteUtil.toBytes((String)listValue);
-            }
-            if (comparator.compare(listValue, filterValue) < 0) {
-              partitionMap.set(i + 1);
-              continue outer4;
-            }
-          }
-        }
-      }
-    }
-
-    return partitionMap;
-  }
-
-  /**
-   * get partition map of range filter on range partition table
-   * @param partitionInfo
-   * @param partitioner
-   * @param filterValue
-   * @param isGreaterThan
-   * @param isEqualTo
-   * @return
-   */
-  public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
-      RangePartitioner partitioner, Object filterValue, boolean isGreaterThan, boolean isEqualTo,
-      DateFormat timestampFormatter, DateFormat dateFormatter) {
-
-    List<String> values = partitionInfo.getRangeInfo();
-    DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
-
-    SerializableComparator comparator =
-        Comparator.getComparator(partitionColumnDataType);
-
-    BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
-
-    int numPartitions = values.size();
-    int result = 0;
-    // the partition index of filter value
-    int partitionIndex = 0;
-    // find the partition of filter value
-    for (; partitionIndex < numPartitions; partitionIndex++) {
-      Object value = PartitionUtil.getDataBasedOnDataType(
-          values.get(partitionIndex), partitionColumnDataType, timestampFormatter, dateFormatter);
-      if (value instanceof String) {
-        value = ByteUtil.toBytes((String)value);
-      }
-      result = comparator.compare(filterValue, value);
-      if (result <= 0) {
-        break;
-      }
-    }
-    if (partitionIndex == numPartitions) {
-      // filter value is in default partition
-      if (isGreaterThan) {
-        // GreaterThan(>), GreaterThanEqualTo(>=)
-        partitionMap.set(0);
-      } else {
-        // LessThan(<), LessThanEqualTo(<=)
-        partitionMap.set(0, partitioner.numPartitions());
-      }
-    } else {
-      // filter value is not in default partition
-      if (result == 0) {
-        // if result is 0, the filter value is a bound value of range partition.
-        if (isGreaterThan) {
-          // GreaterThan(>), GreaterThanEqualTo(>=)
-          partitionMap.set(partitionIndex + 2, partitioner.numPartitions());
-          partitionMap.set(0);
-        } else {
-          if (isEqualTo) {
-            // LessThanEqualTo(<=)
-            partitionMap.set(1, partitionIndex + 3);
-          } else {
-            // LessThan(<)
-            partitionMap.set(1, partitionIndex + 2);
-          }
-        }
-      } else {
-        // the filter value is not a bound value of range partition
-        if (isGreaterThan) {
-          // GreaterThan(>), GreaterThanEqualTo(>=)
-          partitionMap.set(partitionIndex + 1, partitioner.numPartitions());
-          partitionMap.set(0);
-        } else {
-          // LessThan(<), LessThanEqualTo(<=)
-          partitionMap.set(1, partitionIndex + 2);
-        }
-      }
-    }
-    return partitionMap;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PruneAllPartitionFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PruneAllPartitionFilterImpl.java
deleted file mode 100644
index 4b4625e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PruneAllPartitionFilterImpl.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of partition filter to prune all partitions
- */
-public class PruneAllPartitionFilterImpl implements PartitionFilterIntf {
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-    return PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java
deleted file mode 100644
index 6484f1d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.filter.partition;
-
-import java.text.SimpleDateFormat;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.scan.expression.LiteralExpression;
-import org.apache.carbondata.core.scan.partition.ListPartitioner;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-import org.apache.carbondata.core.scan.partition.RangePartitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-/**
- * the implement of Range filter(include <=, <, >=, >)
- */
-public class RangeFilterImpl implements PartitionFilterIntf {
-
-  private LiteralExpression literal;
-  private boolean isGreaterThan;
-  private boolean isEqualTo;
-  private PartitionInfo partitionInfo;
-
-  public RangeFilterImpl(LiteralExpression literal, boolean isGreaterThan, boolean isEqualTo,
-      PartitionInfo partitionInfo) {
-    this.literal = literal;
-    this.isGreaterThan = isGreaterThan;
-    this.isEqualTo = isEqualTo;
-    this.partitionInfo = partitionInfo;
-  }
-
-  @Override
-  public BitSet applyFilter(Partitioner partitioner) {
-
-    SimpleDateFormat timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-
-    SimpleDateFormat dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-            CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-
-    switch (partitionInfo.getPartitionType()) {
-      case LIST:
-        Object filterValueOfList = PartitionUtil.getDataBasedOnDataTypeForFilter(
-            literal.getLiteralExpValue().toString(),
-            partitionInfo.getColumnSchemaList().get(0).getDataType());
-        if (filterValueOfList instanceof String) {
-          filterValueOfList = ByteUtil.toBytes((String)filterValueOfList);
-        }
-        return PartitionFilterUtil.getPartitionMapForRangeFilter(partitionInfo,
-            (ListPartitioner) partitioner, filterValueOfList, isGreaterThan, isEqualTo,
-            timestampFormatter, dateFormatter);
-      case RANGE:
-        Object filterValueOfRange = PartitionUtil.getDataBasedOnDataTypeForFilter(
-            literal.getLiteralExpValue().toString(),
-            partitionInfo.getColumnSchemaList().get(0).getDataType());
-        if (filterValueOfRange instanceof String) {
-          filterValueOfRange = ByteUtil.toBytes((String)filterValueOfRange);
-        }
-        return PartitionFilterUtil.getPartitionMapForRangeFilter(partitionInfo,
-            (RangePartitioner) partitioner, filterValueOfRange, isGreaterThan, isEqualTo,
-            timestampFormatter, dateFormatter);
-      default:
-        return PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), true);
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/HashPartitioner.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/HashPartitioner.java
deleted file mode 100644
index 8562abb..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/HashPartitioner.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.partition;
-
-/**
- * Hash Partitioner
- */
-public class HashPartitioner implements Partitioner {
-
-  private int numPartitions = 0;
-
-  public HashPartitioner(int numPartitions) {
-    this.numPartitions = numPartitions;
-  }
-
-  @Override
-  public int numPartitions() {
-    return numPartitions;
-  }
-
-  @Override
-  public int getPartition(Object key) {
-    if (key == null) {
-      return 0;
-    }
-    return (key.hashCode() & Integer.MAX_VALUE) % numPartitions;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/ListPartitioner.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/ListPartitioner.java
deleted file mode 100644
index 8936b0d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/ListPartitioner.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.partition;
-
-import java.text.SimpleDateFormat;
-import java.util.List;
-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.schema.PartitionInfo;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-/**
- * List Partitioner
- */
-public class ListPartitioner implements Partitioner {
-
-  private SimpleDateFormat timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-
-  private SimpleDateFormat dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-
-  /**
-   * Map the value of ListPartition to partition id.
-   */
-  private Map<Object, Integer> map = new java.util.HashMap<Object, Integer>();
-
-  private int numPartitions;
-
-  public ListPartitioner(PartitionInfo partitionInfo) {
-    List<List<String>> values = partitionInfo.getListInfo();
-    DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
-    numPartitions = values.size();
-    for (int i = 0; i < numPartitions; i++) {
-      for (String value : values.get(i)) {
-        map.put(PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
-            timestampFormatter, dateFormatter), i + 1);
-      }
-    }
-  }
-
-  /**
-   * Number of partitions
-   * add extra default partition
-   * @return
-   */
-  @Override
-  public int numPartitions() {
-    return numPartitions + 1;
-  }
-
-  @Override
-  public int getPartition(Object key) {
-    Integer partition = map.get(key);
-    if (partition == null) {
-      return 0;
-    }
-    return partition;
-  }
-}
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
deleted file mode 100644
index 66ceeb8..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.partition;
-
-import java.math.BigDecimal;
-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;
-
-public class PartitionUtil {
-
-  public static Partitioner getPartitioner(PartitionInfo partitionInfo) {
-    switch (partitionInfo.getPartitionType()) {
-      case HASH:
-        return new HashPartitioner(partitionInfo.getNumPartitions());
-      case LIST:
-        return new ListPartitioner(partitionInfo);
-      case RANGE:
-        return new RangePartitioner(partitionInfo);
-      default:
-        throw new UnsupportedOperationException(
-            "unsupported partition type: " + partitionInfo.getPartitionType().name());
-    }
-  }
-
-  public static Object getDataBasedOnDataType(String data, DataType actualDataType,
-      DateFormat timestampFormatter, DateFormat dateFormatter) {
-    if (data == null) {
-      return null;
-    }
-    if (actualDataType != DataTypes.STRING && StringUtils.isEmpty(data)) {
-      return null;
-    }
-    try {
-      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 (DataTypes.isDecimal(actualDataType)) {
-        return new BigDecimal(data);
-      } else {
-        return data;
-      }
-    } catch (Exception ex) {
-      return null;
-    }
-  }
-
-  /**
-   * convert the string value of partition filter to the Object
-   * @param data
-   * @param actualDataType
-   * @return
-   */
-  public static Object getDataBasedOnDataTypeForFilter(String data, DataType actualDataType) {
-    if (data == null) {
-      return null;
-    }
-    if (actualDataType != DataTypes.STRING && StringUtils.isEmpty(data)) {
-      return null;
-    }
-    try {
-      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 (DataTypes.isDecimal(actualDataType)) {
-        return new BigDecimal(data);
-      } else {
-        return data;
-      }
-    } catch (Exception ex) {
-      return null;
-    }
-  }
-
-  /**
-   * generate a BitSet by size
-   * @param size
-   * @param initValue true: initialize all bits to true
-   * @return
-   */
-  public static BitSet generateBitSetBySize(int size, boolean initValue) {
-    BitSet bitSet = new BitSet(size);
-    if (initValue) {
-      bitSet.set(0, size);
-    }
-    return bitSet;
-  }
-
-}
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
deleted file mode 100644
index 478196b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.partition;
-
-import java.text.SimpleDateFormat;
-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;
-import org.apache.carbondata.core.util.comparator.Comparator;
-import org.apache.carbondata.core.util.comparator.SerializableComparator;
-
-/**
- * Range Partitioner
- */
-public class RangePartitioner implements Partitioner {
-
-  private int numPartitions;
-  private Object[] bounds;
-  private SerializableComparator comparator;
-
-  private SimpleDateFormat timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-
-  private SimpleDateFormat dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-
-  public RangePartitioner(PartitionInfo partitionInfo) {
-    List<String> values = partitionInfo.getRangeInfo();
-    DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
-    numPartitions = values.size();
-    bounds = new Object[numPartitions];
-    if (partitionColumnDataType == DataTypes.STRING) {
-      for (int i = 0; i < numPartitions; i++) {
-        bounds[i] = ByteUtil.toBytes(values.get(i));
-      }
-    } else {
-      for (int i = 0; i < numPartitions; i++) {
-        bounds[i] = PartitionUtil.getDataBasedOnDataType(values.get(i), partitionColumnDataType,
-            timestampFormatter, dateFormatter);
-      }
-    }
-    comparator = Comparator.getComparator(partitionColumnDataType);
-  }
-
-  /**
-   * number of partitions
-   * add extra default partition
-   *
-   * @return
-   */
-  @Override
-  public int numPartitions() {
-    return numPartitions + 1;
-  }
-
-  @Override
-  public int getPartition(Object key) {
-    if (key == null) {
-      return 0;
-    } else {
-      for (int i = 0; i < numPartitions; i++) {
-        if (comparator.compare(key, bounds[i]) < 0) {
-          return i + 1;
-        }
-      }
-      return 0;
-    }
-  }
-}
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 00d8b2a..ddd0491 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
@@ -2034,16 +2034,6 @@ public final class CarbonUtil {
         column.setDataType(DataTypes.createDecimalType(column.getPrecision(), column.getScale()));
       }
     }
-    if (tableInfo.getFactTable().getPartitionInfo() != null) {
-      List<ColumnSchema> partitionColumns =
-          tableInfo.getFactTable().getPartitionInfo().getColumnSchemaList();
-      for (ColumnSchema column : partitionColumns) {
-        DataType dataType = column.getDataType();
-        if (DataTypes.isDecimal(dataType)) {
-          column.setDataType(DataTypes.createDecimalType(column.getPrecision(), column.getScale()));
-        }
-      }
-    }
   }
 
   /**
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index 9c9a02f..0660504 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -58,10 +58,6 @@ CarbonData DDL statements are documented here,which includes:
 * [PARTITION](#partition)
   * [STANDARD PARTITION(HIVE)](#standard-partition)
     * [INSERT OVERWRITE PARTITION](#insert-overwrite)
-  * [CARBONDATA PARTITION](#create-hash-partition-table)
-    * [HASH PARTITION](#create-hash-partition-table)
-    * [RANGE PARTITION](#create-range-partition-table)
-    * [LIST PARTITION](#create-list-partition-table)
   * [SHOW PARTITIONS](#show-partitions)
   * [ADD PARTITION](#add-a-new-partition)
   * [SPLIT PARTITION](#split-a-partition)
@@ -960,95 +956,6 @@ Users can specify which columns to include and exclude for local dictionary gene
   WHERE au.country = 'US';
   ```
 
-### CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.
-
-  The partition supports three type:(Hash,Range,List), similar to other system's partition features, CarbonData's partition feature can be used to improve query performance by filtering on the partition column.
-
-### Create Hash Partition Table
-
-  This command allows us to create hash partition.
-
-  ```
-  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                    [(col_name data_type , ...)]
-  PARTITIONED BY (partition_col_name data_type)
-  STORED AS carbondata
-  [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
-                  'NUM_PARTITIONS'='N' ...)]
-  ```
-
-  **NOTE:** N is the number of hash partitions
-
-
-  Example:
-  ```
-  CREATE TABLE IF NOT EXISTS hash_partition_table(
-      col_A STRING,
-      col_B INT,
-      col_C LONG,
-      col_D DECIMAL(10,2),
-      col_F TIMESTAMP
-  ) PARTITIONED BY (col_E LONG)
-  STORED AS carbondata TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
-  ```
-
-### Create Range Partition Table
-
-  This command allows us to create range partition.
-  ```
-  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                    [(col_name data_type , ...)]
-  PARTITIONED BY (partition_col_name data_type)
-  STORED AS carbondata
-  [TBLPROPERTIES ('PARTITION_TYPE'='RANGE',
-                  'RANGE_INFO'='2014-01-01, 2015-01-01, 2016-01-01, ...')]
-  ```
-
-  **NOTE:**
-  * The 'RANGE_INFO' must be defined in ascending order in the table properties.
-  * The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.
-
-  Example:
-  ```
-  CREATE TABLE IF NOT EXISTS range_partition_table(
-      col_A STRING,
-      col_B INT,
-      col_C LONG,
-      col_D DECIMAL(10,2),
-      col_E LONG
-  ) partitioned by (col_F Timestamp)
-  STORED BY 'carbondata'
-  TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-  'RANGE_INFO'='2015-01-01, 2016-01-01, 2017-01-01, 2017-02-01')
-  ```
-
-### Create List Partition Table
-
-  This command allows us to create list partition.
-  ```
-  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                    [(col_name data_type , ...)]
-  PARTITIONED BY (partition_col_name data_type)
-  STORED AS carbondata
-  [TBLPROPERTIES ('PARTITION_TYPE'='LIST',
-                  'LIST_INFO'='A, B, C, ...')]
-  ```
-  **NOTE:** List partition supports list info in one level group.
-
-  Example:
-  ```
-  CREATE TABLE IF NOT EXISTS list_partition_table(
-      col_B INT,
-      col_C LONG,
-      col_D DECIMAL(10,2),
-      col_E LONG,
-      col_F TIMESTAMP
-   ) PARTITIONED BY (col_A STRING)
-  STORED AS carbondata
-  TBLPROPERTIES('PARTITION_TYPE'='LIST',
-  'LIST_INFO'='aaaa, bbbb, (cccc, dddd), eeee')
-  ```
-
 
 ### Show Partitions
 
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
deleted file mode 100644
index 1c4b1bc..0000000
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.examples
-
-import java.io.File
-
-import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.examples.util.ExampleUtils
-import org.apache.carbondata.spark.exception.ProcessMetaDataException
-
-
-
-object CarbonPartitionExample {
-
-  def main(args: Array[String]) {
-    val spark = ExampleUtils.createCarbonSession("CarbonPartitionExample")
-    exampleBody(spark)
-    spark.close()
-  }
-
-  def exampleBody(spark : SparkSession): Unit = {
-    val rootPath = new File(this.getClass.getResource("/").getPath
-                            + "../../../..").getCanonicalPath
-    val testData = s"$rootPath/integration/spark-common-test/src/test/resources/partition_data.csv"
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-    // range partition with bucket defined
-    spark.sql("DROP TABLE IF EXISTS t0")
-    spark.sql("""
-                | CREATE TABLE IF NOT EXISTS t0
-                | (
-                | id Int,
-                | vin String,
-                | phonenumber Long,
-                | country String,
-                | area String,
-                | salary Int
-                | )
-                | PARTITIONED BY (logdate Timestamp)
-                | STORED BY 'carbondata'
-                | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-                | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01')
-              """.stripMargin)
-
-    // none partition table
-    spark.sql("DROP TABLE IF EXISTS t1")
-    spark.sql("""
-                | CREATE TABLE IF NOT EXISTS t1
-                | (
-                | id Int,
-                | vin String,
-                | logdate Timestamp,
-                | phonenumber Long,
-                | country String,
-                | area String
-                | )
-                | STORED BY 'carbondata'
-              """.stripMargin)
-
-    // list partition
-    spark.sql("DROP TABLE IF EXISTS t2")
-    spark.sql("""
-                | CREATE TABLE IF NOT EXISTS t2
-                | (
-                | id Int,
-                | vin String,
-                | logdate Timestamp,
-                | phonenumber Long,
-                | country String,
-                | salary Int
-                | )
-                | PARTITIONED BY (area String)
-                | STORED BY 'carbondata'
-                | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-                | 'LIST_INFO'='Asia, America, Europe', 'DICTIONARY_EXCLUDE' ='area')
-              """.stripMargin)
-
-    // hash partition
-    spark.sql("DROP TABLE IF EXISTS t3")
-    spark.sql("""
-                | CREATE TABLE IF NOT EXISTS t3
-                | (
-                | id Int,
-                | logdate Timestamp,
-                | phonenumber Long,
-                | country String,
-                | area String,
-                | salary Int
-                | )
-                | PARTITIONED BY (vin String)
-                | STORED BY 'carbondata'
-                | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
-              """.stripMargin)
-
-    // list partition
-    spark.sql("DROP TABLE IF EXISTS t5")
-    spark.sql("""
-                | CREATE TABLE IF NOT EXISTS t5
-                | (
-                | id Int,
-                | vin String,
-                | logdate Timestamp,
-                | phonenumber Long,
-                | area String,
-                | salary Int
-                |)
-                | PARTITIONED BY (country String)
-                | STORED BY 'carbondata'
-                | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-                | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, NotGood), Korea ')
-              """.stripMargin)
-
-    // load data into partition table
-    spark.sql(s"""
-       LOAD DATA LOCAL INPATH '$testData' into table t0 options('BAD_RECORDS_ACTION'='FORCE')
-       """)
-    spark.sql(s"""
-       LOAD DATA LOCAL INPATH '$testData' into table t5 options('BAD_RECORDS_ACTION'='FORCE')
-       """)
-
-    // alter list partition table t5 to add a partition
-    spark.sql(s"""Alter table t5 add partition ('OutSpace')""".stripMargin)
-    // alter list partition table t5 to split partition 4 into 3 independent partition
-    spark.sql(
-      s"""
-         Alter table t5 split partition(4) into ('Canada', 'Russia', '(Good, NotGood)')
-       """.stripMargin)
-
-    spark.sql("""select * from t5 where country = 'Good' """).show(100, false)
-
-    spark.sql("select * from t0 order by salary ").show(100, false)
-    spark.sql("select * from t5 order by salary ").show(100, false)
-
-    // hive partition table
-    spark.sql("DROP TABLE IF EXISTS t7")
-    spark.sql("""
-                | create table t7(id int, name string) partitioned by (city string)
-                | row format delimited fields terminated by ','
-              """.stripMargin)
-    spark.sql("alter table t7 add partition (city = 'Hangzhou')")
-
-    // not default db partition table
-    try {
-      spark.sql(s"DROP TABLE IF EXISTS partitionDB.t9")
-    } catch {
-      case ex: NoSuchDatabaseException => LOGGER.error(ex.getMessage())
-    }
-    spark.sql(s"DROP DATABASE IF EXISTS partitionDB")
-    spark.sql(s"CREATE DATABASE partitionDB")
-    spark.sql(s"""
-                 | CREATE TABLE IF NOT EXISTS partitionDB.t9(
-                 | id Int,
-                 | logdate Timestamp,
-                 | phonenumber Int,
-                 | country String,
-                 | area String
-                 | )
-                 | PARTITIONED BY (vin String)
-                 | STORED BY 'carbondata'
-                 | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
-                """.stripMargin)
-
-    // show tables
-    spark.sql("SHOW TABLES").show()
-
-    // show partitions
-    try {
-      spark.sql("""SHOW PARTITIONS t1""").show(100, false)
-    } catch {
-      case ex: ProcessMetaDataException => LOGGER.error(ex.getMessage())
-    }
-    spark.sql("""SHOW PARTITIONS t0""").show(100, false)
-    spark.sql("""SHOW PARTITIONS t3""").show(100, false)
-    spark.sql("""SHOW PARTITIONS t5""").show(100, false)
-    spark.sql("""SHOW PARTITIONS t7""").show(100, false)
-    spark.sql("""SHOW PARTITIONS partitionDB.t9""").show(100, false)
-
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
-    // drop table
-    spark.sql("DROP TABLE IF EXISTS t0")
-    spark.sql("DROP TABLE IF EXISTS t1")
-    spark.sql("DROP TABLE IF EXISTS t2")
-    spark.sql("DROP TABLE IF EXISTS t3")
-    spark.sql("DROP TABLE IF EXISTS t5")
-    spark.sql("DROP TABLE IF EXISTS t7")
-    spark.sql("DROP TABLE IF EXISTS partitionDB.t9")
-    spark.sql(s"DROP DATABASE IF EXISTS partitionDB")
-  }
-}
diff --git a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
index 853f778..cef2b97 100644
--- a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
+++ b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
@@ -62,10 +62,6 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
     CarbonDataFrameExample.exampleBody(spark)
   }
 
-  test("CarbonPartitionExample") {
-    CarbonPartitionExample.exampleBody(spark)
-  }
-
   test("CarbonSessionExample") {
     CarbonSessionExample.exampleBody(spark)
   }
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 5daf767..1180991 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -64,6 +64,7 @@ enum Encoding{
 	DIRECT_COMPRESS_VARCHAR = 14;  // Identifies that a columm is encoded using DirectCompressCodec, it is used for long string columns
 }
 
+// Only NATIVE_HIVE is supported, others are deprecated since CarbonData 2.0
 enum PartitionType{
   RANGE = 0;
   RANGE_INTERVAL = 1;
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
index 0fecb75..52796fb 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -20,7 +20,6 @@ package org.apache.carbondata.hadoop.api;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.LinkedList;
@@ -36,7 +35,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.metadata.schema.SchemaReader;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
@@ -131,7 +129,6 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
         }
       }
       // this will be null in case of corrupt schema file.
-      PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
       DataMapFilter filter = getFilterPredicates(job.getConfiguration());
 
       // if external table Segments are found, add it to the List
@@ -171,7 +168,7 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
       }
       if (useBlockDataMap) {
         // do block filtering and get split
-        splits = getSplits(job, filter, externalTableSegments, null, partitionInfo, null);
+        splits = getSplits(job, filter, externalTableSegments);
       } else {
         List<CarbonFile> carbonFiles = null;
         if (null != this.fileLists) {
@@ -252,17 +249,18 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
    * @return
    * @throws IOException
    */
-  private List<InputSplit> getSplits(JobContext job, DataMapFilter dataMapFilter,
-      List<Segment> validSegments, BitSet matchedPartitions, PartitionInfo partitionInfo,
-      List<Integer> oldPartitionIdList) throws IOException {
+  private List<InputSplit> getSplits(
+      JobContext job,
+      DataMapFilter dataMapFilter,
+      List<Segment> validSegments) throws IOException {
 
     numSegments = validSegments.size();
     List<InputSplit> result = new LinkedList<InputSplit>();
 
     // for each segment fetch blocks matching filter in Driver BTree
     List<CarbonInputSplit> dataBlocksOfSegment =
-        getDataBlocksOfSegment(job, carbonTable, dataMapFilter, matchedPartitions, validSegments,
-            partitionInfo, oldPartitionIdList, new ArrayList<Segment>(), new ArrayList<String>());
+        getDataBlocksOfSegment(job, carbonTable, dataMapFilter, validSegments,
+            new ArrayList<Segment>(), new ArrayList<String>());
     numBlocks = dataBlocksOfSegment.size();
     result.addAll(dataBlocksOfSegment);
     return result;
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
index 1382368..5f7814a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -22,7 +22,6 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -48,9 +47,7 @@ import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.metadata.schema.SchemaReader;
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -494,8 +491,7 @@ m filterExpression
    * get data blocks of given segment
    */
   protected List<CarbonInputSplit> getDataBlocksOfSegment(JobContext job, CarbonTable carbonTable,
-      DataMapFilter expression, BitSet matchedPartitions, List<Segment> segmentIds,
-      PartitionInfo partitionInfo, List<Integer> oldPartitionIdList,
+      DataMapFilter expression, List<Segment> segmentIds,
       List<Segment> invalidSegments, List<String> segmentsToBeRefreshed)
       throws IOException {
 
@@ -509,35 +505,13 @@ m filterExpression
         getPrunedBlocklets(job, carbonTable, expression, segmentIds, invalidSegments,
             segmentsToBeRefreshed);
     List<CarbonInputSplit> resultFilteredBlocks = new ArrayList<>();
-    int partitionIndex = 0;
-    List<Integer> partitionIdList = new ArrayList<>();
-    if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
-      partitionIdList = partitionInfo.getPartitionIds();
-    }
     for (ExtendedBlocklet blocklet : prunedBlocklets) {
-
-      // OldPartitionIdList is only used in alter table partition command because it change
-      // partition info first and then read data.
-      // For other normal query should use newest partitionIdList
-      if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
-        long partitionId = Long.parseLong(CarbonTablePath.DataFileUtil
-            .getTaskIdFromTaskNo(CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath())));
-        if (oldPartitionIdList != null) {
-          partitionIndex = oldPartitionIdList.indexOf((int) partitionId);
-        } else {
-          partitionIndex = partitionIdList.indexOf((int) partitionId);
-        }
-      }
-      if (partitionIndex != -1) {
-        // matchedPartitions variable will be null in two cases as follows
-        // 1. the table is not a partition table
-        // 2. the table is a partition table, and all partitions are matched by query
-        // for partition table, the task id of carbaondata file name is the partition id.
-        // if this partition is not required, here will skip it.
-        if (matchedPartitions == null || matchedPartitions.get(partitionIndex)) {
-          resultFilteredBlocks.add(blocklet.getInputSplit());
-        }
-      }
+      // matchedPartitions variable will be null in two cases as follows
+      // 1. the table is not a partition table
+      // 2. the table is a partition table, and all partitions are matched by query
+      // for partition table, the task id of carbaondata file name is the partition id.
+      // if this partition is not required, here will skip it.
+      resultFilteredBlocks.add(blocklet.getInputSplit());
     }
     statistic
         .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index deae842..6a0f946 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -20,7 +20,6 @@ package org.apache.carbondata.hadoop.api;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -37,8 +36,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-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.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
@@ -48,7 +45,6 @@ import org.apache.carbondata.core.profiler.ExplainCollector;
 import org.apache.carbondata.core.readcommitter.LatestFilesReadCommittedScope;
 import org.apache.carbondata.core.readcommitter.ReadCommittedScope;
 import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope;
-import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.statusmanager.FileFormat;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -83,11 +79,8 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
       "mapreduce.input.carboninputformat.segmentnumbers";
   // comma separated list of input files
   public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
-  private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
   private static final Logger LOG =
       LogServiceFactory.getLogService(CarbonTableInputFormat.class.getName());
-  private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
-  private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
   private static final String CARBON_TRANSACTIONAL_TABLE =
       "mapreduce.input.carboninputformat.transactional";
   public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
@@ -133,12 +126,12 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
       if (!accessStreamingSegments) {
         List<Segment> validSegments = segments.getValidSegments();
         streamSegments = segments.getStreamSegments();
-        streamSegments = getFilteredSegment(job, streamSegments, true, readCommittedScope);
+        streamSegments = getFilteredSegment(job, streamSegments, readCommittedScope);
         if (validSegments.size() == 0) {
           return getSplitsOfStreaming(job, streamSegments, carbonTable);
         }
         List<Segment> filteredSegmentToAccess =
-            getFilteredSegment(job, segments.getValidSegments(), true, readCommittedScope);
+            getFilteredSegment(job, segments.getValidSegments(), readCommittedScope);
         if (filteredSegmentToAccess.size() == 0) {
           return getSplitsOfStreaming(job, streamSegments, carbonTable);
         } else {
@@ -146,7 +139,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
         }
       } else {
         List<Segment> filteredNormalSegments =
-            getFilteredNormalSegments(job, segments.getValidSegments(),
+            getFilteredNormalSegments(segments.getValidSegments(),
                 getSegmentsToAccess(job, readCommittedScope));
         streamSegments = segments.getStreamSegments();
         if (filteredNormalSegments.size() == 0) {
@@ -170,34 +163,20 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
     validAndInProgressSegments.addAll(segments.getListOfInProgressSegments());
     // get updated filtered list
     List<Segment> filteredSegmentToAccess =
-        getFilteredSegment(job, new ArrayList<>(validAndInProgressSegments), false,
-            readCommittedScope);
+        getFilteredSegment(job, new ArrayList<>(validAndInProgressSegments), readCommittedScope);
 
     // process and resolve the expression
     DataMapFilter dataMapFilter = getFilterPredicates(job.getConfiguration());
-    // this will be null in case of corrupt schema file.
-    PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
 
     if (dataMapFilter != null) {
       dataMapFilter.resolve(false);
     }
-    // prune partitions for filter query on partition table
-    BitSet matchedPartitions = null;
-    if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
-      matchedPartitions = setMatchedPartitions(null, dataMapFilter, partitionInfo, null);
-      if (matchedPartitions != null) {
-        if (matchedPartitions.cardinality() == 0) {
-          return new ArrayList<InputSplit>();
-        } else if (matchedPartitions.cardinality() == partitionInfo.getNumPartitions()) {
-          matchedPartitions = null;
-        }
-      }
-    }
 
     // do block filtering and get split
-    List<InputSplit> splits =
-        getSplits(job, dataMapFilter, filteredSegmentToAccess, matchedPartitions, partitionInfo,
-            null, updateStatusManager, segments.getInvalidSegments());
+    List<InputSplit> splits = getSplits(
+        job, dataMapFilter, filteredSegmentToAccess,
+        updateStatusManager, segments.getInvalidSegments());
+
     // add all splits of streaming
     List<InputSplit> splitsOfStreaming = getSplitsOfStreaming(job, streamSegments, carbonTable);
     if (!splitsOfStreaming.isEmpty()) {
@@ -222,12 +201,11 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
    * For Pre Aggregate rules it will set all the valid segments for both streaming and
    * and normal for fact table, so if any handoff happened in between it will
    * select only new hand off segments segments for fact.
-   * @param job
    * @param validSegments
    * @param segmentsToAccess
    * @return
    */
-  private List<Segment> getFilteredNormalSegments(JobContext job, List<Segment> validSegments,
+  private List<Segment> getFilteredNormalSegments(List<Segment> validSegments,
       Segment[] segmentsToAccess) {
     List<Segment> segmentToAccessSet = Arrays.asList(segmentsToAccess);
     List<Segment> filteredSegment = new ArrayList<>();
@@ -244,7 +222,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
    * `INPUT_SEGMENT_NUMBERS` in job configuration
    */
   private List<Segment> getFilteredSegment(JobContext job, List<Segment> validSegments,
-      boolean validationRequired, ReadCommittedScope readCommittedScope) {
+      ReadCommittedScope readCommittedScope) {
     Segment[] segmentsToAccess = getSegmentsToAccess(job, readCommittedScope);
     List<Segment> segmentToAccessSet =
         new ArrayList<>(new HashSet<>(Arrays.asList(segmentsToAccess)));
@@ -343,91 +321,6 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
   }
 
   /**
-   * Read data in one segment. For alter table partition statement
-   * @param job
-   * @param targetSegment
-   * @param oldPartitionIdList  get old partitionId before partitionInfo was changed
-   * @return
-   */
-  public List<InputSplit> getSplitsOfOneSegment(JobContext job, String targetSegment,
-      List<Integer> oldPartitionIdList, PartitionInfo partitionInfo) {
-    try {
-      carbonTable = getOrCreateCarbonTable(job.getConfiguration());
-      ReadCommittedScope readCommittedScope =
-          getReadCommitted(job, carbonTable.getAbsoluteTableIdentifier());
-      this.readCommittedScope = readCommittedScope;
-
-      List<Segment> segmentList = new ArrayList<>();
-      Segment segment = Segment.getSegment(targetSegment, carbonTable.getTablePath());
-      segmentList.add(
-          new Segment(segment.getSegmentNo(), segment.getSegmentFileName(), readCommittedScope));
-      setSegmentsToAccess(job.getConfiguration(), segmentList);
-
-      // process and resolve the expression
-      DataMapFilter filter = getFilterPredicates(job.getConfiguration());
-      CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
-      // this will be null in case of corrupt schema file.
-      if (null == carbonTable) {
-        throw new IOException("Missing/Corrupt schema file for table.");
-      }
-      if (filter != null) {
-        filter.processFilterExpression();
-      }
-      // prune partitions for filter query on partition table
-      String partitionIds = job.getConfiguration().get(ALTER_PARTITION_ID);
-      // matchedPartitions records partitionIndex, not partitionId
-      BitSet matchedPartitions = null;
-      if (partitionInfo != null) {
-        matchedPartitions =
-            setMatchedPartitions(partitionIds, filter, partitionInfo, oldPartitionIdList);
-        if (matchedPartitions != null) {
-          if (matchedPartitions.cardinality() == 0) {
-            return new ArrayList<InputSplit>();
-          } else if (matchedPartitions.cardinality() == partitionInfo.getNumPartitions()) {
-            matchedPartitions = null;
-          }
-        }
-      }
-
-      // do block filtering and get split
-      List<InputSplit> splits = getSplits(job, filter, segmentList, matchedPartitions,
-          partitionInfo, oldPartitionIdList, new SegmentUpdateStatusManager(carbonTable),
-          new ArrayList<Segment>());
-      return splits;
-    } catch (IOException e) {
-      throw new RuntimeException("Can't get splits of the target segment ", e);
-    }
-  }
-
-  /**
-   * set the matched partition indices into a BitSet
-   * @param partitionIds  from alter table command, for normal query, it's null
-   * @param filter   from query
-   * @param partitionInfo
-   * @param oldPartitionIdList  only used in alter table command
-   * @return
-   */
-  private BitSet setMatchedPartitions(String partitionIds, DataMapFilter filter,
-      PartitionInfo partitionInfo, List<Integer> oldPartitionIdList) {
-    BitSet matchedPartitions = null;
-    if (null != partitionIds) {
-      String[] partList = partitionIds.replace("[", "").replace("]", "").split(",");
-      // partList[0] -> use the first element to initiate BitSet, will auto expand later
-      matchedPartitions = new BitSet(Integer.parseInt(partList[0].trim()));
-      for (String partitionId : partList) {
-        Integer index = oldPartitionIdList.indexOf(Integer.parseInt(partitionId.trim()));
-        matchedPartitions.set(index);
-      }
-    } else {
-      if (null != filter) {
-        matchedPartitions = new FilterExpressionProcessor()
-            .getFilteredPartitions(filter.getExpression(), partitionInfo);
-      }
-    }
-    return matchedPartitions;
-  }
-
-  /**
    * {@inheritDoc}
    * Configurations FileInputFormat.INPUT_DIR, CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS
    * are used to get table path to read.
@@ -436,8 +329,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
    * @throws IOException
    */
   private List<InputSplit> getSplits(JobContext job, DataMapFilter expression,
-      List<Segment> validSegments, BitSet matchedPartitions, PartitionInfo partitionInfo,
-      List<Integer> oldPartitionIdList, SegmentUpdateStatusManager updateStatusManager,
+      List<Segment> validSegments, SegmentUpdateStatusManager updateStatusManager,
       List<Segment> invalidSegments) throws IOException {
 
     List<String> segmentsToBeRefreshed = new ArrayList<>();
@@ -455,14 +347,14 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
     numSegments = validSegments.size();
     List<InputSplit> result = new LinkedList<InputSplit>();
     UpdateVO invalidBlockVOForSegmentId = null;
-    Boolean isIUDTable = false;
+    boolean isIUDTable = false;
 
     isIUDTable = (updateStatusManager.getUpdateStatusDetails().length != 0);
 
     // for each segment fetch blocks matching filter in Driver BTree
     List<org.apache.carbondata.hadoop.CarbonInputSplit> dataBlocksOfSegment =
-        getDataBlocksOfSegment(job, carbonTable, expression, matchedPartitions, validSegments,
-            partitionInfo, oldPartitionIdList, invalidSegments, segmentsToBeRefreshed);
+        getDataBlocksOfSegment(job, carbonTable, expression, validSegments,
+            invalidSegments, segmentsToBeRefreshed);
     numBlocks = dataBlocksOfSegment.size();
     for (org.apache.carbondata.hadoop.CarbonInputSplit inputSplit : dataBlocksOfSegment) {
 
@@ -535,7 +427,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
 
     // TODO: currently only batch segment is supported, add support for streaming table
     List<Segment> filteredSegment =
-        getFilteredSegment(job, allSegments.getValidSegments(), false, readCommittedScope);
+        getFilteredSegment(job, allSegments.getValidSegments(), readCommittedScope);
     boolean isIUDTable = (updateStatusManager.getUpdateStatusDetails().length != 0);
     /* In the select * flow, getSplits() method was clearing the segmentMap if,
     segment needs refreshing. same thing need for select count(*) flow also.
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
index f4c6c22..c63aa68 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
@@ -40,10 +40,6 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
   protected Dictionary[] dictionaries;
 
   protected DataType[] dataTypes;
-  /**
-   * carbon columns
-   */
-  protected CarbonColumn[] carbonColumns;
 
   /**
    * This initialization is done inside executor task
@@ -55,7 +51,6 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
   @Override
   public void initialize(CarbonColumn[] carbonColumns,
       CarbonTable carbonTable) throws IOException {
-    this.carbonColumns = carbonColumns;
     dictionaries = new Dictionary[carbonColumns.length];
     dataTypes = new DataType[carbonColumns.length];
     for (int i = 0; i < carbonColumns.length; i++) {
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala
index ed803ca..14045a6 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala
@@ -177,15 +177,6 @@ class BatchSortLoad3TestCase extends QueryTest with BeforeAndAfterAll {
     sql(s"""drop table if exists  uniqdata_c""").collect
   }
 
-
-  //Batch_sort_Loading_001-01-01-01_001-TC_064
-//  test("Batch_sort_Loading_001-01-01-01_001-TC_064", Include) {
-//    sql(s"""drop table if exists uniqdata""").collect
-//    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='20160302,20150302')""").collect
-//    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/7000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',','QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-//    sql(s"""drop table if exists uniqdata""").collect
-//  }
-
   override def afterAll {
   }
 }
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionTestCase.scala
deleted file mode 100644
index 31ec14e..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionTestCase.scala
+++ /dev/null
@@ -1,520 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util._
-import org.scalatest.BeforeAndAfterAll
-
-/**
- * Test Class for partitionTestCase to verify all scenerios
- */
-
-class PartitionTestCase extends QueryTest with BeforeAndAfterAll {
-         
-
-  //Verify exception if column in partitioned by is already specified in table schema
-  test("Partition-Local-sort_TC001", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (INTEGER_COLUMN1 int)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='List','LIST_INFO'='1,3')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify table is created with Partition
-  ignore("Partition-Local-sort_TC002", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST','LIST_INFO'='3')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception partitioned by is not specified in the DDL
-  test("Partition-Local-sort_TC003", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='List','NUM_PARTITIONS'='3')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if List info is not given with List type partition
-  test("Partition-Local-sort_TC004", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='List')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //exception should not be thrown if Partition type is not given
-  test("Partition-Local-sort_TC005", Include) {
-    try {
-      sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('LIST_INFO'='1,2')""").collect
-      sql(s"""drop table if exists uniqdata""").collect
-    } catch {
-      case _ => assert(false)
-    }
-  }
-
-
-  //Verify exception if Partition type is 'range' and LIST_INFO Is provided
-  test("Partition-Local-sort_TC006", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'LIST_INFO'='1,2')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if Partition type is 'range' and NUM_PARTITIONS Is provided
-  test("Partition-Local-sort_TC007", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'NUM_PARTITIONS'='1')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify table is created if Partition type is 'range' and RANGE_INFO Is provided
-  ignore("Partition-Local-sort_TC008", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='20160302,20150302')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify table is created if Partition type is 'LIST' and LIST_INFO Is provided
-  test("Partition-Local-sort_TC009", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double) PARTITIONED BY (DOJ int)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,2')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if Partition type is 'LIST' and NUM_PARTITIONS Is provided
-  test("Partition-Local-sort_TC010", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ int)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'NUM_PARTITIONS'='1')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if Partition type is 'LIST' and RANGE_INFO Is provided
-  test("Partition-Local-sort_TC011", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'RANGE_INFO'='20160302,20150302')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if datatype is not provided with partition column
-  test("Partition-Local-sort_TC012", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='20160302,20150302')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if a non existent file header  is provided in partition
-  test("Partition-Local-sort_TC013", Include) {
-    intercept[Exception] {
-      sql(s"""drop table if exists uniqdata""").collect
-      sql(
-        s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='20160302,20150302')
-
-  LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOJ,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    }
-    sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if Partition By Is empty
-  test("Partition-Local-sort_TC014", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY ()STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')
-  """).collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify load with List Partition
-  test("Partition-Local-sort_TC015", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(28)), "partitionTestCase_Partition-Local-sort_TC015")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify load with List Partition and limit 1
-  ignore("Partition-Local-sort_TC016", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select * from uniqdata limit 1""",
-      Seq(Row("CUST_NAME_00002","ACTIVE_EMUI_VERSION_00002",null,null,null,12345678903.0000000000,22345678903.0000000000,1.123456749E10,-1.123456749E10,3,null,2)), "partitionTestCase_Partition-Local-sort_TC016")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify load with List Partition and select partition column
-  test("Partition-Local-sort_TC017", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select CUST_ID from uniqdata order by CUST_ID limit 1""",
-      Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC017")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception if 2 partition columns are provided
-  test("Partition-Local-sort_TC018", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""
-  CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (CUST_ID int , DOJ timestamp) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with range partition with limit 1
-  ignore("Partition-Local-sort_TC019", Include) {
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='0,5,10,29')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select * from uniqdata limit 1""",
-      Seq(Row("CUST_NAME_00003","ACTIVE_EMUI_VERSION_00003",null,null,null,12345678904.0000000000,22345678904.0000000000,1.123456749E10,-1.123456749E10,4,null,5)), "partitionTestCase_Partition-Local-sort_TC019")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with range partition
-  ignore("Partition-Local-sort_TC020", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='0,5,10,29')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    sql(s"""select count(*) from uniqdata limit 1""").collect
-    sql(s"""Seq(Row(28))""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with hash partition with limit 1
-  ignore("Partition-Local-sort_TC021", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='5')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select * from uniqdata limit 1""",
-      Seq(Row("CUST_NAME_00003","ACTIVE_EMUI_VERSION_00003",null,null,null,12345678904.0000000000,22345678904.0000000000,1.123456749E10,-1.123456749E10,4,null,5)), "partitionTestCase_Partition-Local-sort_TC021")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with hash partition
-  ignore("Partition-Local-sort_TC022", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='5')""").collect
-
-   sql(s"""LOAD DATA INPATH  ''$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata limit 1""",
-      Seq(Row(28)), "partitionTestCase_Partition-Local-sort_TC022")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with List partition after compaction
-  test("Partition-Local-sort_TC023", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""alter table uniqdata compact 'minor'""").collect
-    checkAnswer(s"""select count(*) from uniqdata limit 1""",
-      Seq(Row(112)), "partitionTestCase_Partition-Local-sort_TC023")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with Range partition after compaction
-  test("Partition-Local-sort_TC024", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='Range', 'RANGE_INFO'='0,5,10,30')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""alter table uniqdata compact 'minor'""").collect
-    checkAnswer(s"""select count(*) from uniqdata limit 1""",
-      Seq(Row(112)), "partitionTestCase_Partition-Local-sort_TC024")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data load with Hash partition after compaction
-  test("Partition-Local-sort_TC025", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='5')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""alter table uniqdata compact 'minor'""").collect
-    checkAnswer(s"""select count(*) from uniqdata limit 1""",
-      Seq(Row(112)), "partitionTestCase_Partition-Local-sort_TC025")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify join operation on List partition
-  test("Partition-Local-sort_TC026", Include) {
-     sql(s"""drop table if exists uniqdata1""").collect
-   sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-
-   sql(s"""CREATE TABLE uniqdata1 (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata1 OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    sql(s"""select a.cust_id, b.cust_id from uniqdata a, uniqdata1 b where a.cust_id > b.cust_id""").collect
-
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify data when sublist is provided in LIST_INFO
-  test("Partition-Local-sort_TC028", Include) {
-     sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,(1,2),3')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(28)), "partitionTestCase_Partition-Local-sort_TC028")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify exception is thrown if partition column is dropped
-  test("Partition-Local-sort_TC029", Include) {
-    intercept[Exception] {
-       sql(s"""drop table if exists uniqdata""").collect
-      sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')
-
-  alter table uniqdata drop columns(CUST_ID)
-
-  """).collect
-    }
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify insert is successful on list partition
-  ignore("Partition-Local-sort_TC030", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-   sql(s"""insert into table uniqdata values ('a', '1','2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 1)""").collect
-   sql(s"""insert into table uniqdata values ('a', '1', '2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 0)""").collect
-    checkAnswer(s"""select * from uniqdata""",
-      Seq(Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",1),Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",0)), "partitionTestCase_Partition-Local-sort_TC030")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify insert is successful on range partition
-  ignore("Partition-Local-sort_TC031", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='0,3,5')""").collect
-   sql(s"""insert into table uniqdata values ('a', '1','2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 1)""").collect
-   sql(s"""insert into table uniqdata values ('a', '1', '2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 0)""").collect
-    checkAnswer(s"""select * from uniqdata""",
-      Seq(Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",1),Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",0)), "partitionTestCase_Partition-Local-sort_TC031")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify insert is successful on HASH partition
-  ignore("Partition-Local-sort_TC032", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='10')""").collect
-   sql(s"""insert into table uniqdata values ('a', '1','2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 1)""").collect
-   sql(s"""insert into table uniqdata values ('a', '1', '2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 0)""").collect
-    sql(s"""select * from uniqdata""").collect
-    sql(s"""Seq(Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",1),Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",0))""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with > filter condition and list partition
-  test("Partition-Local-sort_TC033", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,0,3,4')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID>3""",
-      Seq(Row(4)), "partitionTestCase_Partition-Local-sort_TC033")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with = filter condition and list partition
-  test("Partition-Local-sort_TC034", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,0,3,4')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID=3""",
-      Seq(Row(8)), "partitionTestCase_Partition-Local-sort_TC034")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with = value not in list_info and list partition
-  test("Partition-Local-sort_TC035", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,0,3,4')""").collect
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID=10""",
-      Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC035")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with > filter condition and range partition
-  ignore("Partition-Local-sort_TC036", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1,0,3,4')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID>3""",
-      Seq(Row(4)), "partitionTestCase_Partition-Local-sort_TC036")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with = filter condition and list partition
-  ignore("Partition-Local-sort_TC037", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1,0,3,4')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID=3""",
-      Seq(Row(8)), "partitionTestCase_Partition-Local-sort_TC037")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with = value not in list_info and list partition
-  ignore("Partition-Local-sort_TC038", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1,0,3,4')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID=10""",
-      Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC038")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with > filter condition and hash partition
-  ignore("Partition-Local-sort_TC039", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='1,0,3,4')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID>3""",
-      Seq(Row(4)), "partitionTestCase_Partition-Local-sort_TC039")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with = filter condition and hash partition
-  ignore("Partition-Local-sort_TC040", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='1,0,3,4')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID=3""",
-      Seq(Row(8)), "partitionTestCase_Partition-Local-sort_TC040")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //Verify date with = value not in list_info and hash partition
-  ignore("Partition-Local-sort_TC041", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-
-   sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='1,0,3,4')""").collect
-    checkAnswer(s"""select count(*) from uniqdata where CUST_ID=10""",
-      Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC041")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-  override def afterAll {
-    sql("drop table if exists uniqdata")
-  }
-}
\ No newline at end of file
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
index b11a57c..812aab5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
@@ -152,7 +152,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
                 """.stripMargin)
         }
         assert(exception.getMessage.contains(
-            "LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: binaryfield is not a string/complex/varchar datatype column. " +
+            "LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: binaryField is not a string/complex/varchar datatype column. " +
                     "LOCAL_DICTIONARY_COLUMN should be no dictionary string/complex/varchar datatype column"))
     }
 
@@ -190,7 +190,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
                 """.stripMargin)
         }
         assert(exception.getMessage.contains(
-            "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryfield"))
+            "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryField"))
     }
 
     test("Unsupport DICTIONARY_INCLUDE for binary, multiple column") {
@@ -210,7 +210,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
                 """.stripMargin)
         }
         assert(exception.getMessage.contains(
-            "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryfield"))
+            "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryField"))
     }
 
     test("Supports DICTIONARY_EXCLUDE for binary") {
@@ -262,7 +262,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
                   | tblproperties('inverted_index'='binaryField','SORT_COLUMNS'='binaryField')
                 """.stripMargin)
         }
-        assert(exception.getMessage.contains("sort_columns is unsupported for binary datatype column: binaryfield"))
+        assert(exception.getMessage.contains("sort_columns is unsupported for binary datatype column: binaryField"))
     }
 
     test("COLUMN_META_CACHE doesn't support binary") {
@@ -1583,61 +1583,6 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             Seq(Row("binary"), Row("1"), Row("binary"), Row("1"), Row("binary"), Row("1")))
     }
 
-    test("Create table and load data with binary column for partition") {
-        sql("DROP TABLE IF EXISTS binaryTable")
-        sql(
-            s"""
-               | CREATE TABLE IF NOT EXISTS binaryTable (
-               |    id int,
-               |    label boolean,
-               |    name string,
-               |    autoLabel boolean)
-               | PARTITIONED BY(binaryfield binary)
-               | STORED BY 'carbondata'
-               | TBLPROPERTIES('SORT_COLUMNS'='','PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
-             """.stripMargin)
-        sql(
-            s"""
-               | LOAD DATA LOCAL INPATH '$resourcesPath/binarystringdatawithHead.csv'
-               | INTO TABLE binaryTable
-               | partition(binaryfield)
-               | OPTIONS('header'='true','DELIMITER'='|')
-             """.stripMargin)
-
-        val result = sql("desc formatted binaryTable").collect()
-        var flag = false
-        result.foreach { each =>
-            if ("binary".equals(each.get(1))) {
-                flag = true
-            }
-        }
-        assert(flag)
-
-        checkAnswer(sql("SELECT COUNT(*) FROM binaryTable"), Seq(Row(3)))
-        try {
-            val df = sql("SELECT * FROM binaryTable").collect()
-            assert(3 == df.length)
-
-            df.foreach { each =>
-                assert(5 == each.length)
-                if (2 == each.get(0)) {
-                    assert("binary".equals(new String(each.getAs[Array[Byte]](4))))
-                } else if (1 == each.get(0)) {
-                    assert("Hello world".equals(new String(each.getAs[Array[Byte]](4))))
-                } else if (3 == each.get(0)) {
-                    assert("1".equals(new String(each.getAs[Array[Byte]](4))))
-                } else {
-                    assert(false)
-                }
-            }
-
-        } catch {
-            case e: Exception =>
-                e.printStackTrace()
-                assert(false)
-        }
-    }
-
     test("Select query with average function for substring of binary column is executed.") {
         sql("DROP TABLE IF EXISTS uniqdata")
         sql(
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
index b12e269..9fbc061 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
@@ -546,7 +546,7 @@ class TestLoadDataWithCompression extends QueryTest with BeforeAndAfterEach with
     checkAnswer(sql(s"SELECT count(*) FROM $tableName"), Seq(Row(8)))
     val carbonTable = CarbonEnv.getCarbonTable(Option("default"), tableName)(sqlContext.sparkSession)
     val tableColumnCompressor = carbonTable.getTableInfo.getFactTable.getTableProperties.get(CarbonCommonConstants.COMPRESSOR)
-    assertResult(compressorName.toLowerCase())(tableColumnCompressor)
+    assertResult(compressorName)(tableColumnCompressor)
 
     sql(s"DROP TABLE IF EXISTS $tableName")
     // reuse the customize compressor again
@@ -555,7 +555,7 @@ class TestLoadDataWithCompression extends QueryTest with BeforeAndAfterEach with
     checkAnswer(sql(s"SELECT count(*) FROM $tableName"), Seq(Row(8)))
     val carbonTable2 = CarbonEnv.getCarbonTable(Option("default"), tableName)(sqlContext.sparkSession)
     val tableColumnCompressor2 = carbonTable2.getTableInfo.getFactTable.getTableProperties.get(CarbonCommonConstants.COMPRESSOR)
-    assertResult(compressorName.toLowerCase())(tableColumnCompressor2)
+    assertResult(compressorName)(tableColumnCompressor2)
   }
 
   private def generateAllDataTypeFiles(lineNum: Int, csvDir: String,
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
index 473c9c6..5e10a74 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
@@ -461,7 +461,7 @@ class TestCreateDDLForComplexMapType extends QueryTest with BeforeAndAfterAll {
     }
     assert(exception1.getMessage
       .contains(
-        "sort_columns is unsupported for map datatype column: mapfield"))
+        "sort_columns is unsupported for map datatype column: mapField"))
   }
 
   test("Data Load Fail Issue") {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
index 6059cb6..0815eab 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
@@ -153,20 +153,6 @@ class TestCreateTableLike extends QueryTest with BeforeAndAfterEach with BeforeA
     checkTableProperties(TableIdentifier("sourceTable"), TableIdentifier("targetTable"))
   }
 
-  test("table with partition") {
-    sql("""
-          | CREATE TABLE IF NOT EXISTS pt_tbl(
-          |   a int, b string
-          | )
-          | PARTITIONED BY (area string)
-          | STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-          | 'LIST_INFO'='Asia, America, Europe')
-        """.stripMargin)
-    sql("create table targetTable like pt_tbl")
-    checkTableProperties(TableIdentifier("pt_tbl"), TableIdentifier("targetTable"))
-  }
-
   test("table with hive partition") {
     sql(
       """
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala
index dac9124..110319a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala
@@ -45,7 +45,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
          | TBLPROPERTIES('SORT_COLUMNS'='stringField', 'SORT_SCOPE'='GLOBAL_SORT')
        """.stripMargin)
 
-    checkExistence(sql("DESCRIBE FORMATTED tableWithGlobalSort"), true, "global_sort")
+    checkExistence(sql("DESCRIBE FORMATTED tableWithGlobalSort"), true, "GLOBAL_SORT")
 
     sql(
       s"""
@@ -59,7 +59,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
 
     sql("DESCRIBE FORMATTED tableWithLocalSort")
 
-    checkExistence(sql("DESCRIBE FORMATTED tableWithLocalSort"), true, "local_sort")
+    checkExistence(sql("DESCRIBE FORMATTED tableWithLocalSort"), true, "LOCAL_SORT")
 
     sql(
       s"""
@@ -71,7 +71,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
          | TBLPROPERTIES('SORT_COLUMNS'='stringField', 'SORT_SCOPE'='BATCH_SORT')
        """.stripMargin)
 
-    checkExistence(sql("DESCRIBE FORMATTED tableWithBatchSort"), true, "batch_sort")
+    checkExistence(sql("DESCRIBE FORMATTED tableWithBatchSort"), true, "BATCH_SORT")
 
     sql(
       s"""
@@ -83,7 +83,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
          | TBLPROPERTIES('SORT_COLUMNS'='stringField', 'SORT_SCOPE'='NO_SORT')
        """.stripMargin)
 
-    checkExistence(sql("DESCRIBE FORMATTED tableWithNoSort"), true, "no_sort")
+    checkExistence(sql("DESCRIBE FORMATTED tableWithNoSort"), true, "NO_SORT")
   }
 
   test("test create table with sort scope in abnormal cases") {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
index 557e748..cfd23ed 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
@@ -709,11 +709,11 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
       .contains("Unsupported operation on non transactional table"))
 
     //9. Show partition
-    exception = intercept[MalformedCarbonCommandException] {
+    val ex = intercept[AnalysisException] {
       sql("Show partitions sdkOutputTable").show(false)
     }
-    assert(exception.getMessage()
-      .contains("Unsupported operation on non transactional table"))
+    assert(ex.getMessage()
+      .contains("SHOW PARTITIONS is not allowed on a table that is not partitioned"))
 
     //12. Streaming table creation
     // No need as External table don't accept table properties
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
index e31896f..637611b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
@@ -60,7 +60,7 @@ class CompactionSupportGlobalSortBigFileTest extends QueryTest with BeforeAndAft
       """
         | CREATE TABLE compaction_globalsort(id INT, name STRING, city STRING, age INT)
         | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='global_sort')
       """.stripMargin)
 
     sql("DROP TABLE IF EXISTS carbon_localsort")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
index ed0f466..dd54971 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
@@ -42,7 +42,7 @@ class CompactionSupportGlobalSortFunctionTest extends QueryTest with BeforeAndAf
       """
         | CREATE TABLE compaction_globalsort(id INT, name STRING, city STRING, age INT)
         | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT', 'GLOBAL_SORT_PARTITIONS'='1')
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='global_sort', 'GLOBAL_SORT_PARTITIONS'='1')
       """.stripMargin)
 
     sql("DROP TABLE IF EXISTS carbon_localsort")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
index a1d0fe0..1563811 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
@@ -43,7 +43,7 @@ class CompactionSupportGlobalSortParameterTest extends QueryTest with BeforeAndA
       """
         | CREATE TABLE compaction_globalsort(id INT, name STRING, city STRING, age INT)
         | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT', 'GLOBAL_SORT_PARTITIONS'='1')
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='global_sort', 'GLOBAL_SORT_PARTITIONS'='1')
       """.stripMargin)
 
     sql("DROP TABLE IF EXISTS carbon_localsort")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 255f399..4a0e74f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -138,20 +138,6 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
   }
 
   // ----------------------------------- Configuration Validity -----------------------------------
-  test("Don't support GLOBAL_SORT on partitioned table") {
-    sql("DROP TABLE IF EXISTS carbon_globalsort_partitioned")
-    sql(
-      """
-        | CREATE TABLE carbon_globalsort_partitioned(name STRING, city STRING, age INT)
-        | PARTITIONED BY (id INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3', 'SORT_SCOPE'='GLOBAL_SORT', 'sort_columns' = 'name, city')
-      """.stripMargin)
-
-    intercept[MalformedCarbonCommandException] {
-      sql(s"LOAD DATA LOCAL INPATH '$filePath' INTO TABLE carbon_globalsort_partitioned")
-    }
-  }
 
   test("Number of partitions should be greater than 0") {
     intercept[MalformedCarbonCommandException] {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
index 724e9f4..8793608 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
@@ -85,7 +85,7 @@ class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with Bef
     }
     assert(e.getMessage
       .equals(
-        "DICTIONARY_EXCLUDE column: ccc does not exist in table or unsupported for complex child " +
+        "DICTIONARY_EXCLUDE column: CCC does not exist in table or unsupported for complex child " +
         "column. Please check the create table statement."))
   }
 
@@ -95,7 +95,7 @@ class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with Bef
     }
     assert(e.getMessage
       .equals(
-        "DICTIONARY_INCLUDE column: aaa does not exist in table or unsupported for complex child " +
+        "DICTIONARY_INCLUDE column: AAA does not exist in table or unsupported for complex child " +
         "column. Please check the create table statement."))
   }
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala
deleted file mode 100644
index 82c6d48..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala
+++ /dev/null
@@ -1,1238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.testsuite.partition
-
-import java.sql.{Date, Timestamp}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.Row
-import org.scalatest.BeforeAndAfterAll
-import scala.collection.mutable
-
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.apache.spark.sql.test.util.QueryTest
-
-class TestAllDataTypeForPartitionTable extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
-
-    dropTable
-  }
-
-  override def afterAll = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
-    dropTable
-  }
-
-  def dropTable = {
-    sql("drop table if exists allTypeTable_hash_smallInt")
-    sql("drop table if exists allTypeTable_hash_int")
-    sql("drop table if exists allTypeTable_hash_bigint")
-    sql("drop table if exists allTypeTable_hash_float")
-    sql("drop table if exists allTypeTable_hash_double")
-    sql("drop table if exists allTypeTable_hash_decimal")
-    sql("drop table if exists allTypeTable_hash_timestamp")
-    sql("drop table if exists allTypeTable_hash_date")
-    sql("drop table if exists allTypeTable_hash_string")
-    sql("drop table if exists allTypeTable_hash_varchar")
-    sql("drop table if exists allTypeTable_hash_char")
-
-    sql("drop table if exists allTypeTable_list_smallInt")
-    sql("drop table if exists allTypeTable_list_int")
-    sql("drop table if exists allTypeTable_list_bigint")
-    sql("drop table if exists allTypeTable_list_float")
-    sql("drop table if exists allTypeTable_list_double")
-    sql("drop table if exists allTypeTable_list_decimal")
-    sql("drop table if exists allTypeTable_list_timestamp")
-    sql("drop table if exists allTypeTable_list_date")
-    sql("drop table if exists allTypeTable_list_string")
-    sql("drop table if exists allTypeTable_list_varchar")
-    sql("drop table if exists allTypeTable_list_char")
-
-    sql("drop table if exists allTypeTable_range_smallInt")
-    sql("drop table if exists allTypeTable_range_int")
-    sql("drop table if exists allTypeTable_range_bigint")
-    sql("drop table if exists allTypeTable_range_float")
-    sql("drop table if exists allTypeTable_range_double")
-    sql("drop table if exists allTypeTable_range_decimal")
-    sql("drop table if exists allTypeTable_range_timestamp")
-    sql("drop table if exists allTypeTable_range_date")
-    sql("drop table if exists allTypeTable_range_string")
-    sql("drop table if exists allTypeTable_range_varchar")
-    sql("drop table if exists allTypeTable_range_char")
-  }
-
-
-  test("allTypeTable_hash_smallInt") {
-    val tableName = "allTypeTable_hash_smallInt"
-
-    sql(
-      s"""create table $tableName(
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(smallIntField smallInt)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = -32768"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 128"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 32767"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_int") {
-    val tableName = "allTypeTable_hash_int"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(intField int)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = -2147483648"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 32768"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 2147483647"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_bigint") {
-    val tableName = "allTypeTable_hash_bigint"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(bigIntField bigint)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = -9223372036854775808"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 2147483648"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 9223372036854775807"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_float") {
-    val tableName = "allTypeTable_hash_float"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(floatField float)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = -2147483648.1"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483647.1"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483648.1"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_double") {
-    val tableName = "allTypeTable_hash_double"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(doubleField double)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = -9223372036854775808.1"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = 9223372036854775807.1"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2")),
-        Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-
-  }
-
-  test("allTypeTable_hash_decimal") {
-    val tableName = "allTypeTable_hash_decimal"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(decimalField decimal(25, 4))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('-9223372036854775808.1234' as decimal(25, 4))"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775807.1234' as decimal(25, 4))"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775808.1234' as decimal(25, 4))"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_timestamp") {
-    val tableName = "allTypeTable_hash_timestamp"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(timestampField timestamp)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-11 00:00:01'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-12 23:59:59'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-13 23:59:59'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  ignore("allTypeTable_hash_date") {
-    val tableName = "allTypeTable_hash_date"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(dateField date)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-11'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-12'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-13'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_string") {
-    val tableName = "allTypeTable_hash_string"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(stringField string)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_varchar") {
-    val tableName = "allTypeTable_hash_varchar"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(varcharField varchar(10))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_hash_char") {
-    val tableName = "allTypeTable_hash_char"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(charField char(10))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='hash','num_partitions'='3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_smallInt") {
-    val tableName = "allTypeTable_list_smallInt"
-
-    sql(
-      s"""create table $tableName(
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(smallIntField smallInt)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='-32768, 32767')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = -32768"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 128"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 32767"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_int") {
-    val tableName = "allTypeTable_list_int"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(intField int)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='-2147483648, 2147483647')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = -2147483648"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 32768"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 2147483647"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_bigint") {
-    val tableName = "allTypeTable_list_bigint"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(bigIntField bigint)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='-9223372036854775808, 9223372036854775807')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = -9223372036854775808"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 2147483648"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 9223372036854775807"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_float") {
-    val tableName = "allTypeTable_list_float"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(floatField float)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='-2147483648.1,2147483648.1')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = -2147483648.1"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483647.1"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483648.1"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_double") {
-    val tableName = "allTypeTable_list_double"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(doubleField double)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='-9223372036854775808.1,9223372036854775808.1')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = -9223372036854775808.1"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = 9223372036854775807.1"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2")),
-        Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-
-  }
-
-  test("allTypeTable_list_decimal") {
-    val tableName = "allTypeTable_list_decimal"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(decimalField decimal(25, 4))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='-9223372036854775808.1234, 9223372036854775808.1234')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('-9223372036854775808.1234' as decimal(25, 4))"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775807.1234' as decimal(25, 4))"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775808.1234' as decimal(25, 4))"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_timestamp") {
-    val tableName = "allTypeTable_list_timestamp"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(timestampField timestamp)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='2017-06-11 00:00:01, 2017-06-13 23:59:59')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-11 00:00:01'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-12 23:59:59'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-13 23:59:59'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_date") {
-    val tableName = "allTypeTable_list_date"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(dateField date)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='2017-06-11,2017-06-13')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-11'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-12'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-13'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_string") {
-    val tableName = "allTypeTable_list_string"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(stringField string)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='abc1,abc2')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_varchar") {
-    val tableName = "allTypeTable_list_varchar"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(varcharField varchar(10))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='abcd1,abcd3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_list_char") {
-    val tableName = "allTypeTable_list_char"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(charField char(10))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='list','list_info'='abcde1,abcde3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_smallInt") {
-    val tableName = "allTypeTable_range_smallInt"
-
-    sql(
-      s"""create table $tableName(
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(smallIntField smallInt)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='0, 129')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = -32768"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 128"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 32767"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_int") {
-    val tableName = "allTypeTable_range_int"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(intField int)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='-1, 32769')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = -2147483648"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 32768"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 2147483647"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_bigint") {
-    val tableName = "allTypeTable_range_bigint"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(bigIntField bigint)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='-9223372036854775807, 2147483649')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = -9223372036854775808"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 2147483648"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 9223372036854775807"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_float") {
-    val tableName = "allTypeTable_range_float"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(floatField float)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='-2147483647.1,2147483648.1')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = -2147483648.1"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483647.1"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483648.1"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_double") {
-    val tableName = "allTypeTable_range_double"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(doubleField double)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='-9223372036854775807.1,9223372036854775808.1')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = -9223372036854775808.1"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = 9223372036854775807.1"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2")),
-        Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-
-  }
-
-  test("allTypeTable_range_decimal") {
-    val tableName = "allTypeTable_range_decimal"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(decimalField decimal(25, 4))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='-9223372036854775807.1234, 9223372036854775808.1234')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('-9223372036854775808.1234' as decimal(25, 4))"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775807.1234' as decimal(25, 4))"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775808.1234' as decimal(25, 4))"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_timestamp") {
-    val tableName = "allTypeTable_range_timestamp"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(timestampField timestamp)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='2017-06-11 00:00:02, 2017-06-13 23:59:59')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-11 00:00:01'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-12 23:59:59'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-13 23:59:59'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  ignore("allTypeTable_range_date") {
-    val tableName = "allTypeTable_range_date"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | stringField string,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(dateField date)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='2017-06-12,2017-06-13')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-11'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-12'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-13'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_string") {
-    val tableName = "allTypeTable_range_string"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | varcharField varchar(10),
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(stringField string)
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='abc2,abc3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_varchar") {
-    val tableName = "allTypeTable_range_varchar"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | charField char(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(varcharField varchar(10))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='abcd2,abcd3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-
-  test("allTypeTable_range_char") {
-    val tableName = "allTypeTable_range_char"
-
-    sql(
-      s"""create table $tableName(
-         | smallIntField smallInt,
-         | intField int,
-         | bigIntField bigint,
-         | floatField float,
-         | doubleField double,
-         | decimalField decimal(25, 4),
-         | timestampField timestamp,
-         | dateField date,
-         | stringField string,
-         | varcharField varchar(10),
-         | arrayField array<string>,
-         | structField struct<col1:string, col2:string, col3:string>)
-         | partitioned by(charField char(10))
-         | stored by 'carbondata'
-         | tblproperties('partition_type'='range','range_info'='abcde2,abcde3')
-      """.stripMargin)
-
-    sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
-      "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
-      Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
-      Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
-    checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
-      Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestCompactionForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestCompactionForPartitionTable.scala
deleted file mode 100644
index 0217486..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestCompactionForPartitionTable.scala
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.testsuite.partition
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-class TestCompactionForPartitionTable extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    dropTable
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
-    sql(
-      """
-        | CREATE TABLE originTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-  }
-
-  test("minor compaction") {
-    sql("create table part_minor_compact(a String, b int) partitioned by (c int) stored by 'carbondata' tblproperties('PARTITION_TYPE'='LIST','LIST_INFO'='1,2')")
-    sql("insert into part_minor_compact select 'a', 2, 3 from originTable limit 1")
-    sql("insert into part_minor_compact select 'b', 3, 4 from originTable limit 1")
-    sql("insert into part_minor_compact select 'c', 4, 5 from originTable limit 1")
-    sql("insert into part_minor_compact select 'd', 1, 2 from originTable limit 1")
-
-    checkAnswer(sql("select * from part_minor_compact where c = 4"), Seq(Row("b", 3, 4)))
-
-    sql("alter table part_minor_compact compact 'minor'")
-
-    checkAnswer(sql("select * from part_minor_compact where c = 4"), Seq(Row("b", 3, 4)))
-  }
-
-  test("major compaction") {
-    sql("create table part_major_compact(a String, b int) partitioned by (c int) stored by 'carbondata' tblproperties('PARTITION_TYPE'='LIST','LIST_INFO'='1,2')")
-    sql("insert into part_major_compact select 'a', 2, 3 from originTable limit 1")
-    sql("insert into part_major_compact select 'b', 3, 4 from originTable limit 1")
-    sql("insert into part_major_compact select 'c', 4, 5 from originTable limit 1")
-    sql("insert into part_major_compact select 'd', 1, 2 from originTable limit 1")
-
-    checkAnswer(sql("select * from part_major_compact where c = 4"), Seq(Row("b", 3, 4)))
-
-    sql("alter table part_major_compact compact 'major'")
-
-    checkAnswer(sql("select * from part_major_compact where c = 4"), Seq(Row("b", 3, 4)))
-  }
-
-  override def afterAll = {
-    dropTable
-  }
-
-  def dropTable = {
-    sql("drop table if exists part_minor_compact")
-    sql("drop table if exists part_major_compact")
-    sql("drop table if exists originTable")
-  }
-
-}
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
deleted file mode 100644
index b246839..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ /dev/null
@@ -1,430 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.testsuite.partition
-
-import org.apache.spark.sql.{CarbonEnv, SQLContext}
-import org.scalatest.BeforeAndAfterAll
-
-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.partition.PartitionType
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.util.QueryTest
-
-class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
-    dropTable
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
-  }
-
-  test("create partition table: hash partition") {
-    sql(
-      """
-        | CREATE TABLE default.hashTable (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
-    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)
-  }
-
-  test("create partition table: range partition") {
-    sql(
-      """
-        | CREATE TABLE default.rangeTable (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59', 'DICTIONARY_INCLUDE'='doj')
-      """.stripMargin)
-
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
-    assert(partitionInfo.getPartitionType == PartitionType.RANGE)
-    assert(partitionInfo.getRangeInfo.size == 2)
-    assert(partitionInfo.getRangeInfo.get(0).equals("2017-06-11 00:00:02"))
-    assert(partitionInfo.getRangeInfo.get(1).equals("2017-06-13 23:59:59"))
-  }
-
-  test("create partition table: list partition") {
-    sql(
-      """
-        | CREATE TABLE default.listTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory string)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-    val carbonTable =  CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("workgroupcategory"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.STRING)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
-    assert(partitionInfo.getPartitionType == PartitionType.LIST)
-    assert(partitionInfo.getListInfo.size == 3)
-    assert(partitionInfo.getListInfo.get(0).size == 1)
-    assert(partitionInfo.getListInfo.get(0).get(0).equals("0"))
-    assert(partitionInfo.getListInfo.get(1).size == 1)
-    assert(partitionInfo.getListInfo.get(1).get(0).equals("1"))
-    assert(partitionInfo.getListInfo.get(2).size == 2)
-    assert(partitionInfo.getListInfo.get(2).get(0).equals("2"))
-    assert(partitionInfo.getListInfo.get(2).get(1).equals("3"))
-  }
-
-  test("create partition table: list partition with duplicate value") {
-    intercept[Exception] { sql(
-      """
-        | CREATE TABLE default.listTableError (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory string)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3, 1)')
-      """.stripMargin) }
-  }
-
-  test("test exception if partition column is dropped") {
-    sql("drop table if exists test")
-    sql(
-      "create table test(a int, b string) partitioned by (c int) stored by 'carbondata' " +
-      "tblproperties('PARTITION_TYPE'='LIST','list_info'='0,10,5,20')")
-    intercept[Exception] { sql("alter table test drop columns(c)") }
-  }
-
-  test("test describe formatted for partition column") {
-    sql("drop table if exists des")
-    sql(
-      """create table des(a int, b string) partitioned by (c string) stored by 'carbondata'
-        |tblproperties ('partition_type'='list','list_info'='1,2')""".stripMargin)
-    checkExistence(sql("describe formatted des"), true, "Partition Type")
-    sql("drop table if exists des")
-  }
-
-  test("test exception if hash number is invalid") {
-    sql("DROP TABLE IF EXISTS test_hash_1")
-    val exception_test_hash_1: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_hash_1(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='2.1')
-        """.stripMargin
-      )
-    }
-    assert(exception_test_hash_1.getMessage.contains("Invalid partition definition"))
-
-    sql("DROP TABLE IF EXISTS test_hash_2")
-    val exception_test_hash_2: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_hash_2(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='abc')
-        """.stripMargin
-      )
-    }
-    assert(exception_test_hash_2.getMessage.contains("Invalid partition definition"))
-
-    sql("DROP TABLE IF EXISTS test_hash_3")
-    val exception_test_hash_3: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_hash_3(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='-2.1')
-        """.stripMargin
-      )
-    }
-    assert(exception_test_hash_3.getMessage.contains("Invalid partition definition"))
-  }
-
-
-  test("test exception when values in list_info can not match partition column type") {
-    sql("DROP TABLE IF EXISTS test_list_int")
-    val exception_test_list_int: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_int(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_small")
-    val exception_test_list_small: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_small(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 SMALLINT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_small.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_float")
-    val exception_test_list_float: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_float(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_float.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_double")
-    val exception_test_list_double: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_double(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_double.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_bigint")
-    val exception_test_list_bigint: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_bigint(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 BIGINT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_bigint.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_date")
-    val exception_test_list_date: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_date(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DATE) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_date.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_timestamp")
-    val exception_test_list_timestamp: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_timestamp(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 TIMESTAMP) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_list_timestamp.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_list_decimal")
-    val exception_test_list_decimal: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_decimal(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DECIMAL(25, 4)) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='23.23111,2.32')
-        """.stripMargin)
-    }
-    assert(exception_test_list_decimal.getMessage.contains("Invalid Partition Values"))
-  }
-
-  test("test exception when values in range_info can not match partition column type") {
-    sql("DROP TABLE IF EXISTS test_range_int")
-    val exception_test_range_int: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_int(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_int.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_smallint")
-    val exception_test_range_smallint: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_smallint(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 SMALLINT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_smallint.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_float")
-    val exception_test_range_float: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_float(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_float.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_double")
-    val exception_test_range_double: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_double(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_double.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_bigint")
-    val exception_test_range_bigint: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_bigint(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 BIGINT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_bigint.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_date")
-    val exception_test_range_date: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_date(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DATE) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_date.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_timestamp")
-    val exception_test_range_timestamp: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_timestamp(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 TIMESTAMP) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_timestamp.getMessage.contains("Invalid Partition Values"))
-
-    sql("DROP TABLE IF EXISTS test_range_decimal")
-    val exception_test_range_decimal: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_decimal(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DECIMAL(25, 4)) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
-        """.stripMargin)
-    }
-    assert(exception_test_range_decimal.getMessage.contains("Invalid Partition Values"))
-  }
-
-  test("Invalid Partition Range") {
-    val exceptionMessage: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE default.rangeTableInvalid (empno int, empname String, designation String,
-          |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-          |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-          |  utilization int,salary int)
-          | PARTITIONED BY (doj Timestamp)
-          | STORED BY 'org.apache.carbondata.format'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-          |  'RANGE_INFO'='2017-06-11 00:00:02')
-        """.stripMargin)
-    }
-    assert(exceptionMessage.getMessage
-      .contains("Range info must define a valid range.Please check again!"))
-  }
-
-  test("test number of partitions for default partition") {
-    sql("drop table if exists desc")
-    sql("create table desc(name string) partitioned by (num int) stored by 'carbondata'")
-    sql("insert into desc select 'abc',3")
-    sql("insert into desc select 'abc',5")
-    val descFormatted1 = sql("describe formatted desc").collect
-    descFormatted1.find(_.get(0).toString.contains("Number of Partitions")) match {
-      case Some(row) => assert(row.get(1).toString.contains("2"))
-    }
-  }
-
-  override def afterAll = {
-    dropTable
-  }
-
-  def dropTable = {
-    sql("drop table if exists desc")
-    sql("drop table if exists hashTable")
-    sql("drop table if exists rangeTable")
-    sql("drop table if exists listTable")
-    sql("drop table if exists test")
-    sql("DROP TABLE IF EXISTS test_hash_1")
-    sql("DROP TABLE IF EXISTS test_hash_2")
-    sql("DROP TABLE IF EXISTS test_hash_3")
-    sql("DROP TABLE IF EXISTS test_list_int")
-    sql("DROP TABLE IF EXISTS test_list_smallint")
-    sql("DROP TABLE IF EXISTS test_list_bigint")
-    sql("DROP TABLE IF EXISTS test_list_float")
-    sql("DROP TABLE IF EXISTS test_list_double")
-    sql("DROP TABLE IF EXISTS test_list_date")
-    sql("DROP TABLE IF EXISTS test_list_timestamp")
-    sql("DROP TABLE IF EXISTS test_list_decimal")
-    sql("DROP TABLE IF EXISTS test_range_int")
-    sql("DROP TABLE IF EXISTS test_range_smallint")
-    sql("DROP TABLE IF EXISTS test_range_bigint")
-    sql("DROP TABLE IF EXISTS test_range_float")
-    sql("DROP TABLE IF EXISTS test_range_double")
-    sql("DROP TABLE IF EXISTS test_range_date")
-    sql("DROP TABLE IF EXISTS test_range_timestamp")
-    sql("DROP TABLE IF EXISTS test_range_decimal")
-    sql("DROP TABLE IF EXISTS rangeTableInvalid")
-  }
-
-}
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
deleted file mode 100644
index 0633d7f..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.testsuite.partition
-
-import org.apache.spark.sql.CarbonEnv
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-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.partition.PartitionType
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll = {
-    dropTable
-    }
-
-  test("create partition table: hash partition") {
-    sql(
-      """
-        | CREATE TABLE default.hashTable (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
-    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)
-  }
-
-  test("create partition table: range partition") {
-    sql(
-      """
-        | CREATE TABLE default.rangeTable (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59','DICTIONARY_INCLUDE'='doj')
-      """.stripMargin)
-
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
-    assert(partitionInfo.getPartitionType == PartitionType.RANGE)
-    assert(partitionInfo.getRangeInfo.size == 2)
-    assert(partitionInfo.getRangeInfo.get(0).equals("2017-06-11 00:00:02"))
-    assert(partitionInfo.getRangeInfo.get(1).equals("2017-06-13 23:59:59"))
-  }
-
-  test("create partition table: list partition with timestamp datatype") {
-    sql(
-      """
-        | CREATE TABLE default.listTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (projectenddate Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59',
-        |  'DICTIONARY_INCLUDE'='projectenddate')
-      """.stripMargin)
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
-    assert(partitionInfo.getPartitionType == PartitionType.LIST)
-    assert(partitionInfo.getListInfo.size == 2)
-    assert(partitionInfo.getListInfo.get(0).size == 1)
-    assert(partitionInfo.getListInfo.get(0).get(0).equals("2017-06-11 00:00:02"))
-    assert(partitionInfo.getListInfo.get(1).size == 1)
-    assert(partitionInfo.getListInfo.get(1).get(0).equals("2017-06-13 23:59:59"))
-  }
-
-  test("create partition table: list partition with date datatype") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
-
-    sql(
-      """
-        | CREATE TABLE default.listTableDate (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (projectenddate date)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='2017-06-11 , 2017-06-13')
-      """.stripMargin)
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTableDate")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    assert(partitionInfo != null)
-    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
-    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.DATE)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
-    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
-    assert(partitionInfo.getPartitionType == PartitionType.LIST)
-    assert(partitionInfo.getListInfo.size == 2)
-    assert(partitionInfo.getListInfo.get(0).size == 1)
-    assert(partitionInfo.getListInfo.get(0).get(0).equals("2017-06-11"))
-    assert(partitionInfo.getListInfo.get(1).size == 1)
-    assert(partitionInfo.getListInfo.get(1).get(0).equals("2017-06-13"))
-  }
-
-  test("test exception when values in list_info can not match partition column type") {
-    sql("DROP TABLE IF EXISTS test_list_int")
-    val exception_test_list_int: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_list_int(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,2,(abc,efg)')
-        """.stripMargin)
-    }
-    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
-  }
-
-  test("test exception when partition values in rangeTable are in group ") {
-    sql("DROP TABLE IF EXISTS rangeTable")
-    val exception_test_list_int: Exception = intercept[Exception] {
-      sql(
-        """
-          |CREATE TABLE default.rangeTable (empno int, empname String, designation String,
-          |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-          |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-          |  utilization int,salary int)
-          | PARTITIONED BY (doj Timestamp)
-          | STORED BY 'org.apache.carbondata.format'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-          |  'RANGE_INFO'='2017-06-11 00:00:02, (2017-06-13 23:59:59, 2017-09-13 23:45:59)')
-        """.stripMargin)
-    }
-    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
-  }
-
-  test("test exception when values in rangeTable does not match partition column type") {
-    sql("DROP TABLE IF EXISTS rangeTable")
-    val exception_test_list_int: Exception = intercept[Exception] {
-      sql(
-        """
-          |CREATE TABLE default.rangeTable (empno int, empname String, designation String,
-          |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-          |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-          |  utilization int,salary int)
-          | PARTITIONED BY (doj Timestamp)
-          | STORED BY 'org.apache.carbondata.format'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-          |  'RANGE_INFO'='2017-06-11 00:00:02, abc, 2017-09-13 23:45:59')
-        """.stripMargin)
-    }
-    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
-  }
-
-
-  override def afterAll = {
-    dropTable
-  }
-
-  def dropTable = {
-    sql("drop table if exists hashTable")
-    sql("drop table if exists rangeTable")
-    sql("drop table if exists listTable")
-    sql("drop table if exists listTableDate")
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
deleted file mode 100644
index 723d969..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
+++ /dev/null
@@ -1,409 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.testsuite.partition
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{CarbonMetadata, SegmentFileStore}
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.spark.sql.test.util.QueryTest
-
-import org.apache.carbondata.core.datamap.Segment
-
-class TestDataLoadingForPartitionTable extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    dropTable
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
-    sql(
-      """
-        | CREATE TABLE originTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    sql(
-      """
-        | CREATE TABLE originMultiLoads (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originMultiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originMultiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originMultiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-  }
-
-  def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val segmentDir = carbonTable.getSegmentPath(segmentId)
-
-    val dataFiles = if (FileFactory.isFileExist(segmentDir)) {
-      val carbonFile = FileFactory.getCarbonFile(segmentDir)
-      carbonFile.listFiles(new CarbonFileFilter() {
-        override def accept(file: CarbonFile): Boolean = {
-          return file.getName.endsWith(".carbondata")
-        }
-      })
-    } else {
-      val segment = Segment.getSegment(segmentId, carbonTable.getTablePath)
-      val store = new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
-      store.readIndexFiles(new Configuration(false))
-      store.getIndexFilesMap.asScala.flatMap(_._2.asScala).map(f => FileFactory.getCarbonFile(f)).toArray
-    }
-
-    assert(dataFiles.size == partitions.size)
-
-    dataFiles.foreach { dataFile =>
-      val taskId = CarbonTablePath.DataFileUtil.getTaskNo(dataFile.getName).split("_")(0).toInt
-      assert(partitions.exists(_ == taskId))
-    }
-  }
-
-  test("data loading for partition table: hash partition") {
-    sql(
-      """
-        | CREATE TABLE hashTable (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        |
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    validateDataFiles("default_hashTable", "0", Seq(0, 1, 2))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTable order by empno"),
-      sql("select  empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
-  }
-
-  test("data loading for partition table: range partition") {
-    sql(
-      """
-        | CREATE TABLE rangeTable (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    validateDataFiles("default_rangeTable", "0", Seq(0, 1, 2, 4))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
-  }
-
-  test("data loading for partition table: list partition") {
-    sql(
-      """
-        | CREATE TABLE listTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    validateDataFiles("default_listTable", "0", Seq(2, 3))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-  }
-
-  test("single pass data loading for partition table: hash partition") {
-    sql(
-      """
-        | CREATE TABLE hashTableSinglePass (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE hashTableSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
-    validateDataFiles("default_hashTableSinglePass", "0", Seq(0, 1, 2))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTableSinglePass order by empno"),
-      sql("select  empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
-  }
-
-  test("single pass data loading for partition table: range partition") {
-    sql(
-      """
-        | CREATE TABLE rangeTableSinglePass (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTableSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
-    validateDataFiles("default_rangeTableSinglePass", "0", Seq(0, 1, 2, 4))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableSinglePass order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
-  }
-
-  test("single pass data loading for partition table: list partition") {
-    sql(
-      """
-        | CREATE TABLE listTableSinglePass (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
-    validateDataFiles("default_listTableSinglePass", "0", Seq(2, 3))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableSinglePass order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-  }
-
-  test("Insert into for partition table: hash partition") {
-    sql(
-      """
-        | CREATE TABLE hashTableForInsert (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql("insert into hashTableForInsert select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-
-    validateDataFiles("default_hashTableForInsert", "0", Seq(0, 1, 2))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTableForInsert order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
-  }
-
-  test("Insert into for partition table: range partition") {
-    sql(
-      """
-        | CREATE TABLE rangeTableForInsert (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
-      """.stripMargin)
-    sql("insert into rangeTableForInsert select empno, empname, designation, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, doj from originTable")
-
-    validateDataFiles("default_rangeTableForInsert", "0", Seq(0, 1, 2, 4))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableForInsert order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
-  }
-
-  test("Insert into partition table: list partition") {
-    sql(
-      """
-        | CREATE TABLE listTableForInsert (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-    sql("insert into listTableForInsert select empno, empname, designation, doj, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, workgroupcategory from originTable")
-
-    validateDataFiles("default_listTableForInsert", "0", Seq(2, 3))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableForInsert order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-  }
-
-  test("multiple data loading for partition table") {
-    sql(
-      """
-        | CREATE TABLE multiLoads (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from multiLoads order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
-  }
-
-  test("multiple single pass data loading for partition table") {
-    sql(
-      """
-        | CREATE TABLE multiLoadsSinglePass (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoadsSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoadsSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoadsSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from multiLoadsSinglePass order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
-  }
-
-  test("multiple insertInto for partition table") {
-    sql(
-      """
-        | CREATE TABLE multiInserts (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql("insert into multiInserts select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-    sql("insert into multiInserts select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-    sql("insert into multiInserts select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from multiInserts order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
-  }
-
-  test("multiple data loading and insertInto for partition table") {
-    sql(
-      """
-        | CREATE TABLE loadAndInsert (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE loadAndInsert OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql("insert into loadAndInsert select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE loadAndInsert OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from loadAndInsert order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
-  }
-
-  test("list partition with string coloum and  list_info in upper case") {
-    sql(
-      """
-        | CREATE TABLE listTableUpper (empno int, empname String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (designation string)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='SE,SSE')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableUpper OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    validateDataFiles("default_listTableUpper", "0", Seq(0,1,2))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableUpper order by empno"),
-      sql("select empno, empname, designation, doj, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-  }
-
-
-  test("badrecords on partition column") {
-    sql("create table badrecordsPartition(intField1 int, stringField1 string) partitioned by (intField2 int) stored by 'carbondata' tblproperties('partition_type'='hash', 'num_partitions'='5')")
-    sql(s"load data local inpath '$resourcesPath/data_partition_badrecords.csv' into table badrecordsPartition options('bad_records_action'='force')")
-
-    checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 = 13"), Seq(Row(1)))
-    checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 = 14"), Seq(Row(1)))
-    checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 is null"), Seq(Row(9)))
-    checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 is not null"), Seq(Row(2)))
-  }
-
-  override def afterAll = {
-    dropTable
-  }
-
-  def dropTable = {
-    sql("drop table if exists originTable")
-    sql("drop table if exists hashTable")
-    sql("drop table if exists rangeTable")
-    sql("drop table if exists listTable")
-    sql("drop table if exists hashTableForInsert")
-    sql("drop table if exists rangeTableForInsert")
-    sql("drop table if exists listTableForInsert")
-    sql("drop table if exists originMultiLoads")
-    sql("drop table if exists multiLoads")
-    sql("drop table if exists multiInserts")
-    sql("drop table if exists loadAndInsert")
-    sql("drop table if exists listTableUpper")
-    sql("drop table if exists badrecordsPartition")
-    sql("drop table if exists hashTableSinglePass")
-    sql("drop table if exists rangeTableSinglePass")
-    sql("drop table if exists listTableSinglePass")
-    sql("drop table if exists multiLoadsSinglePass")
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestQueryForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestQueryForPartitionTable.scala
deleted file mode 100644
index df0eea3..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestQueryForPartitionTable.scala
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.testsuite.partition
-
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.util.QueryTest
-
-class TestQueryForPartitionTable  extends QueryTest with BeforeAndAfterAll {
-  override def beforeAll = {
-    dropTable
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
-    // create normal table(not a partition table)
-    sql(
-      """
-        | CREATE TABLE originTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE hashTable (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE rangeTable (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE rangeTableOnString (empno int, designation String,
-        | doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int,
-        | deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,
-        | attendance int, utilization int,salary int)
-        | PARTITIONED BY (empname String)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='Ben, Jack, Sam, Tom')
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE rangeTableOnStringNo (empno int, designation String,
-        | doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int,
-        | deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,
-        | attendance int, utilization int,salary int)
-        | PARTITIONED BY (empname String)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='Ben, Jack, Sam, Tom', 'DICTIONARY_EXCLUDE'='empname')
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE listTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE listTableOnString (empno int, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empname String)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-
-    sql(
-      """
-        | CREATE TABLE listTableOnStringNo (empno int, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empname String)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)', 'DICTIONARY_EXCLUDE'='empname')
-      """.stripMargin)
-
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTableOnString OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTableOnStringNo OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableOnString OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableOnStringNo OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-  }
-
-  test("detail query on partition table: hash table") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTable where empno = 13"),
-      sql("select  empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empno = 13"))
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTable where empno in (11, 13)"),
-      sql("select  empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empno in (11, 13)"))
-  }
-
-  test("detail query on partition table: range partition") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj = '2009-07-07 00:00:00'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj = '2009-07-07 00:00:00'"))
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj in (cast('2014-08-15 00:00:00' as timestamp), cast('2009-07-07 00:00:00' as timestamp))"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj in (cast('2014-08-15 00:00:00' as timestamp), cast('2009-07-07 00:00:00' as timestamp))"))
-    // Range
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj >= '2014-08-15 00:00:00'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj >= '2014-08-15 00:00:00'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj <= '2014-08-15 00:00:00'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj <= '2014-08-15 00:00:00'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj > '2014-08-15 00:00:00'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj > '2014-08-15 00:00:00'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj < '2014-08-15 00:00:00'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj < '2014-08-15 00:00:00'"))
-  }
-
-  test("detail query on partition table: range partition on string") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname = 'madhan'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname in ('tom', 'jack')"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
-    // Range
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname >= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname <= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname > 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname < 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
-  }
-
-  test("detail query on partition table: range partition on string no dictionary") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname = 'madhan'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname in ('tom', 'jack')"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
-    // Range
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname >= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname <= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname > 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname < 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
-  }
-
-  test("detail query on partition table: list partition") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory = 2"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory = 2"))
-
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory in (2, 3)"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory in (2, 3)"))
-
-    // Range
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory >= 2"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory >= 2"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory > 2"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory > 2"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory <= 2"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory <= 2"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory < 2"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory < 2"))
-  }
-
-  test("detail query on partition table: list partition on string") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname = 'madhan'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname in ('tom', 'jack')"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
-    // Range
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname >= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname <= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname > 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname < 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
-  }
-
-  test("detail query on partition table: list partition on string no dictionary") {
-    // EqualTo
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname = 'madhan'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
-    // In
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname in ('tom', 'jack')"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
-    // Range
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname >= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname <= 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname > 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
-    checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname < 'tom'"),
-      sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
-  }
-
-  override def afterAll = {
-    dropTable
-  }
-
-  def dropTable = {
-    sql("drop table if exists originTable")
-    sql("drop table if exists hashTable")
-    sql("drop table if exists rangeTable")
-    sql("drop table if exists rangeTableOnString")
-    sql("drop table if exists rangeTableOnStringNo")
-    sql("drop table if exists listTable")
-    sql("drop table if exists listTableOnString")
-    sql("drop table if exists listTableOnStringNo")
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
index 4825968..c9dd879 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
@@ -46,80 +46,7 @@ class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
                 | STORED BY 'carbondata'
               """.stripMargin)
 
-    sql("drop table if exists hashTable")
-    sql(
-      """
-        | CREATE TABLE hashTable (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-
-    sql("drop table if exists rangeTable")
-    sql(
-      """
-        | CREATE TABLE rangeTable (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='01-01-2010, 01-01-2015')
-      """.stripMargin)
-
-    sql("drop table if exists listTable")
-    sql(
-      """
-        | CREATE TABLE listTable (empno int, empname String, designation String, doj Timestamp,
-        |  workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
-
     sql(s"CREATE DATABASE if not exists partitionDB")
-    sql("drop table if exists partitionDB.hashTable")
-    sql("drop table if exists partitionDB.rangeTable")
-    sql("drop table if exists partitionDB.listTable")
-    sql(
-      """
-        | CREATE TABLE partitionDB.hashTable (empname String, designation String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (empno int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE partitionDB.rangeTable (empno int, empname String, designation String,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (doj Timestamp)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        |  'RANGE_INFO'='01-01-2010, 01-01-2015')
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE partitionDB.listTable (empno int, empname String, designation String,
-        |   doj Timestamp,workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (workgroupcategory int)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        |  'LIST_INFO'='0, 1, (2, 3)')
-      """.stripMargin)
 
     sql("DROP TABLE IF EXISTS hiveTable")
     sql("""
@@ -138,52 +65,13 @@ class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
   }
 
   test("show partition table: exception when show not partition table") {
-    val errorMessage = intercept[ProcessMetaDataException] {
+    val errorMessage = intercept[AnalysisException] {
       sql("show partitions notPartitionTable").show()
     }
     assert(errorMessage.getMessage.contains(
       "SHOW PARTITIONS is not allowed on a table that is not partitioned"))
   }
 
-  test("show partition table: hash table") {
-    // EqualTo
-    checkAnswer(sql("show partitions hashTable"), Seq(Row("empno = HASH_NUMBER(3)"), Row("partitionIds = [0, 1, 2]")))
-
-  }
-
-  test("show partition table: desc formatted should show partition type"){
-    //check for partition type exist in desc formatted
-    val result:DataFrame = sql("describe formatted hashTable")
-    checkExistence(result,true,"Partition Type")
-    val row: Array[Row] = result.collect().filter{row: Row => row.getString(0).contains("Partition Type")}
-    assert(row(0).getString(1).contains("HASH"))
-  }
-
-  test("show partition table: range partition") {
-    // EqualTo
-    checkAnswer(sql("show partitions rangeTable"), Seq(Row("0, doj = DEFAULT"),
-      Row("1, doj < 01-01-2010"), Row("2, 01-01-2010 <= doj < 01-01-2015")))
-  }
-
-  test("show partition table: list partition") {
-    // EqualTo
-    checkAnswer(sql("show partitions listTable"), Seq(Row("0, workgroupcategory = DEFAULT"),
-      Row("1, workgroupcategory = 0"), Row("2, workgroupcategory = 1"), Row("3, workgroupcategory = 2, 3")))
-
-  }
-  test("show partition table: not default db") {
-    // EqualTo
-    checkAnswer(sql("show partitions partitionDB.hashTable"), Seq(Row("empno = HASH_NUMBER(3)"),
-      Row("partitionIds = [0, 1, 2]")))
-    // EqualTo
-    checkAnswer(sql("show partitions partitionDB.rangeTable"), Seq(Row("0, doj = DEFAULT"),
-      Row("1, doj < 01-01-2010"), Row("2, 01-01-2010 <= doj < 01-01-2015")))
-    // EqualTo
-    checkAnswer(sql("show partitions partitionDB.listTable"), Seq(Row("0, workgroupcategory = DEFAULT"),
-      Row("1, workgroupcategory = 0"), Row("2, workgroupcategory = 1"), Row("3, workgroupcategory = 2, 3")))
-
-  }
-
   test("show partition table: hive partition table") {
     // EqualTo
     checkAnswer(sql("show partitions hiveTable"), Seq(Row("city=Hangzhou")))
@@ -194,27 +82,8 @@ class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
 
   override def afterAll = {
     sql("drop table if exists notPartitionTable")
-    sql("drop table if exists  hashTable")
-    sql("drop table if exists  listTable")
-    sql("drop table if exists  rangeTable")
     sql("drop table if exists  hiveTable")
     try {
-      sql("drop table if exists  partitionDB.hashTable")
-
-    } catch {
-      case ex: NoSuchDatabaseException => print(ex.getMessage())
-    }
-    try {
-      sql("drop table if exists  partitionDB.rangeTable")
-    } catch {
-      case ex: NoSuchDatabaseException => print(ex.getMessage())
-    }
-    try {
-      sql("drop table if exists  partitionDB.listTable")
-    } catch {
-      case ex: NoSuchDatabaseException => print(ex.getMessage())
-    }
-    try {
       sql("drop table if exists  hiveDB.hiveTable")
     } catch {
       case ex: NoSuchDatabaseException => print(ex.getMessage())
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala
index 14dab1e..c91caf6 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.carbondata.spark.testsuite.partition
 
 import org.apache.spark.sql.test.util.QueryTest
@@ -24,40 +25,14 @@ class TestUpdateForPartitionTable extends QueryTest with BeforeAndAfterAll {
   override def beforeAll: Unit = {
     dropTable
 
-    sql("create table test_range_partition_table (id int) partitioned by (name string) " +
-      "stored by 'carbondata' TBLPROPERTIES('PARTITION_TYPE' = 'RANGE','RANGE_INFO' = 'a,e,f')")
     sql("create table test_hive_partition_table (id int) partitioned by (name string) " +
       "stored by 'carbondata'")
-    sql("create table test_hash_partition_table (id int) partitioned by (name string) " +
-      "stored by 'carbondata' TBLPROPERTIES('PARTITION_TYPE' = 'HASH','NUM_PARTITIONS' = '2')")
-    sql("create table test_list_partition_table (id int) partitioned by (name string) " +
-      "stored by 'carbondata' TBLPROPERTIES('PARTITION_TYPE' = 'LIST','LIST_INFO' = 'a,e,f')")
   }
 
   def dropTable = {
-    sql("drop table if exists test_hash_partition_table")
-    sql("drop table if exists test_list_partition_table")
-    sql("drop table if exists test_range_partition_table")
     sql("drop table if exists test_hive_partition_table")
   }
 
-
-  test ("test update for unsupported partition table") {
-    val updateTables = Array(
-      "test_range_partition_table",
-      "test_list_partition_table",
-      "test_hash_partition_table")
-
-    updateTables.foreach(table => {
-      sql("insert into " + table + " select 1,'b' ")
-      val ex = intercept[UnsupportedOperationException] {
-        sql("update " + table + " set (name) = ('c') where id = 1").collect()
-      }
-      assertResult("Unsupported update operation for range/hash/list partition table")(ex.getMessage)
-    })
-
-  }
-
   test ("test update for hive(standard) partition table") {
 
     sql("insert into test_hive_partition_table select 1,'b' ")
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/DropPartitionCallable.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/DropPartitionCallable.java
deleted file mode 100644
index 50ed39d..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/DropPartitionCallable.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.partition;
-
-import java.util.concurrent.Callable;
-
-import org.apache.carbondata.spark.rdd.PartitionDropper;
-
-import org.apache.spark.sql.execution.command.DropPartitionCallableModel;
-
-public class DropPartitionCallable implements Callable<Void> {
-
-  private DropPartitionCallableModel dropPartitionCallableModel;
-
-  public DropPartitionCallable(DropPartitionCallableModel dropPartitionCallableModel) {
-    this.dropPartitionCallableModel = dropPartitionCallableModel;
-  }
-
-  @Override
-  public Void call() {
-    PartitionDropper.triggerPartitionDrop(dropPartitionCallableModel);
-    return null;
-  }
-}
-
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/SplitPartitionCallable.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/SplitPartitionCallable.java
deleted file mode 100644
index fb21308..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/SplitPartitionCallable.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.partition;
-
-import java.util.concurrent.Callable;
-
-import org.apache.carbondata.spark.rdd.PartitionSplitter;
-
-import org.apache.spark.sql.execution.command.SplitPartitionCallableModel;
-
-/**
- * Callable class which is used to split the partition in a separate callable.
- */
-public class SplitPartitionCallable implements Callable<Void> {
-
-  private final SplitPartitionCallableModel splitPartitionCallableModel;
-
-  public SplitPartitionCallable(SplitPartitionCallableModel splitPartitionCallableModel) {
-    this.splitPartitionCallableModel = splitPartitionCallableModel;
-  }
-
-  @Override
-  public Void call() {
-    PartitionSplitter.triggerPartitionSplit(splitPartitionCallableModel);
-    return null;
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
deleted file mode 100644
index 4da2d9a..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark
-
-import org.apache.spark.Partitioner
-
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.scan.partition.{HashPartitioner => JavaHashPartitioner, ListPartitioner => JavaListPartitioner, RangePartitioner => JavaRangePartitioner}
-import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
-
-object PartitionFactory {
-
-  def getPartitioner(partitionInfo: PartitionInfo): Partitioner = {
-    partitionInfo.getPartitionType match {
-      case PartitionType.HASH => new HashPartitioner(partitionInfo.getNumPartitions)
-      case PartitionType.LIST => new ListPartitioner(partitionInfo)
-      case PartitionType.RANGE => new RangePartitioner(partitionInfo)
-      case partitionType =>
-        throw new CarbonDataLoadingException(s"Unsupported partition type: $partitionType")
-    }
-  }
-}
-
-class HashPartitioner(partitions: Int) extends Partitioner {
-
-  private val partitioner = new JavaHashPartitioner(partitions)
-
-  override def numPartitions: Int = partitioner.numPartitions()
-
-  override def getPartition(key: Any): Int = partitioner.getPartition(key)
-}
-
-class ListPartitioner(partitionInfo: PartitionInfo) extends Partitioner {
-
-  private val partitioner = new JavaListPartitioner(partitionInfo)
-
-  override def numPartitions: Int = partitioner.numPartitions()
-
-  override def getPartition(key: Any): Int = partitioner.getPartition(key)
-}
-
-class RangePartitioner(partitionInfo: PartitionInfo) extends Partitioner {
-
-  private val partitioner = new JavaRangePartitioner(partitionInfo)
-
-  override def numPartitions: Int = partitioner.numPartitions()
-
-  override def getPartition(key: Any): Int = partitioner.getPartition(key)
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
deleted file mode 100644
index 33d5eaa..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.rdd
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution.command.AlterPartitionModel
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.partition.spliter.RowResultProcessor
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
-import org.apache.carbondata.spark.AlterPartitionResult
-import org.apache.carbondata.spark.util.CommonUtil
-
-class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
-    result: AlterPartitionResult[K, V],
-    partitionIds: Seq[String],
-    bucketId: Int,
-    identifier: AbsoluteTableIdentifier,
-    prev: RDD[Array[AnyRef]])
-  extends CarbonRDD[(K, V)](alterPartitionModel.sqlContext.sparkSession, prev) {
-
-  val carbonLoadModel = alterPartitionModel.carbonLoadModel
-  val segmentId = alterPartitionModel.segmentId
-  val oldPartitionIds = alterPartitionModel.oldPartitionIds
-  val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-  val partitionInfo = carbonTable.getPartitionInfo()
-
-  override protected def internalGetPartitions: Array[Partition] = {
-    val sc = alterPartitionModel.sqlContext.sparkContext
-    sc.setLocalProperty("spark.scheduler.pool", "DDL")
-    sc.setLocalProperty("spark.job.interruptOnCancel", "true")
-    firstParent[Array[AnyRef]].partitions
-  }
-
-  override def internalCompute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val rows = firstParent[Array[AnyRef]].iterator(split, context).toList.asJava
-    val iter = new Iterator[(K, V)] {
-      val partitionId: Int = partitionInfo.getPartitionId(split.index)
-      carbonLoadModel.setTaskNo(String.valueOf(partitionId))
-      carbonLoadModel.setSegmentId(segmentId)
-
-      CommonUtil.setTempStoreLocation(split.index, carbonLoadModel,
-        isCompactionFlow = false, isAltPartitionFlow = true)
-      val tempStoreLoc: Array[String] = CarbonDataProcessorUtil.getLocalDataFolderLocation(
-        carbonTable, carbonLoadModel.getTaskNo, segmentId, false, true)
-
-      val loadStatus: Boolean = if (rows.isEmpty) {
-        LOGGER.info("After repartition this split, NO target rows to write back.")
-        true
-      } else {
-        val segmentProperties = PartitionUtils.getSegmentProperties(identifier,
-          segmentId, partitionIds.toList, oldPartitionIds, partitionInfo, carbonTable)
-        val processor = new RowResultProcessor(
-          carbonTable,
-          carbonLoadModel,
-          segmentProperties,
-          tempStoreLoc,
-          bucketId)
-        try {
-          processor.execute(rows)
-        } catch {
-          case e: Exception =>
-            sys.error(s"Exception when executing Row result processor ${ e.getMessage }")
-        } finally {
-          TableProcessingOperations
-            .deleteLocalDataLoadFolderLocation(carbonLoadModel, false, true)
-        }
-      }
-
-      val loadResult = segmentId
-      var finished = false
-
-      override def hasNext: Boolean = {
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        finished = true
-        result.getKey(loadResult, loadStatus)
-      }
-    }
-    iter
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 5e33ea7..65fc9f3 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -101,11 +101,7 @@ class CarbonMergerRDD[K, V](
     val iter = new Iterator[(K, V)] {
       val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
       val carbonSparkPartition = theSplit.asInstanceOf[CarbonSparkPartition]
-      if (carbonTable.isPartitionTable) {
-        carbonLoadModel.setTaskNo(String.valueOf(carbonSparkPartition.partitionId))
-      } else {
-        carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-      }
+      carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
       val partitionSpec = if (carbonTable.isHivePartitionTable) {
         carbonSparkPartition.partitionSpec.get
       } else {
@@ -339,7 +335,6 @@ class CarbonMergerRDD[K, V](
     var defaultParallelism = sparkContext.defaultParallelism
     val result = new java.util.ArrayList[Partition](defaultParallelism)
     var taskPartitionNo = 0
-    var carbonPartitionId = 0
     var noOfBlocks = 0
 
     val taskInfoList = new java.util.ArrayList[Distributable]
@@ -464,7 +459,6 @@ class CarbonMergerRDD[K, V](
     val partitionTaskMap = new util.HashMap[PartitionSpec, String]()
     val counter = new AtomicInteger()
     var indexOfRangeColumn = -1
-    var taskIdCount = 0
     // As we are already handling null values in the filter expression separately so we
     // can remove the null from the ranges we get, else it may lead to duplicate data
     val newRanges = allRanges.filter { range =>
@@ -609,7 +603,6 @@ class CarbonMergerRDD[K, V](
     logInfo("no.of.nodes where data present=" + nodeBlockMap.size())
     defaultParallelism = sparkContext.defaultParallelism
 
-    val isPartitionTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isPartitionTable
     // Create Spark Partition for each task and assign blocks
     nodeBlockMap.asScala.foreach { case (nodeName, splitList) =>
       val taskSplitList = new java.util.ArrayList[NodeInfo](0)
@@ -633,15 +626,11 @@ class CarbonMergerRDD[K, V](
               splitListForRange,
               Array(nodeName))
           }
-          if (isPartitionTable) {
-            carbonPartitionId = Integer.parseInt(taskInfo.getTaskId)
-          }
           result.add(
             new CarbonSparkPartition(
               id,
               taskPartitionNo,
               multiBlockSplit,
-              carbonPartitionId,
               getPartitionNamesFromTask(taskInfo.getTaskId, partitionTaskMap)))
           taskPartitionNo += 1
         }
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
deleted file mode 100644
index 8941636..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ /dev/null
@@ -1,271 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.rdd
-
-import java.util.ArrayList
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.command.AlterPartitionModel
-import org.apache.spark.sql.hive.DistributionUtil
-import org.apache.spark.sql.util.CarbonException
-import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.converter.SparkDataTypeConverterImpl
-import org.apache.carbondata.core.datastore.block.{Distributable, SegmentProperties, TaskBlockInfo}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.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}
-import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator
-import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper
-import org.apache.carbondata.core.util.{ByteUtil, DataTypeUtil}
-import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
-import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
-import org.apache.carbondata.processing.merger.CarbonCompactionUtil
-import org.apache.carbondata.processing.partition.spliter.CarbonSplitExecutor
-import org.apache.carbondata.processing.util.CarbonLoaderUtil
-
-
-/**
- * This RDD is used in alter table partition statement to get data of target partitions,
- * then repartition data according to new partitionInfo
- * @param alterPartitionModel
- * @param absoluteTableIdentifier
- * @param partitionIds  the ids of target partition to be scanned
- * @param bucketId
- */
-class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
-    absoluteTableIdentifier: AbsoluteTableIdentifier,
-    partitionIds: Seq[String],
-    bucketId: Int)
-  extends CarbonRDD[(AnyRef, Array[AnyRef])](alterPartitionModel.sqlContext.sparkSession, Nil) {
-
-  private val queryId = alterPartitionModel.sqlContext.sparkContext.getConf
-    .get("queryId", System.nanoTime() + "")
-  val segmentId = alterPartitionModel.segmentId
-  val carbonLoadModel = alterPartitionModel.carbonLoadModel
-  val oldPartitionIdList = alterPartitionModel.oldPartitionIds
-  var storeLocation: String = null
-  var splitStatus: Boolean = false
-  var blockId: String = null
-  val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-  val dimensions = carbonTable.getAllDimensions.asScala
-  val measures = carbonTable.getAllMeasures.asScala
-  val partitionInfo = carbonTable.getPartitionInfo()
-  val partitionColumn = partitionInfo.getColumnSchemaList().get(0)
-  val partitionDataType = partitionColumn.getDataType
-  val partitionColumnName = partitionColumn.getColumnName
-  var isDimension: Boolean = false
-  val encodingList = partitionColumn.getEncodingList
-  var dimension: CarbonDimension = null
-  var measure: CarbonMeasure = null
-  val noDictionaryIndexGroup: ArrayBuffer[Int] = new ArrayBuffer[Int]()
-  val dictionaryIndexGroup: ArrayBuffer[Int] = new ArrayBuffer[Int]()
-  val measureIndexGroup: ArrayBuffer[Int] = new ArrayBuffer[Int]()
-
-  override def internalGetPartitions: Array[Partition] = {
-    val parallelism = sparkContext.defaultParallelism
-    val jobConf = new JobConf(FileFactory.getConfiguration)
-    val job = new Job(jobConf)
-    val format = CarbonInputFormatUtil.createCarbonTableInputFormat(absoluteTableIdentifier,
-      partitionIds.toList.asJava, job)
-    CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
-    job.getConfiguration.set("query.id", queryId)
-
-    val splits = format.getSplitsOfOneSegment(job, segmentId,
-      oldPartitionIdList.map(_.asInstanceOf[Integer]).asJava, partitionInfo)
-    var partition_num = 0
-    val result = new ArrayList[Partition](parallelism)
-    val blockList = splits.asScala
-      .filter(_.asInstanceOf[CarbonInputSplit].getBucketId.toInt == bucketId)
-      .map(_.asInstanceOf[Distributable])
-    if (!blockList.isEmpty) {
-      val activeNodes = DistributionUtil.ensureExecutorsAndGetNodeList(blockList, sparkContext)
-      val nodeBlockMapping = CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1,
-        parallelism, activeNodes.toList.asJava)
-      nodeBlockMapping.asScala.foreach { case (node, blockList) =>
-        blockList.asScala.foreach { blocksPerTask =>
-          val splits = blocksPerTask.asScala.map(_.asInstanceOf[CarbonInputSplit])
-          if (blocksPerTask.size() != 0) {
-            val multiBlockSplit =
-              new CarbonMultiBlockSplit(splits.asJava, Array(node))
-            val partition = new CarbonSparkPartition(id, partition_num, multiBlockSplit)
-            result.add(partition)
-            partition_num += 1
-          }
-        }
-      }
-    }
-    result.toArray(new Array[Partition](result.size()))
-  }
-
-  override def internalCompute(split: Partition, context: TaskContext):
-  Iterator[(AnyRef, Array[AnyRef])] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-      var exec : CarbonSplitExecutor = null
-      val rows : java.util.List[(AnyRef, Array[AnyRef])] = new ArrayList[(AnyRef, Array[AnyRef])]()
-      try {
-        val inputSplit = split.asInstanceOf[CarbonSparkPartition].split.value
-        val splits = inputSplit.getAllSplits.asScala
-        val tableBlockInfoList = CarbonInputSplit.createBlocks(splits.asJava)
-        val segmentMapping: java.util.Map[String, TaskBlockInfo] =
-          CarbonCompactionUtil.createMappingForSegments(tableBlockInfoList)
-        val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-        var result : java.util.List[PartitionSpliterRawResultIterator] = null
-        try {
-          exec = new CarbonSplitExecutor(segmentMapping, carbonTable)
-          result = exec.processDataBlocks(segmentId, new SparkDataTypeConverterImpl(),
-            FileFactory.getConfiguration)
-        } catch {
-          case e: Throwable =>
-            LOGGER.error(e)
-            if (null != e.getMessage) {
-              CarbonException.analysisException(
-                s"Exception occurred in query execution :: ${e.getMessage}")
-            } else {
-              CarbonException.analysisException(
-                "Exception occurred in query execution. Please check logs.")
-            }
-        }
-        val segmentProperties = PartitionUtils.getSegmentProperties(absoluteTableIdentifier,
-          segmentId, partitionIds.toList, oldPartitionIdList, partitionInfo, carbonTable)
-        val partColIdx = getPartitionColumnIndex(partitionColumnName, segmentProperties)
-        indexInitialise()
-        for (iterator <- result.asScala) {
-          while (iterator.hasNext) {
-            val row = iterator.next()
-            val partitionColumnValue = getPartitionColumnValue(row, partColIdx,
-              segmentProperties)
-            rows.add((partitionColumnValue, row))
-          }
-        }
-      } catch {
-        case e: Exception =>
-          LOGGER.error(e)
-          throw e
-      } finally {
-        if (null != exec) {
-          exec.finish
-        }
-      }
-    val iter = rows.iterator().asScala
-    iter
-  }
-
-  def getPartitionColumnValue(row: Array[AnyRef], partColIdx: Int,
-      segmentProperties: SegmentProperties): AnyRef = {
-    val dims: Array[Byte] = row(0).asInstanceOf[ByteArrayWrapper].getDictionaryKey
-    val keyGen = segmentProperties.getDimensionKeyGenerator
-    val keyArray: Array[Long] = keyGen.getKeyArray(dims)
-    val encodings = partitionColumn.getEncodingList
-    val partitionType = partitionInfo.getPartitionType
-    var partitionValue: AnyRef = null
-    val factor = 1000L
-    if (isDimension) {
-      // direct dictionary
-      if (encodings.contains(Encoding.DIRECT_DICTIONARY)) {
-        val directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-          .getDirectDictionaryGenerator(partitionDataType)
-        val dictionaryIndex = dictionaryIndexGroup.indexOf(partColIdx)
-        val surrogateValue = (keyArray(dictionaryIndex) / factor).toInt
-        partitionValue = directDictionaryGenerator.getValueFromSurrogate(surrogateValue)
-      } else if (!encodings.contains(Encoding.DICTIONARY)) {
-        // no dictionary
-        val byteArray = row(0).asInstanceOf[ByteArrayWrapper].getNoDictionaryKeys
-        val index = noDictionaryIndexGroup.indexOf(partColIdx)
-        partitionValue = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(byteArray(index)
-          , partitionDataType)
-        if (partitionValue.isInstanceOf[UTF8String]) {
-          partitionValue = partitionValue.toString
-        }
-      } else {  // normal dictionary
-        val dict = CarbonLoaderUtil.getDictionary(absoluteTableIdentifier,
-          dimension.getColumnIdentifier, partitionDataType)
-        if (partitionDataType == DataTypes.STRING) {
-          if (partitionType == PartitionType.RANGE) {
-            partitionValue = ByteUtil.
-              toBytes(dict.getDictionaryValueForKey(keyArray(partColIdx).toInt))
-          } else {
-            partitionValue = dict.getDictionaryValueForKey(keyArray(partColIdx).toInt)
-          }
-        } else {
-          partitionValue = dict.getDictionaryValueForKey(keyArray(partColIdx).toInt)
-        }
-
-      }
-    } else {
-      partitionValue = row(measureIndexGroup(partColIdx))
-    }
-    partitionValue
-  }
-
-  def indexInitialise(): Unit = {
-      for (dim: CarbonDimension <- dimensions) {
-        if (!dim.getEncoder.contains(Encoding.DICTIONARY)) {
-          noDictionaryIndexGroup.append(dimensions.indexOf(dim))
-        } else {
-          dictionaryIndexGroup.append(dimensions.indexOf(dim))
-        }
-      }
-      for (msr: CarbonMeasure <- measures) {
-        // index of measure in row
-        measureIndexGroup.append(measures.indexOf(msr) + 1)
-      }
-  }
-
-  /**
-   * get the index of partition column in dimension/measure
-   * @param partitionColumnName
-   * @param segmentProperties
-   * @return
-   */
-  def getPartitionColumnIndex(partitionColumnName: String,
-      segmentProperties: SegmentProperties): Int = {
-    val dimensions = segmentProperties.getDimensions
-    val measures = segmentProperties.getMeasures
-    val columns = dimensions.asScala.map(_.getColName) ++ measures.asScala.map(_.getColName)
-    var index = 0
-    for (i <- 0 until columns.size) {
-      if (columns(i) == partitionColumnName) {
-        index = i
-      }
-    }
-    if (index < dimensions.size()) {
-      isDimension = true
-      dimension = dimensions.get(index)
-    } else {
-      index = index - dimensions.size()
-      measure = measures.get(index)
-    }
-    index
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala
index b473d35..563ab25 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala
@@ -26,7 +26,6 @@ class CarbonSparkPartition(
     val rddId: Int,
     val idx: Int,
     @transient val multiBlockSplit: CarbonMultiBlockSplit,
-    val partitionId: Int = 0,
     val partitionSpec: Option[PartitionSpec] = None)
     extends Partition {
 
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 8861863..7511502 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -463,13 +463,12 @@ class PartitionTableDataLoaderRDD[K, V](
       val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
       val model: CarbonLoadModel = carbonLoadModel
       val carbonTable = model.getCarbonDataLoadSchema.getCarbonTable
-      val partitionInfo = carbonTable.getPartitionInfo()
       val uniqueLoadStatusId =
         carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
       try {
 
         loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS)
-        carbonLoadModel.setTaskNo(String.valueOf(partitionInfo.getPartitionId(theSplit.index)))
+        carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
         carbonLoadModel.setPreFetch(false)
         val recordReaders = Array[CarbonIterator[Array[AnyRef]]] {
           new NewRddIterator(firstParent[Row].iterator(theSplit, context), carbonLoadModel, context)
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
deleted file mode 100644
index bc90b89..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.rdd
-
-import java.io.IOException
-
-import org.apache.spark.sql.execution.command.{AlterPartitionModel, DropPartitionCallableModel}
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.spark.{AlterPartitionResultImpl, PartitionFactory}
-
-object PartitionDropper {
-
-  val logger = LogServiceFactory.getLogService(PartitionDropper.getClass.getName)
-
-  def triggerPartitionDrop(dropPartitionCallableModel: DropPartitionCallableModel): Unit = {
-    val alterPartitionModel = new AlterPartitionModel(dropPartitionCallableModel.carbonLoadModel,
-      dropPartitionCallableModel.segmentId.getSegmentNo,
-      dropPartitionCallableModel.oldPartitionIds,
-      dropPartitionCallableModel.sqlContext
-    )
-    val partitionId = dropPartitionCallableModel.partitionId
-    val oldPartitionIds = dropPartitionCallableModel.oldPartitionIds
-    val dropWithData = dropPartitionCallableModel.dropWithData
-    val carbonTable = dropPartitionCallableModel.carbonTable
-    val dbName = carbonTable.getDatabaseName
-    val tableName = carbonTable.getTableName
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-
-    var finalDropStatus = false
-    val bucketInfo = carbonTable.getBucketingInfo()
-    val bucketNumber = bucketInfo match {
-      case null => 1
-      case _ => bucketInfo.getNumOfRanges
-    }
-    val partitionIndex = oldPartitionIds.indexOf(Integer.valueOf(partitionId))
-    val targetPartitionId = partitionInfo.getPartitionType match {
-      case PartitionType.RANGE => if (partitionIndex == oldPartitionIds.length - 1) {
-        "0"
-      } else {
-        String.valueOf(oldPartitionIds(partitionIndex + 1))
-      }
-      case PartitionType.LIST => "0"
-      case _ => throw new UnsupportedOperationException(
-        s"${partitionInfo.getPartitionType} is not supported")
-    }
-
-    if (!dropWithData) {
-      try {
-        for (i <- 0 until bucketNumber) {
-          val bucketId = i
-          val rdd = new CarbonScanPartitionRDD(alterPartitionModel,
-            absoluteTableIdentifier,
-            Seq(partitionId, targetPartitionId),
-            bucketId
-          ).partitionBy(partitioner).map(_._2)
-
-          val dropStatus = new AlterTableLoadPartitionRDD(alterPartitionModel,
-            new AlterPartitionResultImpl(),
-            Seq(partitionId),
-            bucketId,
-            absoluteTableIdentifier,
-            rdd).collect()
-
-          if (dropStatus.length == 0) {
-            finalDropStatus = false
-          } else {
-            finalDropStatus = dropStatus.forall(_._2)
-          }
-          if (!finalDropStatus) {
-            logger.error(s"Drop Partition request failed for table " +
-                         s"${ dbName }.${ tableName }")
-          }
-        }
-
-        if (finalDropStatus) {
-          try {
-            PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
-              Seq(partitionId, targetPartitionId).toList, dbName,
-              tableName, partitionInfo)
-          } catch {
-            case e: IOException =>
-              throw new IOException("Exception while delete original carbon files ", e)
-          }
-          logger.info(s"Drop Partition request completed for table " +
-                      s"${ dbName }.${ tableName }")
-        }
-      } catch {
-        case e: Exception =>
-          throw new RuntimeException("Exception in dropping partition action", e)
-      }
-    } else {
-      PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
-        Seq(partitionId).toList, dbName, tableName, partitionInfo)
-      logger.info(s"Drop Partition request completed for table " +
-                  s"${ dbName }.${ tableName }")
-    }
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
deleted file mode 100644
index bd46c24..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.rdd
-
-import java.io.IOException
-
-import org.apache.spark.sql.execution.command.{AlterPartitionModel, SplitPartitionCallableModel}
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.spark.{AlterPartitionResultImpl, PartitionFactory}
-
-object PartitionSplitter {
-
-  val logger = LogServiceFactory.getLogService(PartitionSplitter.getClass.getName)
-
-  def triggerPartitionSplit(splitPartitionCallableModel: SplitPartitionCallableModel): Unit = {
-
-     val alterPartitionModel = new AlterPartitionModel(splitPartitionCallableModel.carbonLoadModel,
-       splitPartitionCallableModel.segmentId,
-       splitPartitionCallableModel.oldPartitionIds,
-       splitPartitionCallableModel.sqlContext
-     )
-     val partitionId = splitPartitionCallableModel.partitionId
-     val carbonLoadModel = splitPartitionCallableModel.carbonLoadModel
-     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-     val tableName = carbonTable.getTableName
-     val databaseName = carbonTable.getDatabaseName
-     val bucketInfo = carbonTable.getBucketingInfo()
-     var finalSplitStatus = false
-     val bucketNumber = bucketInfo match {
-       case null => 1
-       case _ => bucketInfo.getNumOfRanges
-     }
-     val partitionInfo = carbonTable.getPartitionInfo()
-     val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-
-     for (i <- 0 until bucketNumber) {
-       val bucketId = i
-       val rdd = new CarbonScanPartitionRDD(
-         alterPartitionModel,
-         absoluteTableIdentifier,
-         Seq(partitionId),
-         bucketId
-       ).partitionBy(partitioner).map(_._2)
-
-       val splitStatus = new AlterTableLoadPartitionRDD(alterPartitionModel,
-         new AlterPartitionResultImpl(),
-         Seq(partitionId),
-         bucketId,
-         absoluteTableIdentifier,
-         rdd).collect()
-
-       if (splitStatus.length == 0) {
-         finalSplitStatus = false
-       } else {
-         finalSplitStatus = splitStatus.forall(_._2)
-       }
-       if (!finalSplitStatus) {
-         logger.error(s"Add/Split Partition request failed for table " +
-                      s"${ databaseName }.${ tableName }")
-       }
-     }
-     if (finalSplitStatus) {
-       try {
-         PartitionUtils.
-           deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
-             Seq(partitionId).toList, databaseName, tableName, partitionInfo)
-       } catch {
-         case e: IOException =>
-           throw new IOException("Exception while delete original carbon files ", e)
-       }
-       logger.info(s"Add/Split Partition request completed for table " +
-                   s"${ databaseName }.${ tableName }")
-     }
-  }
-}
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 5f9f9bb..fd0cf74 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
@@ -583,9 +583,9 @@ object CarbonScalaUtil {
       .isDefined
     if (isLocalDictIncludeDefined && isLocalDictExcludeDefined) {
       val localDictIncludeCols = tableProperties(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE)
-        .split(",").map(_.trim)
+        .split(",").map(_.trim.toLowerCase)
       val localDictExcludeCols = tableProperties(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE)
-        .split(",").map(_.trim)
+        .split(",").map(_.trim.toLowerCase)
       localDictIncludeCols.foreach { distCol =>
         if (localDictExcludeCols.exists(x => x.equalsIgnoreCase(distCol.trim))) {
           val duplicateColumns = (localDictIncludeCols ++ localDictExcludeCols)
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 b16c434..787b591 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
@@ -31,11 +31,8 @@ import scala.math.BigDecimal.RoundingMode
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.spark.{SparkContext, SparkEnv}
-import org.apache.spark.sql.{Row, RowFactory}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
 import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField}
-import org.apache.spark.sql.types.{MetadataBuilder, StringType}
 import org.apache.spark.util.FileUtils
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
@@ -44,14 +41,9 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.memory.{UnsafeMemoryManager, UnsafeSortMemoryManager}
 import org.apache.carbondata.core.metadata.CarbonMetadata
-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.CarbonTable
-import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil, ThreadLocalTaskInfo}
-import org.apache.carbondata.core.util.comparator.Comparator
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, ThreadLocalTaskInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -108,56 +100,6 @@ object CommonUtil {
     isValid
   }
 
-  /**
-   * 1. If partitioned by clause exists, then partition_type should be defined
-   * 2. If partition_type is Hash, then num_partitions should be defined
-   * 3. If partition_type is List, then list_info should be defined
-   * 4. If partition_type is Range, then range_info should be defined
-   * 5. Only support single level partition for now
-   * @param tableProperties
-   * @param partitionerFields
-   * @return partition clause and definition in tblproperties are valid or not
-   */
-  def validatePartitionColumns(tableProperties: Map[String, String],
-      partitionerFields: Seq[PartitionerField]): Boolean = {
-    var isValid: Boolean = true
-    val partitionType = tableProperties.get(CarbonCommonConstants.PARTITION_TYPE)
-    val numPartitions = tableProperties.get(CarbonCommonConstants.NUM_PARTITIONS)
-    val rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO)
-    val listInfo = tableProperties.get(CarbonCommonConstants.LIST_INFO)
-
-    if (partitionType.isEmpty) {
-      isValid = true
-    } else {
-      partitionType.get.toUpperCase() match {
-        case "HASH" => if (!numPartitions.isDefined
-        || scala.util.Try(numPartitions.get.toInt).isFailure
-        || numPartitions.get.toInt <= 0) {
-          isValid = false
-        }
-        case "LIST" => if (!listInfo.isDefined) {
-          isValid = false
-        } else {
-          listInfo.get.replace("(", "").replace(")", "").split(",").map(_.trim).foreach(
-            isValid &= validateTypeConvert(partitionerFields(0), _))
-        }
-        case "RANGE" => if (!rangeInfo.isDefined) {
-          isValid = false
-        } else {
-          rangeInfo.get.split(",").map(_.trim).foreach(
-            isValid &= validateTypeConvert(partitionerFields(0), _))
-        }
-        case "RANGE_INTERVAL" => isValid = false
-        case _ => isValid = true
-      }
-      // only support one partition column for now
-      if (partitionerFields.length > 1 && !partitionType.get.toUpperCase.equals("NATIVE_HIVE")) {
-        isValid = false
-      }
-    }
-    isValid
-  }
-
   def validateTypeConvertForSpark2(partitionerField: PartitionerField, value: String): Boolean = {
     val result = partitionerField.dataType.get.toLowerCase match {
       case "integertype" =>
@@ -259,86 +201,6 @@ object CommonUtil {
     }
   }
 
-  /**
-   * To verify the range info is in correct order
-   * @param rangeInfo
-   * @param columnDataType
-   * @param timestampFormatter
-   * @param dateFormatter
-   */
-  def validateRangeInfo(rangeInfo: List[String], columnDataType: DataType,
-      timestampFormatter: SimpleDateFormat, dateFormatter: SimpleDateFormat): Unit = {
-    if (rangeInfo.size <= 1) {
-      throw new
-         MalformedCarbonCommandException("Range info must define a valid range.Please check again!")
-    }
-    val comparator = Comparator.getComparator(columnDataType)
-    var head = columnDataType match {
-      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 DataTypes.STRING => ByteUtil.toBytes(iterator.next())
-        case _ => PartitionUtil.getDataBasedOnDataType(iterator.next(), columnDataType,
-          timestampFormatter, dateFormatter)
-      }
-      if (next == null) {
-        sys.error(
-          "Data in range info must be the same type with the partition field's type "
-            + columnDataType)
-      }
-      if (comparator.compare(head, next) < 0) {
-        head = next
-      } else {
-        sys.error("Range info must be in ascending order, please check again!")
-      }
-    }
-  }
-
-  def validateSplitListInfo(originListInfo: List[String], newListInfo: List[String],
-      originList: List[List[String]]): Unit = {
-    if (originListInfo.size == 1) {
-      sys.error("The target list partition cannot be split, please check again!")
-    }
-    if (newListInfo.size == 1) {
-      sys.error("Can't split list to one partition, please check again!")
-    }
-    if (!(newListInfo.size < originListInfo.size)) {
-      sys.error("The size of new list must be smaller than original list, please check again!")
-    }
-    val tempList = newListInfo.mkString(",").split(",")
-      .map(_.replace("(", "").replace(")", "").trim)
-    if (tempList.length != originListInfo.size) {
-      sys.error("The total number of elements in new list must equal to original list!")
-    }
-    if (!(tempList diff originListInfo).isEmpty) {
-      sys.error("The elements in new list must exist in original list")
-    }
-  }
-
-  def validateAddListInfo(newListInfo: List[String], originList: List[List[String]]): Unit = {
-    if (newListInfo.size < 1) {
-      sys.error("Please add at least one new partition")
-    }
-    for (originElementGroup <- originList) {
-      for (newElement <- newListInfo ) {
-        if (originElementGroup.contains(newElement)) {
-          sys.error(s"The partition $newElement is already exist! Please check again!")
-        }
-      }
-    }
-  }
-
-  def validateListInfo(listInfo: List[List[String]]): Unit = {
-    val list = listInfo.flatten
-    if (list.distinct.size != list.size) {
-      sys.error("Duplicate elements defined in LIST_INFO!")
-    }
-  }
-
   def validateFields(key: String, fields: Seq[Field]): Boolean = {
     var isValid: Boolean = false
     fields.foreach { field =>
@@ -635,52 +497,6 @@ object CommonUtil {
     }
   }
 
-  def getPartitionInfo(columnName: String, partitionType: PartitionType,
-      partitionInfo: PartitionInfo): Seq[Row] = {
-    var result = Seq.newBuilder[Row]
-    partitionType match {
-      case PartitionType.RANGE =>
-        result.+=(RowFactory.create("0" + ", " + columnName + " = DEFAULT"))
-        val rangeInfo = partitionInfo.getRangeInfo
-        val size = rangeInfo.size() - 1
-        for (index <- 0 to size) {
-          if (index == 0) {
-            val id = partitionInfo.getPartitionId(index + 1).toString
-            val desc = columnName + " < " + rangeInfo.get(index)
-            result.+=(RowFactory.create(id + ", " + desc))
-          } else {
-            val id = partitionInfo.getPartitionId(index + 1).toString
-            val desc = rangeInfo.get(index - 1) + " <= " + columnName + " < " + rangeInfo.get(index)
-            result.+=(RowFactory.create(id + ", " + desc))
-          }
-        }
-      case PartitionType.RANGE_INTERVAL =>
-        result.+=(RowFactory.create(columnName + " = "))
-      case PartitionType.LIST =>
-        result.+=(RowFactory.create("0" + ", " + columnName + " = DEFAULT"))
-        val listInfo = partitionInfo.getListInfo
-        listInfo.asScala.foreach {
-          f =>
-            val id = partitionInfo.getPartitionId(listInfo.indexOf(f) + 1).toString
-            val desc = columnName + " = " + f.toArray().mkString(", ")
-            result.+=(RowFactory.create(id + ", " + desc))
-        }
-      case PartitionType.HASH =>
-        val hashNumber = partitionInfo.getNumPartitions
-        result.+=(RowFactory.create(columnName + " = HASH_NUMBER(" + hashNumber.toString() + ")"))
-        result.+=(RowFactory.create(s"partitionIds = ${partitionInfo.getPartitionIds}"))
-      case others =>
-        result.+=(RowFactory.create(columnName + " = "))
-    }
-    val rows = result.result()
-    rows
-  }
-
-  def partitionInfoOutput: Seq[Attribute] = Seq(
-    AttributeReference("Partition(Id, DESC)", StringType, false,
-      new MetadataBuilder().putString("comment", "partition").build())()
-  )
-
   /**
    * Method to clear the memory for a task
    * if present
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 90dfc7c..d21e6e5 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
@@ -414,7 +414,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
     }
 
     // get partitionInfo
-    val partitionInfo = getPartitionInfo(partitionCols, tableProperties)
+    val partitionInfo = getPartitionInfo(partitionCols)
     if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
       // validate the column_meta_cache option
       val tableColumns = dims.map(x => x.name.get) ++ msrs.map(x => x.name.get)
@@ -550,7 +550,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       throw new MalformedCarbonCommandException(errMsg)
     }
 
-    if (!dataTypeErr.isEmpty) {
+    if (dataTypeErr.nonEmpty) {
       val errMsg = s"long_string_columns: ${
         dataTypeErr.mkString(",")
       } ,its data type is not string. Please check the create table statement."
@@ -558,45 +558,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
     }
   }
 
-  /**
-   * @param partitionCols
-   * @param tableProperties
-   */
-  protected def getPartitionInfo(partitionCols: Seq[PartitionerField],
-      tableProperties: Map[String, String]): Option[PartitionInfo] = {
-    val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-    val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-      .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+  protected def getPartitionInfo(partitionCols: Seq[PartitionerField]): Option[PartitionInfo] = {
     if (partitionCols.isEmpty) {
       None
     } else {
-      var partitionType: String = ""
-      var numPartitions = 0
-      var rangeInfo = List[String]()
-      var listInfo = List[List[String]]()
-
-      val columnDataType = DataTypeConverterUtil.
-        convertToCarbonType(partitionCols.head.dataType.get)
-      if (tableProperties.get(CarbonCommonConstants.PARTITION_TYPE).isDefined) {
-        partitionType = tableProperties(CarbonCommonConstants.PARTITION_TYPE)
-      }
-      if (tableProperties.get(CarbonCommonConstants.NUM_PARTITIONS).isDefined) {
-        numPartitions = tableProperties(CarbonCommonConstants.NUM_PARTITIONS)
-          .toInt
-      }
-      if (tableProperties.get(CarbonCommonConstants.RANGE_INFO).isDefined) {
-        rangeInfo = tableProperties(CarbonCommonConstants.RANGE_INFO).split(",")
-          .map(_.trim()).toList
-        CommonUtil.validateRangeInfo(rangeInfo, columnDataType, timestampFormatter, dateFormatter)
-      }
-      if (tableProperties.get(CarbonCommonConstants.LIST_INFO).isDefined) {
-        val originListInfo = tableProperties(CarbonCommonConstants.LIST_INFO)
-        listInfo = PartitionUtils.getListInfo(originListInfo)
-        CommonUtil.validateListInfo(listInfo)
-      }
       val cols : ArrayBuffer[ColumnSchema] = new ArrayBuffer[ColumnSchema]()
       partitionCols.foreach(partition_col => {
         val columnSchema = new ColumnSchema
@@ -605,21 +570,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         columnSchema.setColumnName(partition_col.partitionColumn)
         cols += columnSchema
       })
-
-      var partitionInfo : PartitionInfo = null
-      partitionType.toUpperCase() match {
-        case "HASH" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.HASH)
-          partitionInfo.initialize(numPartitions)
-        case "RANGE" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.RANGE)
-          partitionInfo.setRangeInfo(rangeInfo.asJava)
-          partitionInfo.initialize(rangeInfo.size + 1)
-        case "LIST" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.LIST)
-          partitionInfo.setListInfo(listInfo.map(_.asJava).asJava)
-          partitionInfo.initialize(listInfo.size + 1)
-        case _ => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.NATIVE_HIVE)
-          partitionInfo.setListInfo(listInfo.map(_.asJava).asJava)
-      }
-      Some(partitionInfo)
+      Some(new PartitionInfo(cols.asJava, PartitionType.NATIVE_HIVE))
     }
   }
 
@@ -759,8 +710,9 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
 
     // All long_string cols should be there in create table cols and should be of string data type
     if (tableProperties.get(CarbonCommonConstants.LONG_STRING_COLUMNS).isDefined) {
-      varcharCols =
-        tableProperties(CarbonCommonConstants.LONG_STRING_COLUMNS).split(",").map(_.trim)
+      varcharCols = tableProperties(CarbonCommonConstants.LONG_STRING_COLUMNS)
+        .split(",")
+        .map(_.trim.toLowerCase)
       validateLongStringColumns(fields, varcharCols)
     }
 
@@ -1088,12 +1040,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       case Token("TOK_TABLEPROPLIST", list) =>
         list.map {
           case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
-            val reslovedKey = unquoteString(key)
-            if (needToConvertToLowerCase(reslovedKey)) {
-              (reslovedKey, unquoteString(value))
-            } else {
-              (reslovedKey, unquoteStringWithoutLowerConversion(value))
-            }
+            (unquoteString(key), unquoteStringWithoutLowerConversion(value))
         }
     }
   }
@@ -1114,11 +1061,6 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
     }
   }
 
-  private def needToConvertToLowerCase(key: String): Boolean = {
-    val noConvertList = Array("LIST_INFO", "RANGE_INFO")
-    !noConvertList.exists(x => x.equalsIgnoreCase(key))
-  }
-
   protected def validateOptions(optionList: Option[List[(String, String)]]): Unit = {
 
     // validate with all supported options
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
index 1782cdf..61eeaf9 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
@@ -14,231 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.spark.util
 
-import java.io.{File, IOException}
-import java.text.SimpleDateFormat
-import java.util
+package org.apache.spark.util
 
-import scala.collection.JavaConverters._
 import scala.collection.mutable
-import scala.collection.mutable.ListBuffer
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.sql.execution.command.{AlterPartitionModel, DataMapField, Field, PartitionerField}
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.datastore.block.{SegmentProperties, TableBlockInfo}
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, SegmentFileStore}
-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.CarbonTable
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.hadoop.CarbonInputSplit
-import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.spark.sql.execution.command.{DataMapField, Field, PartitionerField}
 
 object PartitionUtils {
 
-  def getListInfo(originListInfo: String): List[List[String]] = {
-    var listInfo = ListBuffer[List[String]]()
-    var templist = ListBuffer[String]()
-    val arr = originListInfo.split(",")
-      .map(_.trim())
-    var groupEnd = true
-    val iter = arr.iterator
-    while (iter.hasNext) {
-      val value = iter.next()
-      if (value.startsWith("(")) {
-        templist += value.replace("(", "").trim()
-        groupEnd = false
-      } else if (value.endsWith(")")) {
-        templist += value.replace(")", "").trim()
-        listInfo += templist.toList
-        templist.clear()
-        groupEnd = true
-      } else {
-        if (groupEnd) {
-          templist += value
-          listInfo += templist.toList
-          templist.clear()
-        } else {
-          templist += value
-        }
-      }
-    }
-    listInfo.toList
-  }
-
-  /**
-   * verify the add/split information and update the partitionInfo:
-   *  1. update rangeInfo/listInfo
-   *  2. update partitionIds
-   */
-  def updatePartitionInfo(partitionInfo: PartitionInfo, partitionIdList: List[Int],
-      partitionId: Int, splitInfo: List[String], timestampFormatter: SimpleDateFormat,
-      dateFormatter: SimpleDateFormat): Unit = {
-    val columnDataType = partitionInfo.getColumnSchemaList.get(0).getDataType
-    val index = partitionIdList.indexOf(partitionId)
-    if (index < 0) {
-      throw new IllegalArgumentException("Invalid Partition Id " + partitionId +
-        "\n Use show partitions table_name to get the list of valid partitions")
-    }
-    if (partitionInfo.getPartitionType == PartitionType.RANGE) {
-      val rangeInfo = partitionInfo.getRangeInfo.asScala.toList
-      val newRangeInfo = partitionId match {
-        case 0 => rangeInfo ++ splitInfo
-        case _ => rangeInfo.take(index - 1) ++ splitInfo ++
-          rangeInfo.takeRight(rangeInfo.size - index)
-      }
-      CommonUtil.validateRangeInfo(newRangeInfo, columnDataType,
-        timestampFormatter, dateFormatter)
-      partitionInfo.setRangeInfo(newRangeInfo.asJava)
-    } else if (partitionInfo.getPartitionType == PartitionType.LIST) {
-      val originList = partitionInfo.getListInfo.asScala.map(_.asScala.toList).toList
-      if (partitionId != 0) {
-        val targetListInfo = partitionInfo.getListInfo.get(index - 1)
-        CommonUtil.validateSplitListInfo(targetListInfo.asScala.toList, splitInfo, originList)
-      } else {
-        CommonUtil.validateAddListInfo(splitInfo, originList)
-      }
-      val addListInfo = PartitionUtils.getListInfo(splitInfo.mkString(","))
-      val newListInfo = partitionId match {
-        case 0 => originList ++ addListInfo
-        case _ => originList.take(index - 1) ++ addListInfo ++
-          originList.takeRight(originList.size - index)
-      }
-      partitionInfo.setListInfo(newListInfo.map(_.asJava).asJava)
-    }
-
-    if (partitionId == 0) {
-      partitionInfo.addPartition(splitInfo.size)
-    } else {
-      partitionInfo.splitPartition(index, splitInfo.size)
-    }
-  }
-
-  /**
-   * Used for alter table partition commands to get segmentProperties in spark node
-   * @param identifier
-   * @param segmentId
-   * @param oldPartitionIdList   Task id group before partition info is changed
-   * @return
-   */
-  def getSegmentProperties(identifier: AbsoluteTableIdentifier, segmentId: String,
-      partitionIds: List[String], oldPartitionIdList: List[Int],
-      partitionInfo: PartitionInfo,
-      carbonTable: CarbonTable): SegmentProperties = {
-    val tableBlockInfoList =
-      getPartitionBlockList(
-        identifier,
-        segmentId,
-        partitionIds,
-        oldPartitionIdList,
-        partitionInfo,
-        carbonTable)
-    val footer = CarbonUtil.readMetadataFile(tableBlockInfoList.get(0))
-    val segmentProperties = new SegmentProperties(footer.getColumnInTable,
-      footer.getSegmentInfo.getColumnCardinality)
-    segmentProperties
-  }
-
-  def getPartitionBlockList(identifier: AbsoluteTableIdentifier, segmentId: String,
-      partitionIds: List[String], oldPartitionIdList: List[Int],
-      partitionInfo: PartitionInfo,
-      carbonTable: CarbonTable): java.util.List[TableBlockInfo] = {
-    val jobConf = new JobConf(FileFactory.getConfiguration)
-    val job = new Job(jobConf)
-    val format = CarbonInputFormatUtil
-      .createCarbonTableInputFormat(identifier, partitionIds.asJava, job)
-    CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
-    val splits = format.getSplitsOfOneSegment(job, segmentId,
-        oldPartitionIdList.map(_.asInstanceOf[Integer]).asJava, partitionInfo)
-    val blockList = splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
-    val tableBlockInfoList = CarbonInputSplit.createBlocks(blockList.asJava)
-    tableBlockInfoList
-  }
-
-  @throws(classOf[IOException])
-  def deleteOriginalCarbonFile(alterPartitionModel: AlterPartitionModel,
-      identifier: AbsoluteTableIdentifier,
-      partitionIds: List[String], dbName: String, tableName: String,
-      partitionInfo: PartitionInfo): Unit = {
-    val carbonLoadModel = alterPartitionModel.carbonLoadModel
-    val segmentId = alterPartitionModel.segmentId
-    val oldPartitionIds = alterPartitionModel.oldPartitionIds
-    val newTime = carbonLoadModel.getFactTimeStamp
-    val tablePath = carbonLoadModel.getTablePath
-    val tableBlockInfoList =
-      getPartitionBlockList(identifier, segmentId, partitionIds, oldPartitionIds,
-        partitionInfo, carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable).asScala
-    val pathList: util.List[String] = new util.ArrayList[String]()
-    tableBlockInfoList.foreach{ tableBlockInfo =>
-      val path = tableBlockInfo.getFilePath
-      val timestamp = CarbonTablePath.DataFileUtil.getTimeStampFromFileName(path)
-      if (timestamp.toLong != newTime) {
-        // add carbondata file
-        pathList.add(path)
-        // add index file
-        val version = tableBlockInfo.getVersion
-        val taskNo = CarbonTablePath.DataFileUtil.getTaskNo(path)
-        val batchNo = CarbonTablePath.DataFileUtil.getBatchNoFromTaskNo(taskNo)
-        val taskId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskNo)
-        val bucketNumber = CarbonTablePath.DataFileUtil.getBucketNo(path)
-        val indexFilePath =
-          new Path(new Path(path).getParent,
-            CarbonTablePath.getCarbonIndexFileName(taskId,
-            bucketNumber.toInt,
-            batchNo,
-            timestamp,
-            segmentId)).toString
-        // indexFilePath could be duplicated when multiple data file related to one index file
-        if (indexFilePath != null && !pathList.contains(indexFilePath)) {
-          pathList.add(indexFilePath)
-        }
-      }
-    }
-    val files: util.List[File] = new util.ArrayList[File]()
-    for (path <- pathList.asScala) {
-      val file = new File(path)
-      files.add(file)
-    }
-    CarbonUtil.deleteFiles(files.asScala.toArray)
-    if (!files.isEmpty) {
-      val carbonTable = alterPartitionModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      val updatedSegFile: String = mergeAndUpdateSegmentFile(alterPartitionModel,
-        identifier,
-        segmentId,
-        carbonTable,
-        files.asScala)
-
-      val segmentFiles = Seq(new Segment(alterPartitionModel.segmentId, updatedSegFile, null))
-        .asJava
-      if (!CarbonUpdateUtil.updateTableMetadataStatus(
-        new util.HashSet[Segment](Seq(new Segment(alterPartitionModel.segmentId,
-          null, null)).asJava),
-        carbonTable,
-        alterPartitionModel.carbonLoadModel.getFactTimeStamp.toString,
-        true,
-        new util.ArrayList[Segment](0),
-        new util.ArrayList[Segment](segmentFiles), "")) {
-        throw new IOException("Data update failed due to failure in table status updation.")
-      }
-    }
-  }
-
   /**
    * Used to extract PartitionerFields for aggregation datamaps.
    * This method will keep generating partitionerFields until the sequence of
@@ -295,49 +79,4 @@ object PartitionUtils {
   }
 
 
-  private def mergeAndUpdateSegmentFile(alterPartitionModel: AlterPartitionModel,
-      identifier: AbsoluteTableIdentifier,
-      segmentId: String,
-      carbonTable: CarbonTable, filesToBeDelete: Seq[File]) = {
-    val metadataDetails =
-      SegmentStatusManager.readTableStatusFile(
-        CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath))
-    val segmentFile =
-      metadataDetails.find(_.getLoadName.equals(segmentId)).get.getSegmentFile
-    var allSegmentFiles: Seq[CarbonFile] = Seq.empty[CarbonFile]
-    val file = SegmentFileStore.writeSegmentFile(
-      carbonTable,
-      alterPartitionModel.segmentId,
-      System.currentTimeMillis().toString)
-    if (segmentFile != null) {
-      allSegmentFiles ++= FileFactory.getCarbonFile(
-        SegmentFileStore.getSegmentFilePath(carbonTable.getTablePath, segmentFile)) :: Nil
-    }
-    val updatedSegFile = {
-      val carbonFile = FileFactory.getCarbonFile(
-        SegmentFileStore.getSegmentFilePath(carbonTable.getTablePath, file))
-      allSegmentFiles ++= carbonFile :: Nil
-
-      val mergedSegFileName = SegmentFileStore.genSegmentFileName(
-        segmentId,
-        alterPartitionModel.carbonLoadModel.getFactTimeStamp.toString)
-      val tmpFile = mergedSegFileName + "_tmp"
-      val segmentStoreFile = SegmentFileStore.mergeSegmentFiles(
-        tmpFile,
-        CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath),
-        allSegmentFiles.toArray)
-      val indexFiles = segmentStoreFile.getLocationMap.values().asScala.head.getFiles
-      filesToBeDelete.foreach(f => indexFiles.remove(f.getName))
-      SegmentFileStore.writeSegmentFile(
-        segmentStoreFile,
-        CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath) +
-        CarbonCommonConstants.FILE_SEPARATOR + mergedSegFileName + CarbonTablePath.SEGMENT_EXT)
-      carbonFile.delete()
-      FileFactory.getCarbonFile(
-        SegmentFileStore.getSegmentFilePath(
-          carbonTable.getTablePath, tmpFile + CarbonTablePath.SEGMENT_EXT)).delete()
-      mergedSegFileName + CarbonTablePath.SEGMENT_EXT
-    }
-    updatedSegFile
-  }
 }
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 57973da..71c5a89 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
@@ -18,26 +18,22 @@
 package org.apache.carbondata.spark.rdd
 
 import java.io.File
-import java.text.SimpleDateFormat
 import java.util
-import java.util.TimeZone
 import java.util.concurrent._
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ListBuffer
 import scala.util.Random
-import scala.util.control.Breaks._
 
 import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapreduce.Job
 import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
 import org.apache.spark.{SparkEnv, TaskContext}
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer, NewHadoopRDD, RDD}
+import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer}
 import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.{CompactionModel, ExecutionErrors, UpdateTableModel}
@@ -57,27 +53,22 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnarFormatVersion, SegmentFileStore}
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion, SegmentFileStore}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope
-import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, ThreadLocalSessionInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.{IndexServerLoadEvent, OperationContext, OperationListenerBus}
+import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.indexserver.{DistributedRDDUtils, IndexServer}
-import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
-import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, StringArrayWritable}
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.processing.util.{Auditor, CarbonDataProcessorUtil, CarbonLoaderUtil}
-import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
+import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
+import org.apache.carbondata.spark.{DataLoadResultImpl, _}
 import org.apache.carbondata.spark.load._
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
@@ -369,12 +360,10 @@ object CarbonDataRDDFactory {
             }
           }
         } else {
-          status = if (carbonTable.getPartitionInfo() != null) {
-            loadDataForPartitionTable(sqlContext, dataFrame, carbonLoadModel, hadoopConf)
-          } else if (dataFrame.isEmpty && isSortTable &&
-                     carbonLoadModel.getRangePartitionColumn != null &&
-                     (sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT) ||
-                      sortScope.equals(SortScopeOptions.SortScope.LOCAL_SORT))) {
+          status = if (dataFrame.isEmpty && isSortTable &&
+                       carbonLoadModel.getRangePartitionColumn != null &&
+                       (sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT) ||
+                        sortScope.equals(SortScopeOptions.SortScope.LOCAL_SORT))) {
             DataLoadProcessBuilderOnSpark
               .loadDataUsingRangeSort(sqlContext.sparkSession, carbonLoadModel, hadoopConf)
           } else if (isSortTable && sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) {
@@ -987,141 +976,6 @@ object CarbonDataRDDFactory {
   }
 
   /**
-   * repartition the input data for partition table.
-   */
-  private def repartitionInputData(
-      sqlContext: SQLContext,
-      dataFrame: Option[DataFrame],
-      carbonLoadModel: CarbonLoadModel,
-      hadoopConf: Configuration): RDD[Row] = {
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionColumn = partitionInfo.getColumnSchemaList.get(0).getColumnName
-    val partitionColumnDataType = partitionInfo.getColumnSchemaList.get(0).getDataType
-    val columns = carbonLoadModel.getCsvHeaderColumns
-    var partitionColumnIndex = -1
-    breakable {
-      for (i <- 0 until columns.length) {
-        if (partitionColumn.equalsIgnoreCase(columns(i))) {
-          partitionColumnIndex = i
-          break
-        }
-      }
-    }
-    if (partitionColumnIndex == -1) {
-      throw new DataLoadingException("Partition column not found.")
-    }
-
-    val specificTimestampFormat = carbonLoadModel.getTimestampformat
-    val specificDateFormat = carbonLoadModel.getDateFormat
-    val timeStampFormat =
-      if (specificTimestampFormat != null && !specificTimestampFormat.trim.isEmpty) {
-        new SimpleDateFormat(specificTimestampFormat)
-      } else {
-        val timestampFormatString = CarbonProperties.getInstance().getProperty(
-          CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-        new SimpleDateFormat(timestampFormatString)
-      }
-
-    val dateFormat = if (specificDateFormat != null && !specificDateFormat.trim.isEmpty) {
-      new SimpleDateFormat(specificDateFormat)
-    } else {
-      val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
-        .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-      new SimpleDateFormat(dateFormatString)
-    }
-    dateFormat.setTimeZone(TimeZone.getTimeZone("GMT"))
-
-    // generate RDD[(K, V)] to use the partitionBy method of PairRDDFunctions
-    val inputRDD: RDD[(String, Row)] = if (dataFrame.isDefined) {
-      // input data from DataFrame
-      val complexDelimiters = carbonLoadModel.getComplexDelimiters
-      val serializationNullFormat =
-        carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
-      dataFrame.get.rdd.map { row =>
-        if (null != row && row.length > partitionColumnIndex &&
-            null != row.get(partitionColumnIndex)) {
-          (CarbonScalaUtil.getString(row.get(partitionColumnIndex), serializationNullFormat,
-            complexDelimiters, timeStampFormat, dateFormat), row)
-        } else {
-          (null, row)
-        }
-      }
-    } else {
-      // input data from csv files
-      CommonUtil.configureCSVInputFormat(hadoopConf, carbonLoadModel)
-      hadoopConf.set(FileInputFormat.INPUT_DIR, carbonLoadModel.getFactFilePath)
-      val columnCount = columns.length
-      val jobConf = new JobConf(hadoopConf)
-      SparkHadoopUtil.get.addCredentials(jobConf)
-      new NewHadoopRDD[NullWritable, StringArrayWritable](
-        sqlContext.sparkContext,
-        classOf[CSVInputFormat],
-        classOf[NullWritable],
-        classOf[StringArrayWritable],
-        jobConf
-      ).map { currentRow =>
-        if (null == currentRow || null == currentRow._2) {
-          val row = new StringArrayRow(new Array[String](columnCount))
-          (null, row)
-        } else {
-          val row = new StringArrayRow(new Array[String](columnCount))
-          val values = currentRow._2.get()
-          if (values != null && values.length > partitionColumnIndex) {
-            (currentRow._2.get()(partitionColumnIndex), row.setValues(currentRow._2.get()))
-          } else {
-            (null, row.setValues(currentRow._2.get()))
-          }
-        }
-      }
-    }
-
-    val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-    if (partitionColumnDataType == DataTypes.STRING) {
-      if (partitionInfo.getPartitionType == PartitionType.RANGE) {
-        inputRDD.map { row => (ByteUtil.toBytes(row._1), row._2) }
-          .partitionBy(partitioner)
-          .map(_._2)
-      } else {
-        inputRDD.partitionBy(partitioner)
-          .map(_._2)
-      }
-    } else {
-      inputRDD.map { row =>
-        (PartitionUtil.getDataBasedOnDataType(row._1, partitionColumnDataType, timeStampFormat,
-          dateFormat), row._2)
-      }
-        .partitionBy(partitioner)
-        .map(_._2)
-    }
-  }
-
-  /**
-   * Execute load process for partition table
-   */
-  private def loadDataForPartitionTable(
-      sqlContext: SQLContext,
-      dataFrame: Option[DataFrame],
-      carbonLoadModel: CarbonLoadModel,
-      hadoopConf: Configuration
-  ): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
-    try {
-      val rdd = repartitionInputData(sqlContext, dataFrame, carbonLoadModel, hadoopConf)
-      new PartitionTableDataLoaderRDD(
-        sqlContext.sparkSession,
-        new DataLoadResultImpl(),
-        carbonLoadModel,
-        rdd
-      ).collect()
-    } catch {
-      case ex: Exception =>
-        LOGGER.error("load data failed for partition table", ex)
-        throw ex
-    }
-  }
-
-  /**
    * Execute load process to load from input dataframe
    */
   private def loadDataFrame(
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
index 6792887..a187b6b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
@@ -70,14 +70,6 @@ private[sql] case class CarbonProjectForUpdateCommand(
       return Array(Row(updatedRowCount)).toSeq
     }
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
-    if (carbonTable.getPartitionInfo != null &&
-      (carbonTable.getPartitionInfo.getPartitionType == PartitionType.RANGE ||
-        carbonTable.getPartitionInfo.getPartitionType == PartitionType.HASH ||
-        carbonTable.getPartitionInfo.getPartitionType == PartitionType.LIST)) {
-      throw new UnsupportedOperationException("Unsupported update operation for range/" +
-        "hash/list partition table")
-    }
-
     setAuditTable(carbonTable)
     setAuditInfo(Map("plan" -> plan.simpleString))
     columns.foreach { col =>
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
deleted file mode 100644
index 2a0d113..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ /dev/null
@@ -1,273 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command.partition
-
-import java.util
-import java.util.concurrent.{Executors, ExecutorService, Future}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.util.AlterTableUtil
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.{DataMapStoreManager, Segment}
-import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.spark.partition.DropPartitionCallable
-
-case class CarbonAlterTableDropPartitionCommand(
-    model: AlterTableDropPartitionModel)
-  extends AtomicRunnableCommand {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-  private val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-
-  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
-    if (model.partitionId.equals("0")) {
-      sys.error(s"Cannot drop default partition! Please use delete statement!")
-    }
-    val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
-    val tableName = model.tableName
-    setAuditTable(dbName, tableName)
-    setAuditInfo(Map("partition" -> model.partitionId))
-    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-    val carbonTable = CarbonEnv.getCarbonTable(Option(dbName), tableName)(sparkSession)
-    val tablePath = carbonTable.getTablePath
-    val partitionInfo = carbonTable.getPartitionInfo()
-    if (partitionInfo == null) {
-      throwMetadataException(dbName, tableName, "table is not a partition table")
-    }
-    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
-    // keep a copy of partitionIdList before update partitionInfo.
-    // will be used in partition data scan
-    oldPartitionIds.addAll(partitionIds.asJava)
-    val partitionIndex = partitionIds.indexOf(Integer.valueOf(model.partitionId))
-    partitionInfo.getPartitionType match {
-      case PartitionType.HASH => sys.error(s"Hash partition cannot be dropped!")
-      case PartitionType.RANGE =>
-        val rangeInfo = new util.ArrayList(partitionInfo.getRangeInfo)
-        val rangeToRemove = partitionInfo.getRangeInfo.get(partitionIndex - 1)
-        rangeInfo.remove(rangeToRemove)
-        partitionInfo.setRangeInfo(rangeInfo)
-      case PartitionType.LIST =>
-        val listInfo = new util.ArrayList(partitionInfo.getListInfo)
-        val listToRemove = partitionInfo.getListInfo.get(partitionIndex - 1)
-        listInfo.remove(listToRemove)
-        partitionInfo.setListInfo(listInfo)
-      case PartitionType.RANGE_INTERVAL =>
-        throwMetadataException(dbName, tableName,
-          "Dropping range interval partition is unsupported")
-      case _ => throw new UnsupportedOperationException(
-        s"${partitionInfo.getPartitionType} is not supported")
-    }
-    partitionInfo.dropPartition(partitionIndex)
-
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, tablePath)
-    val tableSchema = wrapperTableInfo.getFactTable
-    tableSchema.setPartitionInfo(partitionInfo)
-    wrapperTableInfo.setFactTable(tableSchema)
-    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    val thriftTable =
-      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-      .setTime_stamp(System.currentTimeMillis)
-    carbonMetaStore.updateTableSchemaForAlter(
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-      thriftTable,
-      null,
-      carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
-    // sparkSession.catalog.refreshTable(tableName)
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
-    val tableName = model.tableName
-    var locks = List.empty[ICarbonLock]
-    var success = false
-    try {
-      val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-        LockUsage.COMPACTION_LOCK,
-        LockUsage.DELETE_SEGMENT_LOCK,
-        LockUsage.DROP_TABLE_LOCK,
-        LockUsage.CLEAN_FILES_LOCK,
-        LockUsage.ALTER_PARTITION_LOCK)
-      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
-        locksToBeAcquired)(sparkSession)
-      val carbonLoadModel = new CarbonLoadModel()
-      val table = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      // Need to fill dimension relation
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-      carbonLoadModel.setTableName(table.getTableName)
-      carbonLoadModel.setCarbonTransactionalTable(table.isTransactionalTable)
-      carbonLoadModel.setDatabaseName(table.getDatabaseName)
-      carbonLoadModel.setTablePath(table.getTablePath)
-      val loadStartTime = CarbonUpdateUtil.readCurrentTime
-      carbonLoadModel.setFactTimeStamp(loadStartTime)
-      val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
-        .getOrElse(CarbonCommonConstants.COMPRESSOR,
-          CompressorFactory.getInstance().getCompressor.getName)
-      carbonLoadModel.setColumnCompressor(columnCompressor)
-      alterTableDropPartition(
-        sparkSession.sqlContext,
-        model.partitionId,
-        carbonLoadModel,
-        model.dropWithData,
-        oldPartitionIds.asScala.toList
-      )
-      success = true
-    } catch {
-      case e: Exception =>
-        sys.error(s"Drop Partition failed. Please check logs for more info. ${ e.getMessage } ")
-        success = false
-    } finally {
-      CacheProvider.getInstance().dropAllCache()
-      AlterTableUtil.releaseLocks(locks)
-      LOGGER.info("Locks released after alter table drop partition action.")
-    }
-    LOGGER.info(s"Alter table drop partition is successful for table $dbName.$tableName")
-    Seq.empty
-  }
-
-  private def alterTableDropPartition(sqlContext: SQLContext,
-      partitionId: String,
-      carbonLoadModel: CarbonLoadModel,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]): Unit = {
-    try {
-      startDropThreads(
-        sqlContext,
-        carbonLoadModel,
-        partitionId,
-        dropWithData,
-        oldPartitionIds)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception in start dropping partition thread. ${ e.getMessage }")
-        throw e
-    }
-  }
-
-  private def startDropThreads(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      partitionId: String,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]): Unit = {
-    val numberOfCores = CarbonProperties.getInstance().getNumberOfAltPartitionCores
-    val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores.toInt)
-    try {
-      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
-      val validSegments = segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTable)
-        .getValidSegments.asScala
-      val threadArray: Array[Thread] = new Array[Thread](validSegments.size)
-      var i = 0
-      for (segmentId: Segment <- validSegments) {
-        threadArray(i) = dropPartitionThread(sqlContext, carbonLoadModel, executor,
-          segmentId, partitionId, dropWithData, oldPartitionIds)
-        threadArray(i).start()
-        i += 1
-      }
-      for (thread <- threadArray) {
-        thread.join()
-      }
-      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(carbonTable)
-      refresher.refreshSegments(validSegments.map(_.getSegmentNo).asJava)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception when dropping partition: ${ e.getMessage }")
-    } finally {
-      executor.shutdown()
-      try {
-        TableProcessingOperations
-          .deletePartialLoadDataIfExist(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable,
-            false)
-      } catch {
-        case e: Exception =>
-          LOGGER.error(s"Exception in dropping partition thread while deleting partial load file" +
-                       s" ${ e.getMessage }")
-      }
-    }
-  }
-
-  override protected def opName: String = "DROP CUSTOM PARTITION"
-}
-
-case class dropPartitionThread(sqlContext: SQLContext,
-    carbonLoadModel: CarbonLoadModel,
-    executor: ExecutorService,
-    segmentId: Segment,
-    partitionId: String,
-    dropWithData: Boolean,
-    oldPartitionIds: List[Int]) extends Thread {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  override def run(): Unit = {
-    try {
-      executeDroppingPartition(sqlContext, carbonLoadModel, executor,
-        segmentId, partitionId, dropWithData, oldPartitionIds)
-    } catch {
-      case e: Exception =>
-        val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-        LOGGER.error(s"Exception in dropping partition thread: ${ e.getMessage } }", e)
-    }
-  }
-
-  private def executeDroppingPartition(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      executor: ExecutorService,
-      segmentId: Segment,
-      partitionId: String,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]): Unit = {
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val model = new DropPartitionCallableModel(carbonLoadModel,
-      segmentId, partitionId, oldPartitionIds, dropWithData, carbonTable, sqlContext)
-    val future: Future[Void] = executor.submit(new DropPartitionCallable(model))
-    try {
-      future.get
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception in partition drop thread ${ e.getMessage }", e)
-        throw e
-    }
-  }
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
deleted file mode 100644
index d880ddf..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ /dev/null
@@ -1,323 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command.partition
-
-import java.text.SimpleDateFormat
-import java.util
-import java.util.concurrent.{Executors, ExecutorService, Future}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.util.{AlterTableUtil, PartitionUtils}
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.DataMapStoreManager
-import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.events.{OperationContext, OperationListenerBus, PostAlterTableHivePartitionCommandEvent, PreAlterTableHivePartitionCommandEvent}
-import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.spark.partition.SplitPartitionCallable
-
-/**
- * Command for Alter Table Add & Split partition
- * Add is a special case of Splitting the default partition (part0)
- */
-case class CarbonAlterTableSplitPartitionCommand(
-    splitPartitionModel: AlterTableSplitPartitionModel)
-  extends AtomicRunnableCommand {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-  private val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-
-  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
-    val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
-    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-    val tableName = splitPartitionModel.tableName
-    setAuditTable(dbName, tableName)
-    setAuditInfo(Map("partition" -> splitPartitionModel.partitionId))
-    val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
-      .asInstanceOf[CarbonRelation]
-    val tablePath = relation.carbonTable.getTablePath
-    if (relation == null) {
-      throwMetadataException(dbName, tableName, "table not found")
-    }
-    if (null == (CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession))) {
-      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
-      throwMetadataException(dbName, tableName, "table not found")
-    }
-    val carbonTable = relation.carbonTable
-    val partitionInfo = carbonTable.getPartitionInfo
-    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
-    // keep a copy of partitionIdList before update partitionInfo.
-    // will be used in partition data scan
-    oldPartitionIds.addAll(partitionIds.asJava)
-
-    if (partitionInfo == null) {
-      throwMetadataException(dbName, tableName, "Table is not a partition table.")
-    }
-    if (partitionInfo.getPartitionType == PartitionType.HASH) {
-      throwMetadataException(dbName, tableName, "Hash partition table cannot be added or split!")
-    }
-
-    updatePartitionInfo(partitionInfo, partitionIds)
-
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, tablePath)
-    val tableSchema = wrapperTableInfo.getFactTable
-    tableSchema.setPartitionInfo(partitionInfo)
-    wrapperTableInfo.setFactTable(tableSchema)
-    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    val thriftTable =
-      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    carbonMetaStore.updateTableSchemaForAlter(
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-      thriftTable,
-      null,
-      carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
-    Seq.empty
-  }
-
-  private def updatePartitionInfo(partitionInfo: PartitionInfo, partitionIds: List[Int]): Unit = {
-    val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-      .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-
-    val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-
-    PartitionUtils.updatePartitionInfo(
-      partitionInfo,
-      partitionIds,
-      splitPartitionModel.partitionId.toInt,
-      splitPartitionModel.splitInfo,
-      timestampFormatter,
-      dateFormatter)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
-    val tableName = splitPartitionModel.tableName
-    var locks = List.empty[ICarbonLock]
-    var success = false
-    try {
-      val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-        LockUsage.COMPACTION_LOCK,
-        LockUsage.DELETE_SEGMENT_LOCK,
-        LockUsage.DROP_TABLE_LOCK,
-        LockUsage.CLEAN_FILES_LOCK,
-        LockUsage.ALTER_PARTITION_LOCK)
-      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
-        locksToBeAcquired)(sparkSession)
-      val table = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-      val carbonLoadModel = new CarbonLoadModel()
-      val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
-        .getOrElse(CarbonCommonConstants.COMPRESSOR,
-          CompressorFactory.getInstance().getCompressor.getName)
-      carbonLoadModel.setColumnCompressor(columnCompressor)
-      val tablePath = table.getTablePath
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-      carbonLoadModel.setTableName(table.getTableName)
-      carbonLoadModel.setDatabaseName(table.getDatabaseName)
-      carbonLoadModel.setCarbonTransactionalTable(table.isTransactionalTable)
-      carbonLoadModel.setTablePath(tablePath)
-      val loadStartTime = CarbonUpdateUtil.readCurrentTime
-      carbonLoadModel.setFactTimeStamp(loadStartTime)
-      val operationContext = new OperationContext
-      val preAlterTableHivePartitionCommandEvent = PreAlterTableHivePartitionCommandEvent(
-        sparkSession,
-        table)
-      OperationListenerBus.getInstance()
-        .fireEvent(preAlterTableHivePartitionCommandEvent, operationContext)
-      alterTableSplitPartition(
-        sparkSession.sqlContext,
-        splitPartitionModel.partitionId.toInt.toString,
-        carbonLoadModel,
-        oldPartitionIds.asScala.toList
-      )
-      val postAlterTableHivePartitionCommandEvent = PostAlterTableHivePartitionCommandEvent(
-        sparkSession,
-        table)
-      OperationListenerBus.getInstance()
-        .fireEvent(postAlterTableHivePartitionCommandEvent, operationContext)
-      success = true
-    } catch {
-      case e: Exception =>
-        success = false
-        sys.error(s"Add/Split Partition failed. Please check logs for more info. ${ e.getMessage }")
-    } finally {
-      AlterTableUtil.releaseLocks(locks)
-      CacheProvider.getInstance().dropAllCache()
-      val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-      LOGGER.info("Locks released after alter table add/split partition action.")
-      if (success) {
-        LOGGER.info(s"Alter table add/split partition is successful for table $dbName.$tableName")
-      }
-    }
-    Seq.empty
-  }
-
-  private def alterTableSplitPartition(
-      sqlContext: SQLContext,
-      partitionId: String,
-      carbonLoadModel: CarbonLoadModel,
-      oldPartitionIdList: List[Int]
-  ): Unit = {
-    try {
-      startSplitThreads(sqlContext,
-        carbonLoadModel,
-        partitionId,
-        oldPartitionIdList)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception in start splitting partition thread. ${ e.getMessage }")
-        throw e
-    }
-  }
-
-  private def startSplitThreads(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      partitionId: String,
-      oldPartitionIdList: List[Int]): Unit = {
-    val numberOfCores = CarbonProperties.getInstance().getNumberOfAltPartitionCores
-    val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores)
-    try {
-      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
-      val validSegments = segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTable)
-        .getValidSegments.asScala
-      val threadArray: Array[SplitThread] = new Array[SplitThread](validSegments.size)
-      var i = 0
-      validSegments.foreach { segmentId =>
-        threadArray(i) = SplitThread(sqlContext, carbonLoadModel, executor,
-          segmentId.getSegmentNo, partitionId, oldPartitionIdList)
-        threadArray(i).start()
-        i += 1
-      }
-      threadArray.foreach {
-        thread => thread.join()
-      }
-      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(carbonTable)
-      refresher.refreshSegments(validSegments.map(_.getSegmentNo).asJava)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception when split partition: ${ e.getMessage }")
-        throw e
-    } finally {
-      executor.shutdown()
-      try {
-        TableProcessingOperations
-          .deletePartialLoadDataIfExist(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable,
-            false)
-      } catch {
-        case e: Exception =>
-          LOGGER.error(s"Exception in add/split partition thread while deleting partial load file" +
-                       s" ${ e.getMessage }")
-      }
-    }
-  }
-
-  override protected def opName: String = "SPLIT CUSTOM PARTITION"
-}
-
-case class SplitThread(sqlContext: SQLContext,
-    carbonLoadModel: CarbonLoadModel,
-    executor: ExecutorService,
-    segmentId: String,
-    partitionId: String,
-    oldPartitionIdList: List[Int]) extends Thread {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  override def run(): Unit = {
-    var triggeredSplitPartitionStatus = false
-    var exception: Exception = null
-    try {
-      executePartitionSplit(sqlContext,
-        carbonLoadModel, executor, segmentId, partitionId, oldPartitionIdList)
-      triggeredSplitPartitionStatus = true
-    } catch {
-      case e: Exception =>
-        val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-        LOGGER.error(s"Exception in partition split thread: ${ e.getMessage } }")
-        exception = e
-    }
-    if (!triggeredSplitPartitionStatus) {
-      throw new Exception("Exception in split partition " + exception.getMessage)
-    }
-  }
-
-  private def executePartitionSplit( sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      executor: ExecutorService,
-      segment: String,
-      partitionId: String,
-      oldPartitionIdList: List[Int]): Unit = {
-    val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
-      CarbonCommonConstants.DEFAULT_COLLECTION_SIZE
-    )
-    scanSegmentsForSplitPartition(futureList, executor, segment, partitionId,
-      sqlContext, carbonLoadModel, oldPartitionIdList)
-    try {
-      futureList.asScala.foreach { future =>
-        future.get
-      }
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception in partition split thread ${ e.getMessage }", e)
-        throw e
-    }
-  }
-
-  private def scanSegmentsForSplitPartition(futureList: util.List[Future[Void]],
-      executor: ExecutorService,
-      segmentId: String,
-      partitionId: String,
-      sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      oldPartitionIdList: List[Int]): Unit = {
-
-    val splitModel = SplitPartitionCallableModel(carbonLoadModel,
-      segmentId,
-      partitionId,
-      oldPartitionIdList,
-      sqlContext)
-
-    val future: Future[Void] = executor.submit(new SplitPartitionCallable(splitModel))
-    futureList.add(future)
-  }
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
deleted file mode 100644
index 4bcdec4b..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command.partition
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.execution.command.MetadataCommand
-import org.apache.spark.sql.hive.CarbonRelation
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.spark.util.CommonUtil
-
-/**
- * Command for show table partitions Command
- */
-private[sql] case class CarbonShowCarbonPartitionsCommand(
-    tableIdentifier: TableIdentifier)
-  extends MetadataCommand {
-
-  override val output: Seq[Attribute] = CommonUtil.partitionInfoOutput
-
-  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-      .lookupRelation(tableIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
-    val carbonTable = relation.carbonTable
-    setAuditTable(carbonTable)
-    val partitionInfo = carbonTable.getPartitionInfo
-    if (partitionInfo == null) {
-      throwMetadataException(carbonTable.getDatabaseName, carbonTable.getTableName,
-        "SHOW PARTITIONS is not allowed on a table that is not partitioned")
-    }
-    val partitionType = partitionInfo.getPartitionType
-    val columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
-    val LOGGER = LogServiceFactory.getLogService(CarbonShowCarbonPartitionsCommand.getClass.getName)
-    LOGGER.info("partition column name:" + columnName)
-    CommonUtil.getPartitionInfo(columnName, partitionType, partitionInfo)
-  }
-
-  override protected def opName: String = "SHOW CUSTOM PARTITION"
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
index d795d63..8810425 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.command.MetadataCommand
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.common.Strings
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
@@ -197,14 +198,8 @@ private[sql] case class CarbonDescribeFormattedCommand(
         ("Partition Columns",
           partitionInfo.getColumnSchemaList.asScala.map {
             col => s"${col.getColumnName}:${col.getDataType.getName}"}.mkString(", "), ""),
-        ("Number of Partitions", getNumberOfPartitions(carbonTable, sparkSession), ""),
-        ("Partitions Ids", partitionInfo.getPartitionIds.asScala.mkString(","), "")
+        ("Number of Partitions", getNumberOfPartitions(carbonTable, sparkSession), "")
       )
-      if (partitionInfo.getPartitionType == PartitionType.RANGE) {
-        results ++= Seq(("Range", partitionInfo.getRangeInfo.asScala.mkString(", "), ""))
-      } else if (partitionInfo.getPartitionType == PartitionType.LIST) {
-        results ++= Seq(("List", partitionInfo.getListInfo.asScala.mkString(", "), ""))
-      }
     }
     if (partitionSpec.nonEmpty) {
       val partitions = sparkSession.sessionState.catalog.getPartition(tblIdentifier, partitionSpec)
@@ -263,7 +258,7 @@ private[sql] case class CarbonDescribeFormattedCommand(
           .listPartitions(new TableIdentifier(carbonTable.getTableName,
             Some(carbonTable.getDatabaseName))).size.toString
       case _ =>
-        carbonTable.getPartitionInfo.getNumPartitions.toString
+        throw new DeprecatedFeatureException("Custom Partition")
     }
   }
 
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index 37a33ec..d51e85a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.management.{CarbonAlterTableCompactionCommand, CarbonInsertIntoCommand, CarbonLoadDataCommand, RefreshCarbonTableCommand}
-import org.apache.spark.sql.execution.command.partition.{CarbonAlterTableAddHivePartitionCommand, CarbonAlterTableDropHivePartitionCommand, CarbonShowCarbonPartitionsCommand}
+import org.apache.spark.sql.execution.command.partition.{CarbonAlterTableAddHivePartitionCommand, CarbonAlterTableDropHivePartitionCommand}
 import org.apache.spark.sql.execution.command.schema._
 import org.apache.spark.sql.execution.command.table.{CarbonCreateTableLikeCommand, CarbonDescribeFormattedCommand, CarbonDropTableCommand}
 import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
@@ -35,6 +35,7 @@ import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.types.StructField
 import org.apache.spark.util.{CarbonReflectionUtils, DataMapUtil, FileUtils, SparkUtil}
 
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.util.{CarbonProperties, DataTypeUtil, ThreadLocalSessionInfo}
@@ -228,24 +229,6 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
         } else {
           Nil
         }
-      case ShowPartitionsCommand(t, cols) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-          .tableExists(t)(sparkSession)
-        if (isCarbonTable) {
-          val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-            .lookupRelation(t)(sparkSession).asInstanceOf[CarbonRelation].carbonTable
-          if (carbonTable != null && !carbonTable.getTableInfo.isTransactionalTable) {
-            throw new MalformedCarbonCommandException(
-              "Unsupported operation on non transactional table")
-          }
-          if (!carbonTable.isHivePartitionTable) {
-            ExecutedCommandExec(CarbonShowCarbonPartitionsCommand(t)) :: Nil
-          } else {
-            ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
-          }
-        } else {
-          ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
-        }
       case adp@AlterTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData) =>
         val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetaStore
           .tableExists(tableName)(sparkSession)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 82ea8f6..81923ea 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.datamap.{CarbonCreateDataMapCommand, CarbonDataMapRebuildCommand, CarbonDataMapShowCommand, CarbonDropDataMapCommand}
 import org.apache.spark.sql.execution.command.management._
-import org.apache.spark.sql.execution.command.partition.{CarbonAlterTableDropPartitionCommand, CarbonAlterTableSplitPartitionCommand}
 import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableColRenameDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
 import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
 import org.apache.spark.sql.types.StructField
@@ -77,7 +76,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
 
   protected lazy val startCommand: Parser[LogicalPlan] =
     loadManagement | showLoads | alterTable | restructure | updateTable | deleteRecords |
-    alterPartition | datamapManagement | alterTableFinishStreaming | stream | cli |
+    datamapManagement | alterTableFinishStreaming | stream | cli |
     cacheManagement | alterDataMap
 
   protected lazy val loadManagement: Parser[LogicalPlan] =
@@ -86,9 +85,6 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   protected lazy val restructure: Parser[LogicalPlan] =
     alterTableColumnRenameAndModifyDataType | alterTableDropColumn | alterTableAddColumns
 
-  protected lazy val alterPartition: Parser[LogicalPlan] =
-    alterAddPartition | alterSplitPartition | alterDropPartition
-
   protected lazy val datamapManagement: Parser[LogicalPlan] =
     createDataMap | dropDataMap | showDataMap | refreshDataMap
 
@@ -98,40 +94,6 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   protected lazy val cacheManagement: Parser[LogicalPlan] =
     showCache | dropCache
 
-  protected lazy val alterAddPartition: Parser[LogicalPlan] =
-    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (ADD ~> PARTITION ~>
-      "(" ~> repsep(stringLit, ",") <~ ")") <~ opt(";") ^^ {
-      case dbName ~ table ~ addInfo =>
-        val alterTableAddPartitionModel = AlterTableSplitPartitionModel(dbName, table, "0", addInfo)
-        CarbonAlterTableSplitPartitionCommand(alterTableAddPartitionModel)
-    }
-
-  protected lazy val alterSplitPartition: Parser[LogicalPlan] =
-    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (SPLIT ~> PARTITION ~>
-       "(" ~> numericLit <~ ")") ~ (INTO ~> "(" ~> repsep(stringLit, ",") <~ ")") <~ opt(";") ^^ {
-      case dbName ~ table ~ partitionId ~ splitInfo =>
-        val alterTableSplitPartitionModel =
-          AlterTableSplitPartitionModel(dbName, table, partitionId, splitInfo)
-        if (partitionId == 0) {
-          sys.error("Please use [Alter Table Add Partition] statement to split default partition!")
-        }
-        CarbonAlterTableSplitPartitionCommand(alterTableSplitPartitionModel)
-    }
-
-  protected lazy val alterDropPartition: Parser[LogicalPlan] =
-    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (DROP ~> PARTITION ~>
-      "(" ~> numericLit <~ ")") ~ (WITH ~> DATA).? <~ opt(";") ^^ {
-      case dbName ~ table ~ partitionId ~ withData =>
-        val dropWithData = withData.getOrElse("NO") match {
-          case "NO" => false
-          case _ => true
-        }
-        val alterTableDropPartitionModel =
-          AlterTableDropPartitionModel(dbName, table, partitionId, dropWithData)
-        CarbonAlterTableDropPartitionCommand(alterTableDropPartitionModel)
-    }
-
-
   protected lazy val alterTable: Parser[LogicalPlan] =
     ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (COMPACT ~ stringLit) ~
       (WHERE ~> (SEGMENT ~ "." ~ ID) ~> IN ~> "(" ~> repsep(segmentId, ",") <~ ")").? <~
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
index b720a19..9591bc0 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
@@ -39,7 +39,7 @@ import org.apache.carbondata.core.metadata.schema.SchemaReader
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.spark.CarbonOption
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 /**
  * Utility class to validate the create table and CTAS command,
@@ -271,9 +271,6 @@ object CarbonSparkSqlParserUtil {
     }
     // validate partition clause
     if (partitionerFields.nonEmpty) {
-      if (!CommonUtil.validatePartitionColumns(tableProperties, partitionerFields)) {
-        throw new MalformedCarbonCommandException("Error: Invalid partition definition")
-      }
       // partition columns should not be part of the schema
       val badPartCols = partitionerFields.map(_.partitionColumn.toLowerCase).toSet
         .intersect(colNames.map(_.toLowerCase).toSet)
@@ -302,26 +299,11 @@ object CarbonSparkSqlParserUtil {
         s"Values must be specified for key(s): ${ badKeys.mkString("[", ",", "]") }", ctx)
     }
     props.map { case (key, value) =>
-      if (needToConvertToLowerCase(key)) {
-        (key.toLowerCase, value.toLowerCase)
-      } else {
-        (key.toLowerCase, value)
-      }
+      (key.toLowerCase, value)
     }
   }
 
   /**
-   * check's whether need to convert to lower case
-   *
-   * @param key <String> property key
-   * @return returns <true> if lower case conversion is needed else <false>
-   */
-  def needToConvertToLowerCase(key: String): Boolean = {
-    val noConvertList = Array("LIST_INFO", "RANGE_INFO", "PATH")
-    !noConvertList.exists(x => x.equalsIgnoreCase(key))
-  }
-
-  /**
    * The method validate the create table command and returns the table's columns.
    *
    * @param tableHeader       An instance of CreateTableHeaderContext having parser rules for
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
deleted file mode 100644
index bb04d0a..0000000
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ /dev/null
@@ -1,926 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.testsuite.partition
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-
-import org.apache.hadoop.fs.Path
-import org.apache.spark.sql.CarbonEnv
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{CarbonMetadata, SegmentFileStore}
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-
-class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
-
-
-  override def beforeAll {
-    dropTable
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-      // deactivating the merge-index for old partition implimentation because we are not supporting
-      // merge-index for the same currently.
-        .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,"false")
-    /**
-     * list_table_area_origin
-     * list_table_area
-     */
-    sql("""
-          | CREATE TABLE IF NOT EXISTS list_table_area_origin
-          | (
-          | id Int,
-          | vin string,
-          | logdate Timestamp,
-          | phonenumber Long,
-          | country string,
-          | salary Int
-          | )
-          | PARTITIONED BY (area string)
-          | STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-          | 'LIST_INFO'='Asia, America, Europe')
-        """.stripMargin)
-    sql("""
-          | CREATE TABLE IF NOT EXISTS list_table_area
-          | (
-          | id Int,
-          | vin string,
-          | logdate Timestamp,
-          | phonenumber Long,
-          | country string,
-          | salary Int
-          | )
-          | PARTITIONED BY (area string)
-          | STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-          | 'LIST_INFO'='Asia, America, Europe')
-        """.stripMargin)
-
-    /**
-     * range_table_logdate_origin
-     * range_table_logdate
-     */
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS range_table_logdate_origin
-        | (
-        | id Int,
-        | vin string,
-        | phonenumber Long,
-        | country string,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (logdate Timestamp)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01')
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS range_table_logdate
-        | (
-        | id Int,
-        | vin string,
-        | phonenumber Long,
-        | country string,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (logdate Timestamp)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01', 'DICTIONARY_INCLUDE'='logdate')
-      """.stripMargin)
-
-    /**
-     * list_table_country_origin
-     * list_table_country
-     */
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS list_table_country_origin
-        | (
-        | id Int,
-        | vin string,
-        | logdate Timestamp,
-        | phonenumber Long,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (country string)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, NotGood), Korea ')
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS list_table_country
-        | (
-        | id Int,
-        | vin string,
-        | logdate Timestamp,
-        | phonenumber Long,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (country string)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
-        | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, NotGood), Korea ')
-      """.stripMargin)
-
-    /**
-     * range_table_logdate_split_origin
-     * range_table_logdate_split
-     */
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS range_table_logdate_split_origin
-        | (
-        | id Int,
-        | vin string,
-        | phonenumber Long,
-        | country string,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (logdate Timestamp)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01')
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS range_table_logdate_split
-        | (
-        | id Int,
-        | vin string,
-        | phonenumber Long,
-        | country string,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (logdate Timestamp)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01',
-        | 'DICTIONARY_INCLUDE'='logdate')
-      """.stripMargin)
-
-    /**
-     * range_table_bucket_origin
-     * range_table_bucket
-     */
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS range_table_bucket_origin
-        | (
-        | id Int,
-        | vin string,
-        | phonenumber Long,
-        | country string,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (logdate Timestamp)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01',
-        | 'BUCKETNUMBER'='3',
-        | 'BUCKETCOLUMNS'='country')
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE IF NOT EXISTS range_table_bucket
-        | (
-        | id Int,
-        | vin string,
-        | phonenumber Long,
-        | country string,
-        | area string,
-        | salary Int
-        | )
-        | PARTITIONED BY (logdate Timestamp)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
-        | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01',
-        | 'DICTIONARY_INCLUDE'='logdate',
-        | 'BUCKETNUMBER'='3',
-        | 'BUCKETCOLUMNS'='country')
-      """.stripMargin)
-
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_area_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_country_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate_split_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_bucket_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_area OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_country OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate_split OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_bucket OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-  }
-
-  test("Alter table add partition: List Partition") {
-    sql("""ALTER TABLE list_table_area ADD PARTITION ('OutSpace', 'Hi')""".stripMargin)
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "list_table_area")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val list_info = partitionInfo.getListInfo
-    assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 5)
-    assert(partitionInfo.getNumPartitions == 6)
-    assert(list_info.get(0).get(0) == "Asia")
-    assert(list_info.get(1).get(0) == "America")
-    assert(list_info.get(2).get(0) == "Europe")
-    assert(list_info.get(3).get(0) == "OutSpace")
-    assert(list_info.get(4).get(0) == "Hi")
-    validateDataFiles("default_list_table_area", "0", Seq(1, 2, 4))
-    val result_after = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area")
-    val result_origin = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin")
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql(s"select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area < 'OutSpace' ")
-    val result_origin1 = sql(s"select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area < 'OutSpace' ")
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area <= 'OutSpace' ")
-    val result_origin2 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <= 'OutSpace' ")
-    checkAnswer(result_after2, result_origin2)
-
-    val result_after3 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area = 'OutSpace' ")
-    val result_origin3 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area = 'OutSpace' ")
-    checkAnswer(result_after3, result_origin3)
-
-    val result_after4 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area > 'OutSpace' ")
-    val result_origin4 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area > 'OutSpace' ")
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area >= 'OutSpace' ")
-    val result_origin5 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area >= 'OutSpace' ")
-    checkAnswer(result_after5, result_origin5)
-
-    intercept[Exception]  { sql("""ALTER TABLE DROP PARTITION(0)""")}
-    intercept[Exception]  { sql("""ALTER TABLE DROP PARTITION(0) WITH DATA""")}
-    
-    sql("""ALTER TABLE list_table_area DROP PARTITION(2) WITH DATA""")
-    val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_area")
-    val partitionInfo2 = carbonTable2.getPartitionInfo()
-    val partitionIds2 = partitionInfo2.getPartitionIds
-    val list_info2 = partitionInfo2.getListInfo
-    assert(partitionIds2 == List(0, 1, 3, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo2.getMaxPartitionId == 5)
-    assert(partitionInfo2.getNumPartitions == 5)
-    assert(list_info2.get(0).get(0) == "Asia")
-    assert(list_info2.get(1).get(0) == "Europe")
-    assert(list_info2.get(2).get(0) == "OutSpace")
-    assert(list_info2.get(3).get(0) == "Hi")
-    validateDataFiles("default_list_table_area", "0", Seq(1, 4))
-    checkAnswer(sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area"),
-      sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <> 'America' "))
-  }
-
-  ignore("Alter table add partition: Range Partition") {
-    sql("""ALTER TABLE range_table_logdate ADD PARTITION ('2017/01/01', '2018/01/01')""")
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "range_table_logdate")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val range_info = partitionInfo.getRangeInfo
-    assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 5)
-    assert(partitionInfo.getNumPartitions == 6)
-    assert(range_info.get(0) == "2014/01/01")
-    assert(range_info.get(1) == "2015/01/01")
-    assert(range_info.get(2) == "2016/01/01")
-    assert(range_info.get(3) == "2017/01/01")
-    assert(range_info.get(4) == "2018/01/01")
-    validateDataFiles("default_range_table_logdate", "0", Seq(1, 2, 3, 4, 5))
-    val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin""")
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after2, result_origin2)
-
-    val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after3, result_origin3)
-
-    val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after5, result_origin5)
-
-    sql("""ALTER TABLE range_table_logdate DROP PARTITION(3) WITH DATA;""")
-    val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable(
-      "default",
-      "range_table_logdate"
-    )
-    val partitionInfo1 = carbonTable1.getPartitionInfo()
-    val partitionIds1 = partitionInfo1.getPartitionIds
-    val range_info1 = partitionInfo1.getRangeInfo
-    assert(partitionIds1 == List(0, 1, 2, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMaxPartitionId == 5)
-    assert(partitionInfo1.getNumPartitions == 5)
-    assert(range_info1.get(0) == "2014/01/01")
-    assert(range_info1.get(1) == "2015/01/01")
-    assert(range_info1.get(2) == "2017/01/01")
-    assert(range_info1.get(3) == "2018/01/01")
-    assert(range_info1.size() == 4)
-    validateDataFiles("default_range_table_logdate", "0", Seq(1, 2, 4, 5))
-    val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate""")
-    val result_origin6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate < '2015/01/01 00:00:00' or logdate >= '2016/01/01 00:00:00' """)
-    checkAnswer(result_after6, result_origin6)
-  }
-
-  test("test exception if invalid partition id is provided in alter command") {
-    sql("drop table if exists test_invalid_partition_id")
-
-    sql("CREATE TABLE test_invalid_partition_id (CUST_NAME String,ACTIVE_EMUI_VERSION string,DOB Timestamp,DOJ timestamp, " +
-      "BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10)," +
-      "Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (CUST_ID int)" +
-      " STORED BY 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES ('PARTITION_TYPE'='RANGE','RANGE_INFO'='9090,9500,9800',\"TABLE_BLOCKSIZE\"= \"256 MB\")")
-    intercept[IllegalArgumentException] { sql("ALTER TABLE test_invalid_partition_id SPLIT PARTITION(6) INTO ('9800','9900')") }
-  }
-
-  test("Alter table split partition: List Partition") {
-    sql("""ALTER TABLE list_table_country SPLIT PARTITION(4) INTO ('Canada', 'Russia', '(Good, NotGood)')""".stripMargin)
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_country")
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val list_info = partitionInfo.getListInfo
-    assert(partitionIds == List(0, 1, 2, 3, 6, 7, 8, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 8)
-    assert(partitionInfo.getNumPartitions == 8)
-    assert(list_info.get(0).get(0) == "China")
-    assert(list_info.get(0).get(1) == "US")
-    assert(list_info.get(1).get(0) == "UK")
-    assert(list_info.get(2).get(0) == "Japan")
-    assert(list_info.get(3).get(0) == "Canada")
-    assert(list_info.get(4).get(0) == "Russia")
-    assert(list_info.get(5).get(0) == "Good")
-    assert(list_info.get(5).get(1) == "NotGood")
-    assert(list_info.get(6).get(0) == "Korea")
-    validateDataFiles("default_list_table_country", "0", Seq(1, 2, 3, 8))
-    val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country < 'NotGood' """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country < 'NotGood' """)
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country <= 'NotGood' """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country <= 'NotGood' """)
-    checkAnswer(result_after2, result_origin2)
-
-    val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country = 'NotGood' """)
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country = 'NotGood' """)
-    checkAnswer(result_after3, result_origin3)
-
-    val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country >= 'NotGood' """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country >= 'NotGood' """)
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country > 'NotGood' """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country > 'NotGood' """)
-    checkAnswer(result_after5, result_origin5)
-
-    sql("""ALTER TABLE list_table_country DROP PARTITION(8)""")
-    val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_country")
-    val partitionInfo1 = carbonTable1.getPartitionInfo()
-    val partitionIds1 = partitionInfo1.getPartitionIds
-    val list_info1 = partitionInfo1.getListInfo
-    assert(partitionIds1 == List(0, 1, 2, 3, 6, 7, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMaxPartitionId == 8)
-    assert(partitionInfo1.getNumPartitions == 7)
-    assert(list_info1.get(0).get(0) == "China")
-    assert(list_info1.get(0).get(1) == "US")
-    assert(list_info1.get(1).get(0) == "UK")
-    assert(list_info1.get(2).get(0) == "Japan")
-    assert(list_info1.get(3).get(0) == "Canada")
-    assert(list_info1.get(4).get(0) == "Russia")
-    assert(list_info1.get(5).get(0) == "Korea")
-    validateDataFiles("default_list_table_country", "0", Seq(0, 1, 2, 3))
-    val result_origin6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
-    val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
-    checkAnswer(result_origin6, result_after6)
-  }
-
-  test("Alter table split partition with different List Sequence: List Partition") {
-    sql("""ALTER TABLE list_table_country ADD PARTITION ('(Part1, Part2, Part3, Part4)')""".stripMargin)
-    sql("""ALTER TABLE list_table_country SPLIT PARTITION(9) INTO ('Part4', 'Part2', '(Part1, Part3)')""".stripMargin)
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "list_table_country")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val list_info = partitionInfo.getListInfo
-    assert(partitionIds == List(0, 1, 2, 3, 6, 7, 5, 10, 11, 12).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 12)
-    assert(partitionInfo.getNumPartitions == 10)
-    assert(list_info.get(0).get(0) == "China")
-    assert(list_info.get(0).get(1) == "US")
-    assert(list_info.get(1).get(0) == "UK")
-    assert(list_info.get(2).get(0) == "Japan")
-    assert(list_info.get(3).get(0) == "Canada")
-    assert(list_info.get(4).get(0) == "Russia")
-    assert(list_info.get(5).get(0) == "Korea")
-    assert(list_info.get(6).get(0) == "Part4")
-    assert(list_info.get(7).get(0) == "Part2")
-    assert(list_info.get(8).get(0) == "Part1")
-    assert(list_info.get(8).get(1) == "Part3")
-    validateDataFiles("default_list_table_country", "0", Seq(0, 1, 2, 3))
-    val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country < 'NotGood' """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country < 'NotGood' """)
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country <= 'NotGood' """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country <= 'NotGood' """)
-    checkAnswer(result_after2, result_origin2)
-
-    val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country = 'NotGood' """)
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country = 'NotGood' """)
-    checkAnswer(result_after3, result_origin3)
-
-    val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country >= 'NotGood' """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country >= 'NotGood' """)
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country > 'NotGood' """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country > 'NotGood' """)
-    checkAnswer(result_after5, result_origin5)
-  }
-
-  test("Alter table split partition with extra space in New SubList: List Partition") {
-    sql("""ALTER TABLE list_table_area ADD PARTITION ('(One,Two, Three, Four)')""".stripMargin)
-    sql("""ALTER TABLE list_table_area SPLIT PARTITION(6) INTO ('One', '(Two, Three )', 'Four')""".stripMargin)
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "list_table_area")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val list_info = partitionInfo.getListInfo
-    assert(partitionIds == List(0, 1, 3, 4, 5, 7, 8, 9).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 9)
-    assert(partitionInfo.getNumPartitions == 8)
-    assert(list_info.get(0).get(0) == "Asia")
-    assert(list_info.get(1).get(0) == "Europe")
-    assert(list_info.get(2).get(0) == "OutSpace")
-    assert(list_info.get(3).get(0) == "Hi")
-    assert(list_info.get(4).get(0) == "One")
-    assert(list_info.get(5).get(0) == "Two")
-    assert(list_info.get(5).get(1) == "Three")
-    assert(list_info.get(6).get(0) == "Four")
-    validateDataFiles("default_list_table_area", "0", Seq(1, 4))
-    val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <> 'America' """)
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area < 'Four' """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area < 'Four' and area <> 'America' """)
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area <= 'Four' """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <= 'Four' and area <> 'America'  """)
-    checkAnswer(result_after2, result_origin2)
-
-    val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area = 'Four' """)
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area = 'Four' and area <> 'America'  """)
-    checkAnswer(result_after3, result_origin3)
-
-    val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area >= 'Four' """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area >= 'Four' and area <> 'America'  """)
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area > 'Four' """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area > 'Four' and area <> 'America'  """)
-    checkAnswer(result_after5, result_origin5)
-  }
-
-  test("Alter table split partition: Range Partition") {
-    sql("""ALTER TABLE range_table_logdate_split SPLIT PARTITION(4) INTO ('2017/01/01', '2018/01/01')""")
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "range_table_logdate_split")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val rangeInfo = partitionInfo.getRangeInfo
-    assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 6)
-    assert(partitionInfo.getNumPartitions == 6)
-    assert(rangeInfo.get(0) == "2014/01/01")
-    assert(rangeInfo.get(1) == "2015/01/01")
-    assert(rangeInfo.get(2) == "2016/01/01")
-    assert(rangeInfo.get(3) == "2017/01/01")
-    assert(rangeInfo.get(4) == "2018/01/01")
-    validateDataFiles("default_range_table_logdate_split", "0", Seq(1, 2, 3, 5, 6))
-    val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin""")
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after2, result_origin2)
-
-    val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after3, result_origin3)
-
-    val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after5, result_origin5)
-
-    sql("""ALTER TABLE range_table_logdate_split DROP PARTITION(6)""")
-    val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable(
-      "default",
-      "range_table_logdate_split"
-    )
-    val partitionInfo1 = carbonTable1.getPartitionInfo()
-    val partitionIds1 = partitionInfo1.getPartitionIds
-    val rangeInfo1 = partitionInfo1.getRangeInfo
-    assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMaxPartitionId == 6)
-    assert(partitionInfo1.getNumPartitions == 5)
-    assert(rangeInfo1.get(0) == "2014/01/01")
-    assert(rangeInfo1.get(1) == "2015/01/01")
-    assert(rangeInfo1.get(2) == "2016/01/01")
-    assert(rangeInfo1.get(3) == "2017/01/01")
-    assert(rangeInfo1.size() == 4)
-    validateDataFiles("default_range_table_logdate_split", "0", Seq(0, 1, 2, 3, 5))
-    val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split""")
-    val result_origin6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin""")
-    checkAnswer(result_after6, result_origin6)
-  }
-
-  ignore("Alter table split partition: Range Partition + Bucket") {
-    sql("""ALTER TABLE range_table_bucket SPLIT PARTITION(4) INTO ('2017/01/01', '2018/01/01')""")
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "range_table_bucket")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val rangeInfo = partitionInfo.getRangeInfo
-    assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 6)
-    assert(partitionInfo.getNumPartitions == 6)
-    assert(rangeInfo.get(0) == "2014/01/01")
-    assert(rangeInfo.get(1) == "2015/01/01")
-    assert(rangeInfo.get(2) == "2016/01/01")
-    assert(rangeInfo.get(3) == "2017/01/01")
-    assert(rangeInfo.get(4) == "2018/01/01")
-    validateDataFiles("default_range_table_bucket", "0", Seq(1, 2, 3, 5, 6))
-    val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin""")
-    checkAnswer(result_after, result_origin)
-
-    val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after1, result_origin1)
-
-    val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after2, result_origin2)
-
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_origin3, result_after3)
-
-    val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after4, result_origin4)
-
-    val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
-    checkAnswer(result_after5, result_origin5)
-
-    sql("""ALTER TABLE range_table_bucket DROP PARTITION(6) WITH DATA""")
-    val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
-    val partitionInfo1 = carbonTable1.getPartitionInfo()
-    val partitionIds1 = partitionInfo1.getPartitionIds
-    val rangeInfo1 = partitionInfo1.getRangeInfo
-    assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMaxPartitionId == 6)
-    assert(partitionInfo1.getNumPartitions == 5)
-    assert(rangeInfo1.get(0) == "2014/01/01")
-    assert(rangeInfo1.get(1) == "2015/01/01")
-    assert(rangeInfo1.get(2) == "2016/01/01")
-    assert(rangeInfo1.get(3) == "2017/01/01")
-    assert(rangeInfo1.size() == 4)
-    validateDataFiles("default_range_table_bucket", "0", Seq(1, 2, 3, 5))
-    val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
-    val result_origin6= sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < '2017/01/01 00:00:00' or logdate >= '2018/01/01 00:00:00'""")
-    checkAnswer(result_after6, result_origin6)
-
-    sql("""ALTER TABLE range_table_bucket DROP PARTITION(3)""")
-    val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
-    val partitionInfo2 = carbonTable2.getPartitionInfo()
-    val partitionIds2 = partitionInfo2.getPartitionIds
-    val rangeInfo2 = partitionInfo2.getRangeInfo
-    assert(partitionIds2 == List(0, 1, 2, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo2.getMaxPartitionId == 6)
-    assert(partitionInfo2.getNumPartitions == 4)
-    assert(rangeInfo2.get(0) == "2014/01/01")
-    assert(rangeInfo2.get(1) == "2015/01/01")
-    assert(rangeInfo2.get(2) == "2017/01/01")
-    assert(rangeInfo2.size() == 3)
-    validateDataFiles("default_range_table_bucket", "0", Seq(1, 2, 5))
-    val result_origin7 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
-    val result_after7 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < '2017/01/01 00:00:00' or logdate >= '2018/01/01 00:00:00'""")
-    checkAnswer(result_origin7, result_after7)
-
-    sql("""ALTER TABLE range_table_bucket DROP PARTITION(5)""")
-    val carbonTable3 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
-    val partitionInfo3 = carbonTable3.getPartitionInfo()
-    val partitionIds3 = partitionInfo3.getPartitionIds
-    val rangeInfo3 = partitionInfo3.getRangeInfo
-    assert(partitionIds3 == List(0, 1, 2).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo3.getMaxPartitionId == 6)
-    assert(partitionInfo3.getNumPartitions == 3)
-    assert(rangeInfo3.get(0) == "2014/01/01")
-    assert(rangeInfo3.get(1) == "2015/01/01")
-    assert(rangeInfo3.size() == 2)
-    validateDataFiles("default_range_table_bucket", "0", Seq(0, 1, 2))
-    val result_after8 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
-    val result_origin8 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < '2017/01/01 00:00:00' or logdate >= '2018/01/01 00:00:00'""")
-    checkAnswer(result_after8, result_origin8)
-  }
-
-   test("test exception when alter partition and the values"
-       + "in range_info can not match partition column type") {
-     val exception_test_range_int: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_int(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='11,12')
-        """.stripMargin)
-       sql("ALTER TABLE test_range_int ADD PARTITION ('abc')")
-    }
-     assert(exception_test_range_int.getMessage
-         .contains("Data in range info must be the same type with the partition field's type"))
-
-    sql("DROP TABLE IF EXISTS test_range_smallint")
-    val exception_test_range_smallint: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_smallint(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 SMALLINT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='11,12')
-        """.stripMargin)
-      sql("ALTER TABLE test_range_smallint ADD PARTITION ('abc')")
-    }
-     assert(exception_test_range_smallint.getMessage
-         .contains("Data in range info must be the same type with the partition field's type"))
-
-    sql("DROP TABLE IF EXISTS test_range_float")
-    val exception_test_range_float: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_float(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1.1,2.1')
-        """.stripMargin)
-      sql("ALTER TABLE test_range_float ADD PARTITION ('abc')")
-    }
-     assert(exception_test_range_float.getMessage
-         .contains("Data in range info must be the same type with the partition field's type"))
-
-    sql("DROP TABLE IF EXISTS test_range_double")
-    val exception_test_range_double: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_double(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1000.005,2000.005')
-        """.stripMargin)
-      sql("ALTER TABLE test_range_double ADD PARTITION ('abc')")
-    }
-     assert(exception_test_range_double.getMessage
-         .contains("Data in range info must be the same type with the partition field's type"))
-
-    sql("DROP TABLE IF EXISTS test_range_bigint")
-    val exception_test_range_bigint: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_bigint(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 BIGINT) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='123456789,223456789')
-        """.stripMargin)
-       sql("ALTER TABLE test_range_bigint ADD PARTITION ('abc')")
-    }
-     assert(exception_test_range_bigint.getMessage
-         .contains("Data in range info must be the same type with the partition field's type"))
-
-    sql("DROP TABLE IF EXISTS test_range_date")
-    val exception_test_range_date: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_date(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DATE) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2017-06-11, 2017-06-13')
-        """.stripMargin)
-      sql("ALTER TABLE test_range_date ADD PARTITION ('abc')")
-    }
-    assert(exception_test_range_date.getMessage
-      .contains("Data in range info must be the same type with the partition field's type"))
-
-    sql("DROP TABLE IF EXISTS test_range_timestamp")
-    val exception_test_range_timestamp: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_timestamp(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 TIMESTAMP) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2017/06/11, 2017/06/13')
-        """.stripMargin)
-      sql("ALTER TABLE test_range_timestamp ADD PARTITION ('abc')")
-    }
-    assert(exception_test_range_timestamp.getMessage
-      .contains("Data in range info must be the same type with the partition field's type"))
-    sql("DROP TABLE IF EXISTS test_range_decimal")
-    val exception_test_range_decimal: Exception = intercept[Exception] {
-      sql(
-        """
-          | CREATE TABLE test_range_decimal(col1 INT, col2 STRING)
-          | PARTITIONED BY (col3 DECIMAL(25, 4)) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='22.22,33.33')
-        """.stripMargin)
-      sql("ALTER TABLE test_range_decimal ADD PARTITION ('abc')")
-    }
-    assert(exception_test_range_decimal.getMessage
-         .contains("Data in range info must be the same type with the partition field's type"))
-  }
-
-  test("Add partition to table in or not in default database") {
-    sql("DROP TABLE IF EXISTS carbon_table_default_db")
-    sql(
-      """
-        | CREATE TABLE carbon_table_default_db(id INT, name STRING) PARTITIONED BY (dt STRING)
-        | STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
-      """.stripMargin)
-    sql("ALTER TABLE carbon_table_default_db ADD PARTITION ('2017')")
-    val carbonTable = CarbonEnv
-      .getCarbonTable(Option("default"), "carbon_table_default_db")(sqlContext.sparkSession)
-    val partitionInfo = carbonTable.getPartitionInfo()
-    val partitionIds = partitionInfo.getPartitionIds
-    val range_info = partitionInfo.getRangeInfo
-    assert(partitionIds == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMaxPartitionId == 3)
-    assert(partitionInfo.getNumPartitions == 4)
-    assert(range_info.get(0) == "2015")
-    assert(range_info.get(1) == "2016")
-    assert(range_info.get(2) == "2017")
-
-    sql("CREATE DATABASE IF NOT EXISTS carbondb")
-    sql("DROP TABLE IF EXISTS carbondb.carbontable")
-    sql(
-      """
-        | CREATE TABLE carbondb.carbontable(id INT, name STRING) PARTITIONED BY (dt STRING)
-        | STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
-      """.stripMargin)
-    sql("ALTER TABLE carbondb.carbontable ADD PARTITION ('2017')")
-
-    val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("carbondb", "carbontable")
-    val partitionInfo1 = carbonTable1.getPartitionInfo()
-    val partitionIds1 = partitionInfo1.getPartitionIds
-    val range_info1 = partitionInfo1.getRangeInfo
-    assert(partitionIds1 == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMaxPartitionId == 3)
-    assert(partitionInfo1.getNumPartitions == 4)
-    assert(range_info1.get(0) == "2015")
-    assert(range_info1.get(1) == "2016")
-    assert(range_info1.get(2) == "2017")
-  }
-
-  test("test exception when alter partition's table doesn't exist in a perticular database") {
-    val exception_test_add_partition: Exception = intercept[Exception] {
-      sql("CREATE DATABASE IF NOT EXISTS carbondb")
-      sql("USE default")
-      sql("drop table if exists carbon_table_in_default_db")
-      sql(
-        """
-          | CREATE TABLE carbon_table_in_default_db(id INT, name STRING)
-          | PARTITIONED BY (dt STRING) STORED BY 'carbondata'
-          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
-        """.stripMargin)
-      sql("ALTER TABLE carbondb.carbon_table_in_default_db ADD PARTITION ('2017')")
-    }
-    assert(exception_test_add_partition.getMessage
-      .contains("Table or view 'carbon_table_in_default_db' not found in database 'carbondb'"))
-  }
-
-  def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val dataFiles = getDataFiles(carbonTable, segmentId)
-    validatePartitionTableFiles(partitions, dataFiles)
-  }
-
-  def getDataFiles(carbonTable: CarbonTable, segmentId: String): Array[String] = {
-    val segment = Segment.getSegment(segmentId, carbonTable.getTablePath)
-    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
-    val carbonFile = FileFactory.getCarbonFile(segmentDir)
-    val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
-      override def accept(file: CarbonFile): Boolean = {
-        return file.getName.endsWith(".carbondata")
-      }
-    })
-    dataFiles.map(_.getName)
-  }
-
-  /**
-   * should ensure answer equals to expected list, not only contains
-   * @param partitions
-   * @param dataFiles
-   */
-  def validatePartitionTableFiles(partitions: Seq[Int], dataFiles: Array[String]): Unit = {
-    val partitionIds: ListBuffer[Int] = new ListBuffer[Int]()
-    dataFiles.foreach { dataFile =>
-      val partitionId = CarbonTablePath.DataFileUtil.getTaskNo(dataFile).split("_")(0).toInt
-      partitionIds += partitionId
-      assert(partitions.contains(partitionId))
-    }
-    partitions.foreach(id => assert(partitionIds.contains(id)))
-  }
-
-  override def afterAll = {
-    sql("DROP TABLE IF EXISTS carbondb.carbontable")
-    sql("DROP DATABASE IF EXISTS carbondb")
-    dropTable
-    CarbonProperties.getInstance()
-    .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
-        CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)
-  }
-
-  def dropTable {
-    sql("DROP TABLE IF EXISTS list_table_area_origin")
-    sql("DROP TABLE IF EXISTS range_table_logdate_origin")
-    sql("DROP TABLE IF EXISTS list_table_country_origin")
-    sql("DROP TABLE IF EXISTS range_table_logdate_split_origin")
-    sql("DROP TABLE IF EXISTS range_table_bucket_origin")
-    sql("DROP TABLE IF EXISTS list_table_area")
-    sql("DROP TABLE IF EXISTS range_table_logdate")
-    sql("DROP TABLE IF EXISTS list_table_country")
-    sql("DROP TABLE IF EXISTS range_table_logdate_split")
-    sql("DROP TABLE IF EXISTS range_table_bucket")
-    sql("DROP TABLE IF EXISTS test_range_int")
-    sql("DROP TABLE IF EXISTS test_range_smallint")
-    sql("DROP TABLE IF EXISTS test_range_bigint")
-    sql("DROP TABLE IF EXISTS test_range_float")
-    sql("DROP TABLE IF EXISTS test_range_double")
-    sql("DROP TABLE IF EXISTS test_range_date")
-    sql("DROP TABLE IF EXISTS test_range_timestamp")
-    sql("DROP TABLE IF EXISTS test_range_decimal")
-    sql("drop table if exists test_invalid_partition_id")
-  }
-
-
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index 8d5ee0e..1cdf93b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -32,7 +32,6 @@ import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
-import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -176,7 +175,6 @@ public class CarbonLoadModelBuilder {
     String column_dict = optionsFinal.get("columndict");
     validateDateTimeFormat(timestampformat, "TimestampFormat");
     validateDateTimeFormat(dateFormat, "DateFormat");
-    validateSortScope(sort_scope);
 
     if (Boolean.parseBoolean(bad_records_logger_enable) ||
         LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
@@ -217,12 +215,7 @@ public class CarbonLoadModelBuilder {
           List<String> columnNames = new ArrayList<>();
           List<String> partitionColumns = new ArrayList<>();
           for (int i = 0; i < columns.size(); i++) {
-            if (table.getPartitionInfo() != null && table.getPartitionInfo().getColumnSchemaList()
-                .contains(columns.get(i).getColumnSchema())) {
-              partitionColumns.add(columns.get(i).getColName());
-            } else {
-              columnNames.add(columns.get(i).getColName());
-            }
+            columnNames.add(columns.get(i).getColName());
           }
           columnNames.addAll(partitionColumns);
           fileHeader = Strings.mkString(columnNames.toArray(new String[columnNames.size()]), ",");
@@ -391,19 +384,6 @@ public class CarbonLoadModelBuilder {
     }
   }
 
-  private void validateSortScope(String sortScope) throws InvalidLoadOptionException {
-    if (sortScope != null) {
-      // We support global sort for Hive standard partition, but don't support
-      // global sort for other partition type.
-      if (table.getPartitionInfo() != null &&
-          !table.isHivePartitionTable() &&
-          sortScope.equalsIgnoreCase(SortScopeOptions.SortScope.GLOBAL_SORT.toString())) {
-        throw new InvalidLoadOptionException("Don't support use global sort on "
-            + table.getPartitionInfo().getPartitionType() +  " partition table.");
-      }
-    }
-  }
-
   private void validateGlobalSortPartitions(String globalSortPartitions)
       throws InvalidLoadOptionException {
     if (globalSortPartitions != null) {
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
index cafac7c..b9f1896 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
@@ -33,7 +33,6 @@ import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -66,8 +65,6 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
 
   private boolean[] isNoDictionaryDimensionColumn;
 
-  private DataType[] measureDataType;
-
   private int dimensionCount;
 
   private int measureCount;
@@ -76,8 +73,6 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
 
   private long[] writeCounter;
 
-  private int outputLength;
-
   private CarbonTableIdentifier tableIdentifier;
 
   private String tableName;
@@ -125,9 +120,7 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
       dimensionCount = configuration.getDimensionCount() - noDictWithComplextCount;
       isNoDictionaryDimensionColumn =
           CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
-      measureDataType = configuration.getMeasureDataType();
       measureCount = configuration.getMeasureCount();
-      outputLength = measureCount + (this.noDictWithComplextCount > 0 ? 1 : 0) + 1;
       CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
           .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PARTITION_ID,
               System.currentTimeMillis());
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java b/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
deleted file mode 100644
index 649c18d..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.partition;
-
-import java.util.List;
-
-public interface DataPartitioner {
-
-  /**
-   * All the partitions built by the Partitioner
-   */
-  List<Partition> getAllPartitions();
-
-  /**
-   * Identifies the partitions applicable for the given filter (API used for For query)
-   */
-  List<Partition> getPartitions();
-
-}
-
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java b/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
deleted file mode 100644
index 95cdfb3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.partition;
-
-import java.io.Serializable;
-import java.util.List;
-
-public interface Partition extends Serializable {
-  /**
-   * unique identification for the partition in the cluster.
-   */
-  String getUniqueID();
-
-  /**
-   * result
-   *
-   * @return
-   */
-  List<String> getFilesPath();
-}
\ No newline at end of file
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
deleted file mode 100644
index c03f34e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.partition.spliter;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.scan.executor.QueryExecutor;
-import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
-import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.RowBatch;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Logger;
-
-public abstract class AbstractCarbonQueryExecutor {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName());
-  protected CarbonTable carbonTable;
-  protected QueryModel queryModel;
-  private QueryExecutor queryExecutor;
-  Map<String, TaskBlockInfo> segmentMapping;
-
-  /**
-   * get executor and execute the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  CarbonIterator<RowBatch> executeBlockList(List<TableBlockInfo> blockList,
-      Configuration configuration)
-      throws QueryExecutionException, IOException {
-    queryModel.setTableBlockInfos(blockList);
-    this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel, configuration);
-    return queryExecutor.execute(queryModel);
-  }
-
-  /**
-   * Below method will be used
-   * for cleanup
-   */
-  public void finish() {
-    try {
-      queryExecutor.finish();
-    } catch (QueryExecutionException e) {
-      LOGGER.error("Problem while finish: ", e);
-    }
... 334 lines suppressed ...