You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/12/17 14:16:53 UTC

[11/21] carbondata git commit: [CARBONDATA-2755][Complex DataType Enhancements] Compaction Complex Types (STRUCT AND ARRAY)

[CARBONDATA-2755][Complex DataType Enhancements] Compaction Complex Types (STRUCT AND ARRAY)

Cherry-Picked from PR #2516

This closes #2976


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

Branch: refs/heads/branch-1.5
Commit: e8d25bb3b80a3deda13bc12773e8565c33747131
Parents: 21762c5
Author: sounakr <so...@gmail.com>
Authored: Tue Jul 17 10:35:32 2018 +0530
Committer: Raghunandan S <ca...@gmail.com>
Committed: Mon Dec 17 18:58:33 2018 +0530

----------------------------------------------------------------------
 .../core/datastore/row/WriteStepRowUtil.java    |  23 +-
 .../core/scan/wrappers/ByteArrayWrapper.java    |  10 +
 .../complexType/TestCompactionComplexType.scala | 992 +++++++++++++++++++
 .../complexType/TestComplexDataType.scala       |  32 -
 .../CarbonAlterTableCompactionCommand.scala     |   7 -
 .../processing/datatypes/PrimitiveDataType.java |  10 +-
 .../converter/impl/FieldEncoderFactory.java     |   2 +-
 .../processing/merger/CarbonCompactionUtil.java |  33 +
 .../store/CarbonFactDataHandlerModel.java       |  89 +-
 .../util/CarbonDataProcessorUtil.java           |   7 +-
 10 files changed, 1142 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java b/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
index fe4e10e..49716ac 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
@@ -59,29 +59,40 @@ public class WriteStepRowUtil {
     }
     converted[DICTIONARY_DIMENSION] = dictDimensions;
 
+    Object[] noDictAndComplexKeys =
+        new Object[segmentProperties.getNumberOfNoDictionaryDimension() + segmentProperties
+            .getComplexDimensions().size()];
+
     byte[][] noDictionaryKeys = ((ByteArrayWrapper) row[0]).getNoDictionaryKeys();
-    Object[] noDictKeys = new Object[noDictionaryKeys.length];
     for (int i = 0; i < noDictionaryKeys.length; i++) {
       // in case of compaction rows are collected from result collector and are in byte[].
       // Convert the no dictionary columns to original data,
       // as load expects the no dictionary column with original data.
       if (DataTypeUtil.isPrimitiveColumn(noDicAndComplexColumns[i].getDataType())) {
-        noDictKeys[i] = DataTypeUtil
+        noDictAndComplexKeys[i] = DataTypeUtil
             .getDataBasedOnDataTypeForNoDictionaryColumn(noDictionaryKeys[i],
                 noDicAndComplexColumns[i].getDataType());
         // for timestamp the above method will give the original data, so it should be
         // converted again to the format to be loaded (without micros)
-        if (null != noDictKeys[i]
+        if (null != noDictAndComplexKeys[i]
             && noDicAndComplexColumns[i].getDataType() == DataTypes.TIMESTAMP) {
-          noDictKeys[i] = (long) noDictKeys[i] / 1000L;
+          noDictAndComplexKeys[i] = (long) noDictAndComplexKeys[i] / 1000L;
         }
       } else {
-        noDictKeys[i] = noDictionaryKeys[i];
+        noDictAndComplexKeys[i] = noDictionaryKeys[i];
       }
     }
 
+    // For Complex Type Columns
+    byte[][] complexKeys = ((ByteArrayWrapper) row[0]).getComplexTypesKeys();
+    for (int i = segmentProperties.getNumberOfNoDictionaryDimension();
+         i < segmentProperties.getNumberOfNoDictionaryDimension() + segmentProperties
+             .getComplexDimensions().size(); i++) {
+      noDictAndComplexKeys[i] = complexKeys[i];
+    }
+
     // no dictionary and complex dimension
-    converted[NO_DICTIONARY_AND_COMPLEX] = noDictKeys;
+    converted[NO_DICTIONARY_AND_COMPLEX] = noDictAndComplexKeys;
 
     // measure
     int measureCount = row.length - 1;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
index 4cce8af..65f29d4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
@@ -90,6 +90,16 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
     return this.noDictionaryKeys;
   }
 
