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/13 14:31:31 UTC
carbondata git commit: [CARBONDATA-2755][Complex DataType
Enhancements] Compaction Complex Types (STRUCT AND ARRAY)
Repository: carbondata
Updated Branches:
refs/heads/master d3c907b71 -> 88fa067e0
[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/88fa067e
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/88fa067e
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/88fa067e
Branch: refs/heads/master
Commit: 88fa067e0edb81a50bd23021acaf7469b58b74a9
Parents: d3c907b
Author: sounakr <so...@gmail.com>
Authored: Tue Jul 17 10:35:32 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Dec 13 20:01:22 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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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/88fa067e/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
+ ));
}
}
}