You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by gv...@apache.org on 2017/06/15 08:22:51 UTC

[1/4] carbondata git commit: partitioned by all primitive data type

Repository: carbondata
Updated Branches:
  refs/heads/master 447049a51 -> 4319be8b7


http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
index 566347f..fef542a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
@@ -21,6 +21,9 @@ import org.apache.carbondata.core.metadata.datatype.DataType
 import org.apache.carbondata.format.{DataType => ThriftDataType}
 
 object DataTypeConverterUtil {
+  val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r
+  val FIXED_DECIMALTYPE = """decimaltype\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r
+
   def convertToCarbonType(dataType: String): DataType = {
     dataType.toLowerCase match {
       case "string" => DataType.STRING
@@ -34,6 +37,7 @@ object DataTypeConverterUtil {
       case "double" => DataType.DOUBLE
       case "float" => DataType.DOUBLE
       case "decimal" => DataType.DECIMAL
+      case FIXED_DECIMAL(_, _) => DataType.DECIMAL
       case "timestamp" => DataType.TIMESTAMP
       case "date" => DataType.DATE
       case "array" => DataType.ARRAY
@@ -55,6 +59,7 @@ object DataTypeConverterUtil {
       case "doubletype" => DataType.DOUBLE
       case "floattype" => DataType.DOUBLE
       case "decimaltype" => DataType.DECIMAL
+      case FIXED_DECIMALTYPE(_, _) => DataType.DECIMAL
       case "timestamptype" => DataType.TIMESTAMP
       case "datetype" => DataType.DATE
       case others =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 580dcc6..ee999a4 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -1006,7 +1006,7 @@ object CarbonDataRDDFactory {
     val columns = carbonLoadModel.getCsvHeaderColumns
     var partitionColumnIndex = -1
     for (i <- 0 until columns.length) {
-      if (partitionColumn.equals(columns(i))) {
+      if (partitionColumn.equalsIgnoreCase(columns(i))) {
         partitionColumnIndex = i
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 2e8e024..93d790b 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
@@ -1030,7 +1030,7 @@ object CarbonDataRDDFactory {
     val columns = carbonLoadModel.getCsvHeaderColumns
     var partitionColumnIndex = -1
     for (i <- 0 until columns.length) {
-      if (partitionColumn.equals(columns(i))) {
+      if (partitionColumn.equalsIgnoreCase(columns(i))) {
         partitionColumnIndex = i
       }
     }


[4/4] carbondata git commit: [CARBONDATA-1166] Fix decimal partition column and add test case for all primitive column. This closes #1026

Posted by gv...@apache.org.
[CARBONDATA-1166] Fix decimal partition column and add test case for all primitive column. This closes #1026


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

Branch: refs/heads/master
Commit: 4319be8b7925df0197a58b855936b9d94649f1b7
Parents: 447049a de5346f
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Jun 15 13:52:15 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Thu Jun 15 13:52:15 2017 +0530

----------------------------------------------------------------------
 .../filter/partition/EqualToFilterImpl.java     |    5 +
 .../scan/filter/partition/InFilterImpl.java     |    5 +
 .../scan/filter/partition/RangeFilterImpl.java  |    4 +
 .../test/resources/alldatatypeforpartition.csv  |    4 +
 .../TestAllDataTypeForPartitionTable.scala      | 1240 ++++++++++++++++++
 .../spark/util/DataTypeConverterUtil.scala      |    5 +
 .../spark/rdd/CarbonDataRDDFactory.scala        |    2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |    2 +-
 8 files changed, 1265 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[3/4] carbondata git commit: partitioned by all primitive data type

Posted by gv...@apache.org.
partitioned by all primitive data type


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

Branch: refs/heads/master
Commit: de5346f3cd34f69ce25234ee3fb1cc146c3dc062
Parents: 447049a
Author: QiangCai <da...@gmail.com>
Authored: Fri Jun 9 19:01:34 2017 +0800
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Thu Jun 15 13:47:25 2017 +0530

----------------------------------------------------------------------
 .../filter/partition/EqualToFilterImpl.java     |    5 +
 .../scan/filter/partition/InFilterImpl.java     |    5 +
 .../scan/filter/partition/RangeFilterImpl.java  |    4 +
 .../test/resources/alldatatypeforpartition.csv  |    4 +
 .../TestAllDataTypeForPartitionTable.scala      | 1240 ++++++++++++++++++
 .../spark/util/DataTypeConverterUtil.scala      |    5 +
 .../spark/rdd/CarbonDataRDDFactory.scala        |    2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |    2 +-
 8 files changed, 1265 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java
----------------------------------------------------------------------
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
index 8542a75..4df892d 100644
--- 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
@@ -20,10 +20,12 @@ 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
@@ -47,6 +49,9 @@ public class EqualToFilterImpl implements PartitionFilterIntf {
       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;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java
----------------------------------------------------------------------
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
index 18bec64..0f45b32 100644
--- 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
@@ -20,12 +20,14 @@ 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
@@ -48,6 +50,9 @@ public class InFilterImpl implements PartitionFilterIntf {
       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;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java
----------------------------------------------------------------------
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
index b8a7640..697be32 100644
--- 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
@@ -25,6 +25,7 @@ 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;
 
 /**
  * the implement of Range filter(include <=, <, >=, >)
@@ -57,6 +58,9 @@ public class RangeFilterImpl implements PartitionFilterIntf {
         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);
       default:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/integration/spark-common-test/src/test/resources/alldatatypeforpartition.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/alldatatypeforpartition.csv b/integration/spark-common-test/src/test/resources/alldatatypeforpartition.csv
new file mode 100644
index 0000000..a8e9cb2
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/alldatatypeforpartition.csv
@@ -0,0 +1,4 @@
+smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField
+-32768,-2147483648,-9223372036854775808,-2147483648.1,-9223372036854775808.1,-9223372036854775808.1234,2017-06-11 00:00:01,2017-06-11,abc1,abcd1,abcde1,a$b$c$1,a$b$1
+128,32768,2147483648,2147483647.1,9223372036854775807.1,9223372036854775807.1234,2017-06-12 23:59:59,2017-06-12,abc2,abcd2,abcde2,a$b$c$2,a$b$2
+32767,2147483647,9223372036854775807,2147483648.1,9223372036854775808.1,9223372036854775808.1234,2017-06-13 23:59:59,2017-06-13,abc3,abcd3,abcde3,a$b$c$3,a$b$3
\ No newline at end of file


[2/4] carbondata git commit: partitioned by all primitive data type

Posted by gv...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/de5346f3/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1b69eda
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala
@@ -0,0 +1,1240 @@
+/*
+ * 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.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import scala.collection.mutable
+
+class TestAllDataTypeForPartitionTable extends QueryTest with BeforeAndAfterAll {
+
+  val defaultTimestampFormat = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT)
+
+  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_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+      .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"))))
+  }
+
+  test("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"))))
+  }
+
+  test("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,s

<TRUNCATED>