+
+  /**
+   * to get the complex column data
+   *
+   * @return no complex values
+   */
+  public byte[][] getComplexTypesKeys() {
+    return this.complexTypesKeys;
+  }
+
   /**
    * to generate the hash code
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala
new file mode 100644
index 0000000..6ff8a15
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala
@@ -0,0 +1,992 @@
+/*
+ * 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.integration.spark.testsuite.complexType
+
+import java.sql.Timestamp
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+class TestCompactionComplexType extends QueryTest {
+
+  test("test INT with struct and array, Encoding INT-->BYTE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(20, 30, 40)))),
+        Row(2, Row(600, "abc", mutable.WrappedArray.make(Array(20, 30, 40)))),
+        Row(3, Row(600, "abc", mutable.WrappedArray.make(Array(20, 30, 40))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500$abc$20:30:40')")
+    sql("insert into adaptive values(2,'600$abc$30:30:40')")
+    sql("insert into adaptive values(3,'700$abc$40:30:40')")
+    sql("insert into adaptive values(4,'800$abc$50:30:40')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(20, 30, 40)))),
+        Row(2, Row(600, "abc", mutable.WrappedArray.make(Array(30, 30, 40)))),
+        Row(3, Row(700, "abc", mutable.WrappedArray.make(Array(40, 30, 40)))),
+        Row(4, Row(800, "abc", mutable.WrappedArray.make(Array(50, 30, 40))))))
+  }
+
+  test("test INT with struct and array, Encoding INT-->SHORT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500$abc$200:300:400')")
+    sql("insert into adaptive values(2,'600$abc$300:300:400')")
+    sql("insert into adaptive values(3,'700$abc$400:300:400')")
+    sql("insert into adaptive values(4,'800$abc$500:300:400')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(600, "abc", mutable.WrappedArray.make(Array(300, 300, 400)))),
+        Row(3, Row(700, "abc", mutable.WrappedArray.make(Array(400, 300, 400)))),
+        Row(4, Row(800, "abc", mutable.WrappedArray.make(Array(500, 300, 400))))))
+  }
+
+  test("test INT with struct and array, Encoding INT-->SHORT INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int2.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'50000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(2,'70000$abc$2000000:4000000:4000000')")
+    sql("insert into adaptive values(3,'100000$abc$2000000:5000000:4000000')")
+    sql("insert into adaptive values(4,'200000$abc$2000000:6000000:4000000')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 4000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 5000000, 4000000)))),
+        Row(4, Row(200000, "abc", mutable.WrappedArray.make(Array(2000000, 6000000, 4000000))))))
+  }
+
+  test("test INT with struct and array, Encoding INT-->INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int3.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(7000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:int,name:string,marks:array<int>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000')")
+    sql("insert into adaptive values(2,'700000$abc$210:350:52000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000')")
+    sql("insert into adaptive values(4,'10000001$abd$250:450:62000000')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(700000, "abc", mutable.WrappedArray.make(Array(210, 350, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(4, Row(10000001, "abd", mutable.WrappedArray.make(Array(250, 450, 62000000))))))
+  }
+
+
+  test("test SMALLINT with struct and array SMALLINT --> BYTE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:smallint,name:string," +
+      "marks:array<smallint>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'100$abc$20:30:40')")
+    sql("insert into adaptive values(2,'200$abc$30:40:50')")
+    sql("insert into adaptive values(3,'300$abd$30:41:55')")
+    sql("insert into adaptive values(4,'400$abe$30:42:56')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(100, "abc", mutable.WrappedArray.make(Array(20, 30, 40)))),
+        Row(2, Row(200, "abc", mutable.WrappedArray.make(Array(30, 40, 50)))),
+        Row(3, Row(300, "abd", mutable.WrappedArray.make(Array(30, 41, 55)))),
+        Row(4, Row(400, "abe", mutable.WrappedArray.make(Array(30, 42, 56))))))
+  }
+
+  test("test SMALLINT with struct and array SMALLINT --> SHORT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:smallint,name:string," +
+      "marks:array<smallint>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500$abc$200:300:400')")
+    sql("insert into adaptive values(2,'8000$abc$300:410:500')")
+    sql("insert into adaptive values(3,'9000$abee$310:420:400')")
+    sql("insert into adaptive values(4,'9900$abfffffffffffffff$320:430:500')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(8000, "abc", mutable.WrappedArray.make(Array(300, 410, 500)))),
+        Row(3, Row(9000, "abee", mutable.WrappedArray.make(Array(310, 420, 400)))),
+        Row(4, Row(9900, "abfffffffffffffff", mutable.WrappedArray.make(Array(320, 430, 500))))))
+    sql("insert into adaptive values(5,'500$abc$200:310:400')")
+    sql("insert into adaptive values(6,'8000$abc$300:310:500')")
+    sql("insert into adaptive values(7,'9000$abee$310:320:400')")
+    sql("insert into adaptive values(8,'9900$abfffffffffffffffeeee$320:330:500')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    sql("SHOW SEGMENTS FOR TABLE adaptive").show(200,false)
+    sql("clean files for table adaptive").show(200,false)
+    sql("SHOW SEGMENTS FOR TABLE adaptive").show(200,false)
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(8000, "abc", mutable.WrappedArray.make(Array(300, 410, 500)))),
+        Row(3, Row(9000, "abee", mutable.WrappedArray.make(Array(310, 420, 400)))),
+        Row(4, Row(9900, "abfffffffffffffff", mutable.WrappedArray.make(Array(320, 430, 500)))),
+      Row(5, Row(500, "abc", mutable.WrappedArray.make(Array(200, 310, 400)))),
+      Row(6, Row(8000, "abc", mutable.WrappedArray.make(Array(300, 310, 500)))),
+      Row(7, Row(9000, "abee", mutable.WrappedArray.make(Array(310, 320, 400)))),
+      Row(8, Row(9900, "abfffffffffffffffeeee", mutable.WrappedArray.make(Array(320, 330, 500))))))
+  }
+
+  test("test BigInt with struct and array BIGINT --> BYTE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:bigint,name:string," +
+      "marks:array<bigint>>) stored by 'carbondata'")
+    sql("insert into adaptive values(11,'1$abc$21:30:40')")
+    sql("insert into adaptive values(12,'1$ab1$22:30:40')")
+    sql("insert into adaptive values(13,'1$ab2$23:30:40')")
+    sql("insert into adaptive values(14,'1$ab3$24:30:40')")
+    sql("insert into adaptive values(15,'1$ab4$25:30:40')")
+    sql("insert into adaptive values(16,'1$ab5$26:30:40')")
+    sql("insert into adaptive values(17,'1$ab6$27:30:40')")
+    sql("insert into adaptive values(18,'1$ab7$28:30:40')")
+    sql("insert into adaptive values(19,'1$ab8$29:30:40')")
+    sql("insert into adaptive values(20,'1$ab9$30:30:40')")
+    sql("insert into adaptive values(21,'1$ab10$31:30:40')")
+    sql("insert into adaptive values(22,'1$ab11$32:30:40')")
+    sql("alter table adaptive compact 'major'").show(200,false)
+    sql("SHOW SEGMENTS FOR TABLE adaptive").show(200,false)
+    sql("clean files for table adaptive").show(200,false)
+    sql("SHOW SEGMENTS FOR TABLE adaptive").show(200,false)
+
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(11, Row(1, "abc", mutable.WrappedArray.make(Array(21, 30, 40)))),
+        Row(12, Row(1, "ab1", mutable.WrappedArray.make(Array(22, 30, 40)))),
+        Row(13, Row(1, "ab2", mutable.WrappedArray.make(Array(23, 30, 40)))),
+        Row(14, Row(1, "ab3", mutable.WrappedArray.make(Array(24, 30, 40)))),
+        Row(15, Row(1, "ab4", mutable.WrappedArray.make(Array(25, 30, 40)))),
+        Row(16, Row(1, "ab5", mutable.WrappedArray.make(Array(26, 30, 40)))),
+        Row(17, Row(1, "ab6", mutable.WrappedArray.make(Array(27, 30, 40)))),
+        Row(18, Row(1, "ab7", mutable.WrappedArray.make(Array(28, 30, 40)))),
+        Row(19, Row(1, "ab8", mutable.WrappedArray.make(Array(29, 30, 40)))),
+        Row(20, Row(1, "ab9", mutable.WrappedArray.make(Array(30, 30, 40)))),
+        Row(21, Row(1, "ab10", mutable.WrappedArray.make(Array(31, 30, 40)))),
+        Row(22, Row(1, "ab11", mutable.WrappedArray.make(Array(32, 30, 40))))
+      ))
+  }
+
+  test("test BigInt with struct and array BIGINT --> SHORT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:bigint,name:string," +
+      "marks:array<bigint>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500$abc$200:300:400')")
+    sql("insert into adaptive values(2,'8000$abc$300:400:500')")
+    sql("insert into adaptive values(3,'9000$abc$300:400:500')")
+    sql("insert into adaptive values(4,'10000$abc$300:400:500')")
+    sql("alter table adaptive compact'major'")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(8000, "abc", mutable.WrappedArray.make(Array(300, 400, 500)))),
+        Row(3, Row(9000, "abc", mutable.WrappedArray.make(Array(300, 400, 500)))),
+        Row(4, Row(10000, "abc", mutable.WrappedArray.make(Array(300, 400, 500))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:BIGINT,name:string,marks:array<BIGINT>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact'major'")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400))))
+      ))
+  }
+
+  test("test BigInt with struct and array BIGINT --> SHORT INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:bigint,name:string," +
+      "marks:array<bigint>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'50000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(2,'70000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(3,'100000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(1,'50000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(2,'70000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(3,'100000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(1,'50000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(2,'70000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(3,'100000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(1,'50000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(2,'70000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(3,'100000$abc$2000000:3000000:4000000')")
+    sql("alter table adaptive compact'major'")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000))))
+      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:BIGINT,name:string,marks:array<BIGINT>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int2.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int2.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int2.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int2.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact'major'")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000))))
+      ))
+  }
+
+  test("test BIGINT with struct and array, Encoding INT-->INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:BIGINT,name:string,marks:array<BIGINT>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int3.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int3.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int3.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int3.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact'major'")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(7000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(7000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(7000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(7000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000))))
+      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:BIGINT,name:string,marks:array<BIGINT>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000')")
+    sql("insert into adaptive values(2,'700000$abc$200:300:52000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000')")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000')")
+    sql("insert into adaptive values(2,'700000$abc$200:300:52000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000')")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000')")
+    sql("insert into adaptive values(2,'700000$abc$200:300:52000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000')")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000')")
+    sql("insert into adaptive values(2,'700000$abc$200:300:52000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(700000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(700000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(700000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(700000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000))))
+      ))
+  }
+
+  test("test Double with Struct and Array DOUBLE --> BYTE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string," +
+      "marks:array<double>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'1.323$abc$2.2:3.3:4.4')")
+    sql("insert into adaptive values(2,'1.324$abc$2.2:3.3:4.4')")
+    sql("insert into adaptive values(3,'1.325$abc$2.2:3.3:4.4')")
+    sql("insert into adaptive values(4,'1.326$abc$2.2:3.3:4.4')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(2, Row(1.324, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(3, Row(1.325, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(4, Row(1.326, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string,marks:array<double>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double1.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double1.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double1.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double1.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(2, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(3, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(2, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(3, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(2, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(3, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(2, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4)))),
+        Row(3, Row(1.323, "abc", mutable.WrappedArray.make(Array(2.2, 3.3, 4.4))))
+      ))
+  }
+
+  test("test Double with Struct and Array DOUBLE --> SHORT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string," +
+      "marks:array<double>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'1.323$abc$20.2:30.3:40.4')")
+    sql("insert into adaptive values(2,'1.324$abc$20.2:30.3:40.5')")
+    sql("insert into adaptive values(3,'1.325$abc$20.2:30.3:40.6')")
+    sql("insert into adaptive values(4,'1.326$abc$20.2:30.3:40.7')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(2, Row(1.324, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.5)))),
+        Row(3, Row(1.325, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.6)))),
+        Row(4, Row(1.326, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.7))))
+      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string,marks:array<double>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double2.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double2.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double2.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double2.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(3, Row(4.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(3, Row(4.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(3, Row(4.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4)))),
+        Row(3, Row(4.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 40.4))))
+      ))
+  }
+
+  test("test Double with Struct and Array DOUBLE --> SHORT INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string," +
+      "marks:array<double>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'10.323$abc$20.2:30.3:501.423')")
+    sql("insert into adaptive values(2,'10.323$abc$20.2:30.3:502.421')")
+    sql("insert into adaptive values(3,'10.323$abc$20.2:30.3:503.422')")
+    sql("insert into adaptive values(4,'10.323$abc$20.2:30.3:504.424')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(10.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 501.423)))),
+        Row(2, Row(10.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 502.421)))),
+        Row(3, Row(10.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 503.422)))),
+        Row(4, Row(10.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 504.424))))      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string,marks:array<double>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double3.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double3.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double3.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double3.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(3, Row(50.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(3, Row(50.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(3, Row(50.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423)))),
+        Row(3, Row(50.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 500.423))))
+      ))
+  }
+
+  test("test Double with Struct and Array DOUBLE --> INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string," +
+      "marks:array<double>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'1000.323$abc$20.2:30.3:60000.423')")
+    sql("insert into adaptive values(2,'1000.324$abc$20.2:30.3:70000.424')")
+    sql("insert into adaptive values(3,'1000.325$abc$20.2:30.3:80000.425')")
+    sql("insert into adaptive values(4,'1000.326$abc$20.2:30.3:90000.426')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1000.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 60000.423)))),
+        Row(2, Row(1000.324, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 70000.424)))),
+        Row(3, Row(1000.325, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 80000.425)))),
+        Row(4, Row(1000.326, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 90000.426))))
+      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string,marks:array<double>>)" +
+      " " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double4.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double4.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double4.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_double4.csv' into table adaptive options('delimiter'='," +
+      "'," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(3, Row(50000.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(3, Row(50000.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(3, Row(50000.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(1, Row(1.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(2, Row(2.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423)))),
+        Row(3, Row(50000.323, "abc", mutable.WrappedArray.make(Array(20.2, 30.3, 50000.423))))
+      ))
+  }
+
+  test("test Double with Struct and Array DOUBLE --> DOUBLE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:double,name:string," +
+      "marks:array<double>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'1.797693134862315$abc$2.2:30.3:1.797693134862315')")
+    sql("insert into adaptive values(2,'1.797693134862316$abc$2.2:30.3:1.797693134862316')")
+    sql("insert into adaptive values(3,'1.797693134862317$abc$2.2:30.3:1.797693134862317')")
+    sql("insert into adaptive values(4,'1.797693134862318$abc$2.2:30.3:1.797693134862318')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1,
+        Row(1.797693134862315,
+          "abc",
+          mutable.WrappedArray.make(Array(2.2, 30.3, 1.797693134862315)))),
+        Row(2,
+          Row(1.797693134862316,
+            "abc",
+            mutable.WrappedArray.make(Array(2.2, 30.3, 1.797693134862316)))),
+        Row(3,
+          Row(1.797693134862317,
+            "abc",
+            mutable.WrappedArray.make(Array(2.2, 30.3, 1.797693134862317)))),
+        Row(4,
+          Row(1.797693134862318,
+            "abc",
+            mutable.WrappedArray.make(Array(2.2, 30.3, 1.797693134862318))))
+      ))
+
+  }
+
+  test("test Decimal with Struct") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:decimal(3,2),name:string>) stored by " +
+      "'carbondata'")
+    sql("insert into adaptive values(1,'3.2$abc')")
+    sql("select * from adaptive").show(false)
+  }
+
+  test("test Decimal with Array") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<name:string," +
+      "marks:array<decimal>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'abc$20.2:30.3:40.4')")
+    sql("select * from adaptive").show(false)
+  }
+
+  test("test Timestamp with Struct") {
+    sql("Drop table if exists adaptive")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+    sql(
+      "create table adaptive(roll int, student struct<id:timestamp,name:string>) stored by " +
+      "'carbondata'")
+    sql("insert into adaptive values(1,'2017/01/01 00:00:00$abc')")
+    sql("insert into adaptive values(2,'2017/01/02 00:00:00$abc')")
+    sql("insert into adaptive values(3,'2017/01/03 00:00:00$abc')")
+    sql("insert into adaptive values(4,'2017/01/04 00:00:00$abc')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(Timestamp.valueOf("2017-01-01 00:00:00.0"), "abc")),
+        Row(2, Row(Timestamp.valueOf("2017-01-02 00:00:00.0"), "abc")),
+        Row(3, Row(Timestamp.valueOf("2017-01-03 00:00:00.0"), "abc")),
+        Row(4, Row(Timestamp.valueOf("2017-01-04 00:00:00.0"), "abc"))
+      ))
+  }
+
+  test("test Timestamp with Array") {
+    sql("Drop table if exists adaptive")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+    sql(
+      "create table adaptive(roll int, student struct<name:string," +
+      "marks:array<timestamp>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'abc1$2017/01/01:2018/01/01')")
+    sql("insert into adaptive values(2,'abc2$2017/01/02:2018/01/03')")
+    sql("insert into adaptive values(3,'abc3$2017/01/04:2018/01/05')")
+    sql("insert into adaptive values(4,'abc4$2017/01/06:2018/01/07')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1,
+        Row("abc1",
+          mutable.WrappedArray
+            .make(Array(Timestamp.valueOf("2017-01-01 00:00:00.0"),
+              Timestamp.valueOf("2018-01-01 00:00:00.0"))))),
+        Row(2,
+          Row("abc2",
+            mutable.WrappedArray
+              .make(Array(Timestamp.valueOf("2017-01-02 00:00:00.0"),
+                Timestamp.valueOf("2018-01-03 00:00:00.0"))))),
+        Row(3,
+          Row("abc3",
+            mutable.WrappedArray
+              .make(Array(Timestamp.valueOf("2017-01-04 00:00:00.0"),
+                Timestamp.valueOf("2018-01-05 00:00:00.0"))))),
+        Row(4,
+          Row("abc4",
+            mutable.WrappedArray
+              .make(Array(Timestamp.valueOf("2017-01-06 00:00:00.0"),
+                Timestamp.valueOf("2018-01-07 00:00:00.0")))))
+      ))
+  }
+
+  test("test DATE with Array") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<name:string," +
+      "marks:array<date>>) stored by 'carbondata'")
+    sql("insert into adaptive values(1,'abc$2017-01-01')")
+    sql("select * from adaptive").show(false)
+  }
+
+  test("test LONG with Array and Struct Encoding LONG --> BYTE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:long,name:string,marks:array<long>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'11111$abc$20:30:40')")
+    sql("insert into adaptive values(2,'11111$abc$55:65:75')")
+    sql("insert into adaptive values(3,'11111$abc$88:98:8')")
+    sql("insert into adaptive values(4,'11111$abc$99:9:19')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(11111, "abc", mutable.WrappedArray.make(Array(20, 30, 40)))),
+        Row(2, Row(11111, "abc", mutable.WrappedArray.make(Array(55, 65, 75)))),
+        Row(3, Row(11111, "abc", mutable.WrappedArray.make(Array(88, 98, 8)))),
+        Row(4, Row(11111, "abc", mutable.WrappedArray.make(Array(99, 9, 19))))
+      ))
+  }
+
+  test("test LONG with Array and Struct Encoding LONG --> SHORT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:long,name:string,marks:array<long>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'11111$abc$200:300:400')")
+    sql("insert into adaptive values(2,'11111$abc$201:301:401')")
+    sql("insert into adaptive values(3,'11111$abc$202:302:402')")
+    sql("insert into adaptive values(4,'11111$abc$203:303:403')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(11111, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(11111, "abc", mutable.WrappedArray.make(Array(201, 301, 401)))),
+        Row(3, Row(11111, "abc", mutable.WrappedArray.make(Array(202, 302, 402)))),
+        Row(4, Row(11111, "abc", mutable.WrappedArray.make(Array(203, 303, 403))))
+      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:LONG,name:string,marks:array<LONG>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400))))
+      ))
+  }
+
+  test("test LONG with struct and array, Encoding LONG-->SHORT INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:LONG,name:string,marks:array<LONG>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int2.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:LONG,name:string,marks:array<LONG>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'50000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(2,'70000$abc$2000000:3000000:4000000')")
+    sql("insert into adaptive values(3,'100000$abc$2000000:3000000:4000000')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(50000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(2, Row(70000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000)))),
+        Row(3, Row(100000, "abc", mutable.WrappedArray.make(Array(2000000, 3000000, 4000000))))))
+  }
+
+  test("test LONG with struct and array, Encoding LONG-->INT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:LONG,name:string,marks:array<LONG>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int3.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(7000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000))))))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:LONG,name:string,marks:array<LONG>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000')")
+    sql("insert into adaptive values(2,'700000$abc$200:300:52000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(2, Row(700000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000)))),
+        Row(3, Row(10000000, "abc", mutable.WrappedArray.make(Array(200, 300, 52000000))))))
+  }
+
+  test("test LONG with struct and array, Encoding LONG-->LONG") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:LONG,name:string,marks:array<LONG>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'500000$abc$200:300:52000000000')")
+    sql("insert into adaptive values(2,'700000$abc$200:300:52000000000')")
+    sql("insert into adaptive values(3,'10000000$abc$200:300:52000000000')")
+    sql("select * from adaptive").show(false)
+  }
+
+  test("test SHORT with Array and Struct Encoding SHORT -->BYTE") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:short,name:string,marks:array<short>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'11$abc$20:30:40')")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(11, "abc", mutable.WrappedArray.make(Array(20, 30, 40))))))
+  }
+
+  test("test SHORT with Array and Struct Encoding SHORT --> SHORT") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:SHORT,name:string,marks:array<SHORT>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'11111$abc$200:300:400')")
+    sql("insert into adaptive values(1,'11111$abc$200:300:401')")
+    sql("insert into adaptive values(1,'11111$abc$200:300:402')")
+    sql("insert into adaptive values(1,'11111$abc$200:300:403')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(11111, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(11111, "abc", mutable.WrappedArray.make(Array(200, 300, 401)))),
+        Row(1, Row(11111, "abc", mutable.WrappedArray.make(Array(200, 300, 402)))),
+        Row(1, Row(11111, "abc", mutable.WrappedArray.make(Array(200, 300, 403))))
+      ))
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:SHORT,name:string,marks:array<SHORT>>) " +
+      "stored by 'carbondata'")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql(
+      s"load data inpath '$resourcesPath/adap_int1.csv' into table adaptive options('delimiter'=','," +
+      "'quotechar'='\"','fileheader'='roll,student','complex_delimiter_level_1'='$'," +
+      "'complex_delimiter_level_2'=':')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(1, Row(500, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(2, Row(700, "abc", mutable.WrappedArray.make(Array(200, 300, 400)))),
+        Row(3, Row(800, "abc", mutable.WrappedArray.make(Array(200, 300, 400))))
+      ))
+  }
+
+  test("test Boolean with Struct and Array") {
+    sql("Drop table if exists adaptive")
+    sql(
+      "create table adaptive(roll int, student struct<id:boolean,name:string," +
+      "marks:array<boolean>>) " +
+      "stored by 'carbondata'")
+    sql("insert into adaptive values(1,'true$abc$false:true:false')")
+    sql("insert into adaptive values(1,'true$abc$false:true:true')")
+    sql("insert into adaptive values(1,'true$abc$false:true:true')")
+    sql("insert into adaptive values(1,'true$abc$false:true:false')")
+    sql("alter table adaptive compact 'major' ")
+    checkAnswer(sql("select * from adaptive"),
+      Seq(Row(1, Row(true, "abc", mutable.WrappedArray.make(Array(false, true, false)))),
+        Row(1, Row(true, "abc", mutable.WrappedArray.make(Array(false, true, true)))),
+        Row(1, Row(true, "abc", mutable.WrappedArray.make(Array(false, true, true)))),
+        Row(1, Row(true, "abc", mutable.WrappedArray.make(Array(false, true, false))))
+      ))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
index 220451b..f2e33f3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
@@ -858,38 +858,6 @@ class TestComplexDataType extends QueryTest with BeforeAndAfterAll {
       arrayException.getMessage)
   }
 
-  test("test block compaction") {
-    sql("DROP TABLE IF EXISTS table1")
-    sql(
-      "create table table1 (roll int,person Struct<detail:int,age:string,height:double>) stored " +
-      "by 'carbondata'")
-    sql(
-      "load data inpath '" + resourcesPath +
-      "/Struct.csv' into table table1 options('delimiter'=','," +
-      "'quotechar'='\"','fileheader'='roll,person','complex_delimiter_level_1'='$'," +
-      "'complex_delimiter_level_2'='&')")
-    sql(
-      "load data inpath '" + resourcesPath +
-      "/Struct.csv' into table table1 options('delimiter'=','," +
-      "'quotechar'='\"','fileheader'='roll,person','complex_delimiter_level_1'='$'," +
-      "'complex_delimiter_level_2'='&')")
-    val exception = intercept[UnsupportedOperationException](
-      sql("alter table table1 compact 'major'"))
-    assertResult(
-      "Compaction is unsupported for Table containing Complex Columns")(
-      exception.getMessage)
-    val exception1 = intercept[UnsupportedOperationException](
-      sql("alter table table1 compact 'minor'"))
-    assertResult(
-      "Compaction is unsupported for Table containing Complex Columns")(
-      exception1.getMessage)
-    val exception2 = intercept[UnsupportedOperationException](
-      sql("alter table table1 compact 'custom' where segment.id in (0,1)"))
-    assertResult(
-      "Compaction is unsupported for Table containing Complex Columns")(
-      exception2.getMessage)
-  }
-
   test("test complex datatype double for encoding") {
     sql("DROP TABLE IF EXISTS table1")
     sql(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 1b1d708..f98c0cf 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -87,13 +87,6 @@ case class CarbonAlterTableCompactionCommand(
     if (!table.getTableInfo.isTransactionalTable) {
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
-
-    if (table.getTableInfo.getFactTable.getListOfColumns.asScala
-      .exists(m => m.getDataType.isComplexType)) {
-      throw new UnsupportedOperationException(
-        "Compaction is unsupported for Table containing Complex Columns")
-    }
-
     if (CarbonUtil.hasAggregationDataMap(table) ||
         (table.isChildDataMap && null == operationContext.getProperty(table.getTableName))) {
       // If the compaction request is of 'streaming' type then we need to generate loadCommands

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index c12ed1c..54c8a91 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -111,8 +111,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
 
   private boolean isDictionary;
 
-  private boolean isEmptyBadRecord;
-
   private String nullformat;
 
   private boolean isDirectDictionary;
@@ -133,13 +131,12 @@ public class PrimitiveDataType implements GenericDataType<Object> {
    * @param isDictionary
    */
   public PrimitiveDataType(String name, DataType dataType, String parentName, String columnId,
-      boolean isDictionary, String nullFormat, boolean isEmptyBadRecord) {
+      boolean isDictionary, String nullFormat) {
     this.name = name;
     this.parentname = parentName;
     this.columnId = columnId;
     this.isDictionary = isDictionary;
     this.nullformat = nullFormat;
-    this.isEmptyBadRecord = isEmptyBadRecord;
     this.dataType = dataType;
   }
 
