[CARBONDATA-2606][Complex DataType Enhancements]Fix Null result if First two
 Projection column have same parent and third column has different Parent Struct

Problem:
When multiple columns are there,then the first child elements is only going
to make parent Object Array. For all other cases it should be null.
For e.g. a : <b,c,d>. here as 'a' is the parent column and b, c, d are
child columns during traversal when we encounter the first element in
list i.e. column 'b','a' will be completely filled. In case when column
'c' and 'd' encountered then only place null in the output.
Hence, as Null is placed in the output, Select result is Null if First
two Projection column have same parent and third column has different parent 
Struct column.
Solution: Place null in the end of output

This closes #2489


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

Branch: refs/heads/carbonstore
Commit: 75a602d013fce170d59e66b4f907ff405f855d33
Parents: 0c33857
Author: Indhumathi27 <indhumathi...@gmail.com>
Authored: Wed Jul 11 15:54:55 2018 +0530
Committer: kunal642 <kunalkapoor...@gmail.com>
Committed: Sun Jul 15 12:54:28 2018 +0530

----------------------------------------------------------------------
 .../impl/DictionaryBasedResultCollector.java    | 11 +++++++
 .../core/scan/result/BlockletScannedResult.java |  2 +-
 .../complexType/TestComplexDataType.scala       | 31 ++++++++++++++++++++
 .../sql/CarbonDatasourceHadoopRelation.scala    |  2 +-
 4 files changed, 44 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/75a602d0/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
index 495d7de..3184d80 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
@@ -140,6 +140,17 @@ public class DictionaryBasedResultCollector extends 
AbstractScannedResultCollect
         continue;
       }
       fillMeasureData(scannedResult, row);
+      if 
(scannedResult.complexParentIndexToQueryMap.toString().contains("StructQueryType"))
 {
+        // If a : <b,c> and d : <e,f> are two struct and if a.b,a.c,d.e is 
given in the
+        // projection list,then object array will contain a,null,d as result, 
because for a.b,
+        // a will be filled and for a.c null will be placed.
+        // Instead place null in the end of object array and send a,d,null as 
result.
+        int count = 0;
+        for (int j = 0; j < row.length; j++) {
+          if (row[j] != null) row[count++] = row[j];
+        }
+        while (count < row.length) row[count++] = null;
+      }
       listBasedResult.add(row);
       rowCounter++;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/75a602d0/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
index aac76e8..a25ebff 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
@@ -106,7 +106,7 @@ public abstract class BlockletScannedResult {
   /**
    *
    */
-  private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
+  public Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
 
   private int totalDimensionsSize;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/75a602d0/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 1068ba2..45a9c7a 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
@@ -682,4 +682,35 @@ class TestComplexDataType extends QueryTest with 
BeforeAndAfterAll {
       .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
         backupdateFormat)
   }
+
+  test("test Projection with two struct") {
+    sql("DROP TABLE IF EXISTS test")
+    sql("create table test(id int,a struct<b:int,c:int>, d 
struct<e:int,f:int>) stored by 'carbondata'")
+    sql("insert into test values(1,'2$3','3$2')")
+    checkAnswer(sql("select * from test"),Seq(Row(1,Row(2,3),Row(3,2))))
+    checkAnswer(sql("select a.b,id,a.c from test"),Seq(Row(2,1,3)))
+    checkAnswer(sql("select d.e,d.f from test"),Seq(Row(3,2)))
+    checkAnswer(sql("select a.b,d.e,d.f,id,a.c from test"),Seq(Row(2,3,2,1,3)))
+    checkAnswer(sql("select a.b,d.e,id,a.c,d.f,a.c from test"), Seq(Row(2, 3, 
1, 3, 2, 3)))
+    checkAnswer(sql("select a.b,d.e,d.f from test"), Seq(Row(2, 3, 2)))
+    checkAnswer(sql("select a.b,a.c,id,d.e,d.f from test"), Seq(Row(2, 3, 1, 
3, 2)))
+    checkAnswer(sql("select d.e,d.f,id,a.b,a.c from test"), Seq(Row(3, 2, 1, 
2, 3)))
+    checkAnswer(sql("select d.e,1,d.f from test"), Seq(Row(3, 1, 2)))
+    checkAnswer(sql("select d.e,1,d.f,a.b,id,a.c from test"), Seq(Row(3, 1, 2, 
2, 1, 3)))
+    checkAnswer(sql("select d.e+1,d.f,a.b,d.e,a.c,id from test"), 
Seq(Row(4,2,2,3,3,1)))
+    checkAnswer(sql("select d.f,a.c,a.b,id,a.c,a.b from test"), 
Seq(Row(2,3,2,1,3,2)))
+    checkAnswer(sql("select sum(d.e) from test"), Seq(Row(3)))
+    checkAnswer(sql("select d.f,a.c,a.b,id,a.c,a.b,id,1,id,3,d.f from test"), 
Seq(Row(2,3,2,1,3,2,1,1,1,3,2)))
+  }
+
+  test("test project with struct and array") {
+    sql("DROP TABLE IF EXISTS test")
+    sql("create table test(id int,a struct<b:int,c:int>, d 
struct<e:int,f:int>,person Struct<detail:array<int>>) stored by 'carbondata'")
+    sql("insert into test values(1,'2$3','3$2','5:6:7:8')")
+    checkAnswer(sql("select * from 
test"),Seq(Row(1,Row(2,3),Row(3,2),Row(mutable.WrappedArray.make(Array(5,6,7,8))))))
+    checkAnswer(sql("select a.b,id,a.c,person.detail[0] from 
test"),Seq(Row(2,1,3,5)))
+    checkAnswer(sql("select 
a.b,id,a.c,person.detail[0],d.e,d.f,person.detail[1],id from 
test"),Seq(Row(2,1,3,5,3,2,6,1)))
+    checkAnswer(sql("select 
a.b,id,a.c,person.detail[0],d.e,d.f,person.detail[1],id,1,a.b from 
test"),Seq(Row(2,1,3,5,3,2,6,1,1,2)))
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/75a602d0/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index fa57960..8e402b9 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -148,7 +148,7 @@ case class CarbonDatasourceHadoopRelation(
           } else {
             output += col
           }
-          output = output.distinct
+          output = output.map(_.toLowerCase).distinct
         })
       }
       output.toArray.foreach(projection.addColumn)

Reply via email to