You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@carbondata.apache.org by GitBox <gi...@apache.org> on 2021/09/15 18:42:11 UTC

[GitHub] [carbondata] maheshrajus opened a new pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

maheshrajus opened a new pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218


   
   
    ### Why is this PR needed?
    Alter add complex columns with global sort compaction is failed
    
    ### What changes were proposed in this PR?
   need to add
       
    ### Does this PR introduce any user interface change?
    - No
   
   
    ### Is any new testcase added?
    - Yes
   
       
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710911684



##########
File path: processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLoggerProvider.java
##########
@@ -33,6 +33,16 @@
    * @return
    */
   public static BadRecordsLogger createBadRecordLogger(CarbonDataLoadConfiguration configuration) {
+    return createBadRecordLogger(configuration, false);
+  }
+  /**

Review comment:
       please format this comment. empty line should be added at line:38 and removed in line:43




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-922034299


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4191/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921739903


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5930/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709947754



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+
+
+  test("test the multi-level complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global2")
+    sql("CREATE TABLE alter_global2(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global2 values(1)")
+    // multi-level nested array
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(arr1 array<array<int>>, arr2 array<struct<a1:string," +
+      "map1:Map<string, string>>>) ")
+    sql(
+      "insert into alter_global2 values(1, array(array(1,2)), array(named_struct('a1','st'," +
+      "'map1', map('a','b'))))")
+    // multi-level nested struct
+    sql("ALTER TABLE alter_global2 ADD COLUMNS(struct1 struct<s1:string, arr: array<int>>," +
+        " struct2 struct<num:double,contact:map<string,array<int>>>) ")
+    sql("insert into alter_global2 values(1, " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))))")
+    // multi-level nested map
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(map1 map<string,array<string>>, map2 map<string," +
+      "struct<d:int, s:struct<im:string>>>)")
+    sql("insert into alter_global2 values(1,  " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))),map('a',array('hi')), map('a',named_struct('d',23,'s',named_struct('im'," +
+        "'sh'))))")
+    checkAnswer(sql("select * from alter_global2"),
+      Seq(Row(1, null, null, null, null, null, null),
+        Row(1, make(Array(make(Array(1, 2)))), make(Array(Row("st", Map("a" -> "b")))),
+          null, null, null, null),
+        Row(1, make(Array(make(Array(1, 2)))), make(Array(Row("st", Map("a" -> "b")))),
+          Row("hi", make(Array(1, 2))), Row(2.3, Map("ph" -> make(Array(1, 2)))), null, null),
+        Row(1, make(Array(make(Array(1, 2)))), make(Array(Row("st", Map("a" -> "b")))),
+          Row("hi", make(Array(1, 2))), Row(2.3, Map("ph" -> make(Array(1, 2)))),
+          Map("a" -> make(Array("hi"))), Map("a" -> Row(23, Row("sh"))))
+      ))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global2")
+    assert(addedColumns.size == 6)
+    sql("alter table alter_global2 compact 'minor'")
+    checkAnswer(sql("select * from alter_global2"),

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710905325



##########
File path: processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLoggerProvider.java
##########
@@ -33,6 +33,16 @@
    * @return
    */
   public static BadRecordsLogger createBadRecordLogger(CarbonDataLoadConfiguration configuration) {
+    return createBadRecordLogger(configuration, false);
+  }
+  /**
+   * method returns the BadRecordsLogger instance
+   * @param configuration
+   * @return
+   */
+