@@ -154,19 +151,17 @@ public class PrimitiveDataType implements GenericDataType<Object> {
    * @param useOnePass
    * @param localCache
    * @param nullFormat
-   * @param isEmptyBadRecords
    */
   public PrimitiveDataType(CarbonColumn carbonColumn, String parentName, String columnId,
       CarbonDimension carbonDimension, AbsoluteTableIdentifier absoluteTableIdentifier,
       DictionaryClient client, Boolean useOnePass, Map<Object, Integer> localCache,
-      String nullFormat, Boolean isEmptyBadRecords) {
+      String nullFormat) {
     this.name = carbonColumn.getColName();
     this.parentname = parentName;
     this.columnId = columnId;
     this.carbonDimension = carbonDimension;
     this.isDictionary = isDictionaryDimension(carbonDimension);
     this.nullformat = nullFormat;
-    this.isEmptyBadRecord = isEmptyBadRecords;
     this.dataType = carbonColumn.getDataType();
 
     DictionaryColumnUniqueIdentifier identifier =
@@ -566,7 +561,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
     dataType.parentname = this.parentname;
     dataType.columnId = this.columnId;
     dataType.dictionaryGenerator = this.dictionaryGenerator;
-    dataType.isEmptyBadRecord = this.isEmptyBadRecord;
     dataType.nullformat = this.nullformat;
     dataType.setKeySize(this.keySize);
     dataType.setSurrogateIndex(this.index);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index 435cf24..3b4df75 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -184,7 +184,7 @@ public class FieldEncoderFactory {
     } else {
       return new PrimitiveDataType(carbonColumn, parentName, carbonColumn.getColumnId(),
           (CarbonDimension) carbonColumn, absoluteTableIdentifier, client, useOnePass,
-          localCache, nullFormat, isEmptyBadRecords);
+          localCache, nullFormat);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
index 63f464e..c0af1a4 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
@@ -326,6 +326,11 @@ public class CarbonCompactionUtil {
         updatedCardinalityList.add(value);
       }
       updatedColumnSchemaList.add(dimension.getColumnSchema());
+
+      if (dimension.getNumberOfChild() > 0) {
+        fillColumnSchemaListForComplexDims(dimension.getListOfChildDimensions(),
+            updatedColumnSchemaList, updatedCardinalityList, columnCardinalityMap);
+      }
     }
     // add measures to the column schema list
     List<CarbonMeasure> masterSchemaMeasures =
@@ -338,6 +343,34 @@ public class CarbonCompactionUtil {
   }
 
   /**
+   * This method is to get the chile dimensions of the complex dimension and
+   * update the cardinality for all complex dimensions
+   *
+   * @param carbonDimensionsList
+   * @param updatedColumnSchemaList
+   * @param updatedCardinalityList
+   * @param columnCardinalityMap
+   */
+  private static void fillColumnSchemaListForComplexDims(List<CarbonDimension> carbonDimensionsList,
+      List<ColumnSchema> updatedColumnSchemaList, List<Integer> updatedCardinalityList,
+      Map<String, Integer> columnCardinalityMap) {
+    for (CarbonDimension carbonDimension : carbonDimensionsList) {
+      Integer value = columnCardinalityMap.get(carbonDimension.getColumnId());
+      if (null == value) {
+        updatedCardinalityList.add(getDimensionDefaultCardinality(carbonDimension));
+      } else {
+        updatedCardinalityList.add(value);
+      }
+      updatedColumnSchemaList.add(carbonDimension.getColumnSchema());
+      List<CarbonDimension> childDims = carbonDimension.getListOfChildDimensions();
+      if (null != childDims && childDims.size() > 0) {
+        fillColumnSchemaListForComplexDims(childDims, updatedColumnSchemaList,
+            updatedCardinalityList, columnCardinalityMap);
+      }
+    }
+  }
+
+  /**
    * This method will return the default cardinality based on dimension type
    *
    * @param dimension

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index f70e749..b502da2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -44,7 +44,10 @@ import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datamap.DataMapWriterListener;
+import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
+import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
+import org.apache.carbondata.processing.datatypes.StructDataType;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
@@ -370,10 +373,10 @@ public class CarbonFactDataHandlerModel {
     int[] formattedCardinality = CarbonUtil
         .getFormattedCardinality(segmentProperties.getDimColumnsCardinality(), wrapperColumnSchema);
     carbonFactDataHandlerModel.setColCardinality(formattedCardinality);
-    //TO-DO Need to handle complex types here .
-    Map<Integer, GenericDataType> complexIndexMap =
-        new HashMap<Integer, GenericDataType>(segmentProperties.getComplexDimensions().size());
-    carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
+
+    carbonFactDataHandlerModel.setComplexIndexMap(
+        convertComplexDimensionToGenericDataType(segmentProperties,
+            loadModel.getSerializationNullFormat()));
     DataType[] measureDataTypes = new DataType[segmentProperties.getMeasures().size()];
     int i = 0;
     for (CarbonMeasure msr : segmentProperties.getMeasures()) {
@@ -408,6 +411,84 @@ public class CarbonFactDataHandlerModel {
   }
 
   /**
+   * This routine takes the Complex Dimension and convert into generic DataType.
+   *
+   * @param segmentProperties
+   * @param isNullFormat
+   * @return
+   */
+  private static Map<Integer, GenericDataType> convertComplexDimensionToGenericDataType(
+      SegmentProperties segmentProperties, String isNullFormat) {
+    List<CarbonDimension> complexDimensions = segmentProperties.getComplexDimensions();
+    Map<Integer, GenericDataType> complexIndexMap = new HashMap<>(complexDimensions.size());
+    int dimensionCount = -1;
+    if (segmentProperties.getDimensions().size() == 0) {
+      dimensionCount = 0;
+    } else {
+      dimensionCount = segmentProperties.getDimensions().size() - segmentProperties
+          .getNumberOfNoDictionaryDimension() - segmentProperties.getComplexDimensions().size();
+    }
+    for (CarbonDimension carbonDimension : complexDimensions) {
+      if (carbonDimension.isComplex()) {
+        GenericDataType genericDataType;
+        DataType dataType = carbonDimension.getDataType();
+        if (DataTypes.isArrayType(dataType)) {
+          genericDataType =
+              new ArrayDataType(carbonDimension.getColName(), "", carbonDimension.getColumnId());
+        } else if (DataTypes.isStructType(dataType)) {
+          genericDataType =
+              new StructDataType(carbonDimension.getColName(), "", carbonDimension.getColumnId());
+        } else {
+          // Add Primitive type.
+          throw new RuntimeException("Primitive Type should not be coming in first loop");
+        }
+        if (carbonDimension.getNumberOfChild() > 0) {
+          addChildrenForComplex(carbonDimension.getListOfChildDimensions(), genericDataType,
+              isNullFormat);
+        }
+        genericDataType.setOutputArrayIndex(0);
+        complexIndexMap.put(dimensionCount++, genericDataType);
+      }
+
+    }
+    return complexIndexMap;
+  }
+
+  private static void addChildrenForComplex(List<CarbonDimension> listOfChildDimensions,
+      GenericDataType genericDataType, String isNullFormat) {
+    for (CarbonDimension carbonDimension : listOfChildDimensions) {
+      String parentColName =
+          carbonDimension.getColName().substring(0, carbonDimension.getColName().lastIndexOf("."));
+      DataType dataType = carbonDimension.getDataType();
+      if (DataTypes.isArrayType(dataType)) {
+        GenericDataType arrayGeneric =
+            new ArrayDataType(carbonDimension.getColName(), parentColName,
+                carbonDimension.getColumnId());
+        if (carbonDimension.getNumberOfChild() > 0) {
+          addChildrenForComplex(carbonDimension.getListOfChildDimensions(), arrayGeneric,
+              isNullFormat);
+        }
+        genericDataType.addChildren(arrayGeneric);
+      } else if (DataTypes.isStructType(dataType)) {
+        GenericDataType structGeneric =
+            new StructDataType(carbonDimension.getColName(), parentColName,
+                carbonDimension.getColumnId());
+        if (carbonDimension.getNumberOfChild() > 0) {
+          addChildrenForComplex(carbonDimension.getListOfChildDimensions(), structGeneric,
+              isNullFormat);
+        }
+        genericDataType.addChildren(structGeneric);
+      } else {
+        // Primitive Data Type
+        genericDataType.addChildren(
+            new PrimitiveDataType(carbonDimension.getColumnSchema().getColumnName(),
+                dataType, parentColName, carbonDimension.getColumnId(),
+                carbonDimension.getColumnSchema().hasEncoding(Encoding.DICTIONARY), isNullFormat));
+      }
+    }
+  }
+
+  /**
    * This method will get the store location for the given path, segment id and partition id
    *
    * @return data directory path

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e8d25bb3/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index 1d1f451..98b2543 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -321,9 +321,6 @@ public final class CarbonDataProcessorUtil {
     String nullFormat =
         configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
             .toString();
-    boolean isEmptyBadRecord = Boolean.parseBoolean(
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
-            .toString());
 
     Map<String, GenericDataType> complexTypesMap = new LinkedHashMap<String, GenericDataType>();
     String[] hierarchies = complexTypeString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
@@ -347,8 +344,8 @@ public final class CarbonDataProcessorUtil {
         } else {
           g.addChildren(
               new PrimitiveDataType(levelInfo[0], DataTypeUtil.valueOf(levelInfo[1]),
-                  levelInfo[2], levelInfo[4], levelInfo[3].contains("true"), nullFormat,
-                  isEmptyBadRecord));
+                  levelInfo[2], levelInfo[4], levelInfo[3].contains("true"), nullFormat
+              ));
         }
       }
     }