Review comment:
       remove extra line at 43 and add one at 38




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921589808


   Build Failed  with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/329/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710778083



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -221,13 +221,15 @@ object DataLoadProcessorStepOnSpark {
       modelBroadcast: Broadcast[CarbonLoadModel],
       partialSuccessAccum: LongAccumulator,
       rowCounter: LongAccumulator,
-      keepActualData: Boolean = false): Iterator[CarbonRow] = {
+      keepActualData: Boolean = false,
+      isCompactionFlow: Boolean = false): Iterator[CarbonRow] = {
     val model: CarbonLoadModel = modelBroadcast.value.getCopyWithTaskNo(index.toString)
     val conf = DataLoadProcessBuilder.createConfiguration(model)
     val badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(conf)

Review comment:
       instead of adding get/set in badrecord logger we can pass isCompactionFlow to BadRecordsLoggerProvider.createBadRecordLogger and disable the BadRecordLogger in that method.
   
   That will be a clean implementation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920665436


   Build Failed  with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/323/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710761017



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
##########
@@ -342,7 +347,12 @@ object DataLoadProcessBuilderOnSpark {
       .mapPartitionsWithIndex { case (index, rows) =>
         ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
         DataLoadProcessorStepOnSpark
-          .convertFunc(rows, index, modelBroadcast, partialSuccessAccum, convertStepRowCounter)
+          .convertFunc(rows,
+            index,
+            modelBroadcast,
+            false,

Review comment:
       not required to pass after making isCompactionFlow as the last argument




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921594951


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5927/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
kunal642 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-922634460


   LGTM..PR needs some refactoring..please fix in another PR


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710316879



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
##########
@@ -100,13 +102,15 @@ public ArrayDataType(String name, String parentName, String columnId) {
    * @param parentName
    * @param columnId
    * @param isDictionary
+   * @param nullFormat
    */
   public ArrayDataType(String name, String parentName, String columnId,
-      Boolean isDictionary) {
+      Boolean isDictionary, String nullFormat) {

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710768226



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -219,6 +219,7 @@ object DataLoadProcessorStepOnSpark {
       rows: Iterator[CarbonRow],
       index: Int,
       modelBroadcast: Broadcast[CarbonLoadModel],
+      isCompactionFlow: Boolean = false,

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710840593



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
##########
@@ -342,7 +347,12 @@ object DataLoadProcessBuilderOnSpark {
       .mapPartitionsWithIndex { case (index, rows) =>
         ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
         DataLoadProcessorStepOnSpark
-          .convertFunc(rows, index, modelBroadcast, partialSuccessAccum, convertStepRowCounter)
+          .convertFunc(rows,
+            index,
+            modelBroadcast,
+            false,

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709818240



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+

Review comment:
       remove extra line here

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+
+
+  test("test the multi-level complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global2")
+    sql("CREATE TABLE alter_global2(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global2 values(1)")
+    // multi-level nested array
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(arr1 array<array<int>>, arr2 array<struct<a1:string," +
+      "map1:Map<string, string>>>) ")
+    sql(
+      "insert into alter_global2 values(1, array(array(1,2)), array(named_struct('a1','st'," +
+      "'map1', map('a','b'))))")
+    // multi-level nested struct
+    sql("ALTER TABLE alter_global2 ADD COLUMNS(struct1 struct<s1:string, arr: array<int>>," +
+        " struct2 struct<num:double,contact:map<string,array<int>>>) ")
+    sql("insert into alter_global2 values(1, " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))))")
+    // multi-level nested map
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(map1 map<string,array<string>>, map2 map<string," +
+      "struct<d:int, s:struct<im:string>>>)")
+    sql("insert into alter_global2 values(1,  " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))),map('a',array('hi')), map('a',named_struct('d',23,'s',named_struct('im'," +
+        "'sh'))))")
+    checkAnswer(sql("select * from alter_global2"),
+      Seq(Row(1, null, null, null, null, null, null),
+        Row(1, make(Array(make(Array(1, 2)))), make(Array(Row("st", Map("a" -> "b")))),
+          null, null, null, null),
+        Row(1, make(Array(make(Array(1, 2)))), make(Array(Row("st", Map("a" -> "b")))),
+          Row("hi", make(Array(1, 2))), Row(2.3, Map("ph" -> make(Array(1, 2)))), null, null),
+        Row(1, make(Array(make(Array(1, 2)))), make(Array(Row("st", Map("a" -> "b")))),
+          Row("hi", make(Array(1, 2))), Row(2.3, Map("ph" -> make(Array(1, 2)))),
+          Map("a" -> make(Array("hi"))), Map("a" -> Row(23, Row("sh"))))
+      ))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global2")
+    assert(addedColumns.size == 6)
+    sql("alter table alter_global2 compact 'minor'")
+    checkAnswer(sql("select * from alter_global2"),

Review comment:
       check answer can be reused here

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+
+
+  test("test the multi-level complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global2")
+    sql("CREATE TABLE alter_global2(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global2 values(1)")
+    // multi-level nested array
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(arr1 array<array<int>>, arr2 array<struct<a1:string," +
+      "map1:Map<string, string>>>) ")
+    sql(
+      "insert into alter_global2 values(1, array(array(1,2)), array(named_struct('a1','st'," +
+      "'map1', map('a','b'))))")
+    // multi-level nested struct
+    sql("ALTER TABLE alter_global2 ADD COLUMNS(struct1 struct<s1:string, arr: array<int>>," +
+        " struct2 struct<num:double,contact:map<string,array<int>>>) ")
+    sql("insert into alter_global2 values(1, " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))))")
+    // multi-level nested map
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(map1 map<string,array<string>>, map2 map<string," +
+      "struct<d:int, s:struct<im:string>>>)")
+    sql("insert into alter_global2 values(1,  " +

Review comment:
       please add a insert with null and empty data also




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921593083


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4182/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921165126


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5925/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709846172



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-922034998


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5937/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710760760



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -219,6 +219,7 @@ object DataLoadProcessorStepOnSpark {
       rows: Iterator[CarbonRow],
       index: Int,
       modelBroadcast: Broadcast[CarbonLoadModel],
+      isCompactionFlow: Boolean = false,

Review comment:
       why add this in between? if you keep in last then scala will take the default value if not passed, so change in DataLoadProcessBuilderOnSpark.scala to pass false will not be required 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710295417



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
##########
@@ -100,13 +102,15 @@ public ArrayDataType(String name, String parentName, String columnId) {
    * @param parentName
    * @param columnId
    * @param isDictionary
+   * @param nullFormat
    */
   public ArrayDataType(String name, String parentName, String columnId,
-      Boolean isDictionary) {
+      Boolean isDictionary, String nullFormat) {

Review comment:
       We no need to handle this way. Here, compaction is failing because of logging bad record error for inserting NULL for previous segments. Bad record should be done for Load/Insert flow only. We can avoid performing Bad record validation during validation by passing a flag from global sort flow. Please revert these changes, as it might induce new issues if actual data is NULL/empty data type




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920673004


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4176/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921264136


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4181/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921164728


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4180/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920896111


   Build Success with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/325/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710840447



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -221,13 +221,15 @@ object DataLoadProcessorStepOnSpark {
       modelBroadcast: Broadcast[CarbonLoadModel],
       partialSuccessAccum: LongAccumulator,
       rowCounter: LongAccumulator,
-      keepActualData: Boolean = false): Iterator[CarbonRow] = {
+      keepActualData: Boolean = false,
+      isCompactionFlow: Boolean = false): Iterator[CarbonRow] = {
     val model: CarbonLoadModel = modelBroadcast.value.getCopyWithTaskNo(index.toString)
     val conf = DataLoadProcessBuilder.createConfiguration(model)
     val badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(conf)

Review comment:
       OK, Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710760760



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -219,6 +219,7 @@ object DataLoadProcessorStepOnSpark {
       rows: Iterator[CarbonRow],
       index: Int,
       modelBroadcast: Broadcast[CarbonLoadModel],
+      isCompactionFlow: Boolean = false,

Review comment:
       why add this in between? if you keep in last then scala will take the default value if not passed, so change in DataLoadProcessBuilderOnSpark.scala will not be required 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710010456



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
##########
@@ -171,13 +171,13 @@ public boolean getIsColumnDictionary() {
   }
 
   @Override
-  public void writeByteArray(ArrayObject input, DataOutputStream dataOutputStream,
+  public void writeByteArray(Object input, DataOutputStream dataOutputStream,
       BadRecordLogHolder logHolder, Boolean isWithoutConverter) throws IOException {
-    if (input == null) {
+    if (input == null || input.equals("\\N")) {

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921249585


   Build Success with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/328/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920884864


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5923/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709845377



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
##########
@@ -412,16 +412,18 @@ private void updateValueToByteStream(DataOutputStream dataOutputStream, byte[] v
     dataOutputStream.write(value);
   }
 
-  private void updateNullValue(DataOutputStream dataOutputStream, BadRecordLogHolder logHolder)
-      throws IOException {
+  private void updateNullValue(Object input, DataOutputStream dataOutputStream,
+      BadRecordLogHolder logHolder) throws IOException {
     CarbonUtil.updateNullValueBasedOnDatatype(dataOutputStream, this.carbonDimension.getDataType());
-    String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
-    if (null == message) {
-      message = CarbonDataProcessorUtil
-          .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
-      logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
+    if (null == input) {
+      String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
+      if (null == message) {
+        message = CarbonDataProcessorUtil
+            .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
+        logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
+        logHolder.setReason(message);

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920349874


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4173/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710861289



##########
File path: processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLoggerProvider.java
##########
@@ -33,6 +33,16 @@
    * @return
    */
   public static BadRecordsLogger createBadRecordLogger(CarbonDataLoadConfiguration configuration) {
+    return createBadRecordLogger(configuration, false);
+  }
+  /**
+   * method returns the BadRecordsLogger instance
+   * @param configuration
+   * @return
+   */
+
+  public static BadRecordsLogger createBadRecordLogger(CarbonDataLoadConfiguration configuration,
+      Boolean isForceDisale) {

Review comment:
       ```suggestion
         Boolean isForceDisable) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709966830



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+
+
+  test("test the multi-level complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global2")
+    sql("CREATE TABLE alter_global2(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global2 values(1)")
+    // multi-level nested array
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(arr1 array<array<int>>, arr2 array<struct<a1:string," +
+      "map1:Map<string, string>>>) ")
+    sql(
+      "insert into alter_global2 values(1, array(array(1,2)), array(named_struct('a1','st'," +
+      "'map1', map('a','b'))))")
+    // multi-level nested struct
+    sql("ALTER TABLE alter_global2 ADD COLUMNS(struct1 struct<s1:string, arr: array<int>>," +
+        " struct2 struct<num:double,contact:map<string,array<int>>>) ")
+    sql("insert into alter_global2 values(1, " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))))")
+    // multi-level nested map
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(map1 map<string,array<string>>, map2 map<string," +
+      "struct<d:int, s:struct<im:string>>>)")
+    sql("insert into alter_global2 values(1,  " +

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709785985



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
##########
@@ -412,16 +412,18 @@ private void updateValueToByteStream(DataOutputStream dataOutputStream, byte[] v
     dataOutputStream.write(value);
   }
 
-  private void updateNullValue(DataOutputStream dataOutputStream, BadRecordLogHolder logHolder)
-      throws IOException {
+  private void updateNullValue(Object input, DataOutputStream dataOutputStream,
+      BadRecordLogHolder logHolder) throws IOException {
     CarbonUtil.updateNullValueBasedOnDatatype(dataOutputStream, this.carbonDimension.getDataType());
-    String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
-    if (null == message) {
-      message = CarbonDataProcessorUtil
-          .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
-      logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
+    if (null == input) {
+      String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
+      if (null == message) {
+        message = CarbonDataProcessorUtil
+            .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
+        logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
+        logHolder.setReason(message);

Review comment:
       can move setReason to line 426 after closing one if case of (null == message)

##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
##########
@@ -171,13 +171,13 @@ public boolean getIsColumnDictionary() {
   }
 
   @Override
-  public void writeByteArray(ArrayObject input, DataOutputStream dataOutputStream,
+  public void writeByteArray(Object input, DataOutputStream dataOutputStream,
       BadRecordLogHolder logHolder, Boolean isWithoutConverter) throws IOException {
-    if (input == null) {
+    if (input == null || input.equals("\\N")) {

Review comment:
       check if we could get `SERIALIZATION_NULL_FORMAT ` while creating complex type from FieldEncoderFactory




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920882643


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4178/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920304108


   Build Failed  with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/319/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921758736


   Build Failed  with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/332/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] maheshrajus commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710402835



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
##########
@@ -342,7 +347,12 @@ object DataLoadProcessBuilderOnSpark {
       .mapPartitionsWithIndex { case (index, rows) =>
         ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
         DataLoadProcessorStepOnSpark
-          .convertFunc(rows, index, modelBroadcast, partialSuccessAccum, convertStepRowCounter)
+          .convertFunc(rows,
+            index,
+            modelBroadcast,
+            true,

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-922031666


   Build Success with Spark 3.1, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1/338/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921453773


   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710860620



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
##########
@@ -342,7 +348,11 @@ object DataLoadProcessBuilderOnSpark {
       .mapPartitionsWithIndex { case (index, rows) =>
         ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
         DataLoadProcessorStepOnSpark
-          .convertFunc(rows, index, modelBroadcast, partialSuccessAccum, convertStepRowCounter)
+          .convertFunc(rows,
+            index,

Review comment:
       please revert this change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710329999



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
##########
@@ -342,7 +347,12 @@ object DataLoadProcessBuilderOnSpark {
       .mapPartitionsWithIndex { case (index, rows) =>
         ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
         DataLoadProcessorStepOnSpark
-          .convertFunc(rows, index, modelBroadcast, partialSuccessAccum, convertStepRowCounter)
+          .convertFunc(rows,
+            index,
+            modelBroadcast,
+            true,

Review comment:
       this flow looks like load flow. This change might not be needed. can test and remove if not required




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710295417



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
##########
@@ -100,13 +102,15 @@ public ArrayDataType(String name, String parentName, String columnId) {
    * @param parentName
    * @param columnId
    * @param isDictionary
+   * @param nullFormat
    */
   public ArrayDataType(String name, String parentName, String columnId,
-      Boolean isDictionary) {
+      Boolean isDictionary, String nullFormat) {

Review comment:
       We no need to handle this way. Here, compaction is failing because of logging bad record error for inserting NULL for previous segments. Bad record should be done for Load/Insert flow only. We can avoid performing Bad record validation during compaction by passing a flag from global sort compaction flow. Please revert these changes, as it might induce new issues if actual data is NULL/empty data type




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r710295417



##########
File path: processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
##########
@@ -100,13 +102,15 @@ public ArrayDataType(String name, String parentName, String columnId) {
    * @param parentName
    * @param columnId
    * @param isDictionary
+   * @param nullFormat
    */
   public ArrayDataType(String name, String parentName, String columnId,
-      Boolean isDictionary) {
+      Boolean isDictionary, String nullFormat) {

Review comment:
       We no need to handle this way. Here, compaction is because of logging bad record error for inserting NULL for previous segments. Bad record should be done for Load/Insert flow only. We can avoid performing Bad record validation during validation by passing a flag from global sort flow. Please revert these changes, as it might induce new issues if actual data is NULL/empty data type




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920669842


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5921/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921255642


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5926/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921750349


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_2.4.5/4185/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#discussion_r709822162



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableAddColumns.scala
##########
@@ -583,4 +583,78 @@ class TestAlterTableAddColumns extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true")
   }
+
+  test("test the complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global1")
+    sql("CREATE TABLE alter_global1(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global1 values(1)")
+    sql("insert into alter_global1 values(2)")
+    sql("insert into alter_global1 values(3)")
+    sql( "ALTER TABLE alter_global1 ADD COLUMNS(str1 array<int>)")
+    sql("insert into alter_global1 values(4, array(1))")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    val addedColumns = addedColumnsInSchemaEvolutionEntry("alter_global1")
+    assert(addedColumns.size == 1)
+    sql("alter table alter_global1 compact 'minor'")
+    checkAnswer(sql("select * from alter_global1"),
+      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, make(Array(1)))))
+    sql("DROP TABLE IF EXISTS alter_global1")
+  }
+
+
+  test("test the multi-level complex columns with global sort compaction") {
+    sql("DROP TABLE IF EXISTS alter_global2")
+    sql("CREATE TABLE alter_global2(intField INT) STORED AS carbondata " +
+        "TBLPROPERTIES('sort_columns'='intField','sort_scope'='global_sort')")
+    sql("insert into alter_global2 values(1)")
+    // multi-level nested array
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(arr1 array<array<int>>, arr2 array<struct<a1:string," +
+      "map1:Map<string, string>>>) ")
+    sql(
+      "insert into alter_global2 values(1, array(array(1,2)), array(named_struct('a1','st'," +
+      "'map1', map('a','b'))))")
+    // multi-level nested struct
+    sql("ALTER TABLE alter_global2 ADD COLUMNS(struct1 struct<s1:string, arr: array<int>>," +
+        " struct2 struct<num:double,contact:map<string,array<int>>>) ")
+    sql("insert into alter_global2 values(1, " +
+        "array(array(1,2)), array(named_struct('a1','st','map1', map('a','b'))), " +
+        "named_struct('s1','hi','arr',array(1,2)), named_struct('num',2.3,'contact',map('ph'," +
+        "array(1,2))))")
+    // multi-level nested map
+    sql(
+      "ALTER TABLE alter_global2 ADD COLUMNS(map1 map<string,array<string>>, map2 map<string," +
+      "struct<d:int, s:struct<im:string>>>)")
+    sql("insert into alter_global2 values(1,  " +

Review comment:
       please add a insert with null and empty data also after add columns




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ShreelekhyaG commented on pull request #4218: [CARBONDATA-4285] Fix alter add complex columns with global sort compaction failure

Posted by GitBox <gi...@apache.org>.
ShreelekhyaG commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-921511077


   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4218: [CARBONDATA-4285] Alter add complex columns with global sort compaction is failed

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4218:
URL: https://github.com/apache/carbondata/pull/4218#issuecomment-920346301


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12602/job/ApacheCarbonPRBuilder2.3/5917/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org