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 2017/04/18 12:04:07 UTC

[31/50] incubator-carbondata git commit: fixed NullPointerException for 2nd level subquery

fixed NullPointerException for 2nd level subquery


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

Branch: refs/heads/branch-1.1
Commit: fc0fc7d906c9f6579bc3fc67b5672cac4de55068
Parents: 27a1384
Author: kunal642 <ku...@knoldus.in>
Authored: Tue Apr 11 16:43:39 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Apr 14 12:54:09 2017 +0530

----------------------------------------------------------------------
 .../src/test/resources/temp/data1.csv           |  4 ++
 .../sql/optimizer/CarbonLateDecodeRule.scala    |  6 ++-
 .../carbondata/query/SubQueryTestSuite.scala    | 44 ++++++++++++++++++++
 3 files changed, 53 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fc0fc7d9/integration/spark-common-test/src/test/resources/temp/data1.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/temp/data1.csv b/integration/spark-common-test/src/test/resources/temp/data1.csv
new file mode 100644
index 0000000..44e9c10
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/temp/data1.csv
@@ -0,0 +1,4 @@
+id,name,rating
+1,xyz,5
+2,ghj,2
+3,ghj,3

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fc0fc7d9/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
index 181328d..45cc330 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
@@ -60,7 +60,11 @@ class CarbonLateDecodeRule extends Rule[LogicalPlan] with PredicateHelper {
     if (relations.nonEmpty && !isOptimized(plan)) {
       // In case scalar subquery skip the transformation and update the flag.
       if (relations.exists(_.carbonRelation.isSubquery.nonEmpty)) {
-        relations.foreach(p => p.carbonRelation.isSubquery.remove(0))
+        relations.foreach{carbonDecoderRelation =>
+          if (carbonDecoderRelation.carbonRelation.isSubquery.nonEmpty) {
+            carbonDecoderRelation.carbonRelation.isSubquery.remove(0)
+          }
+        }
         LOGGER.info("Skip CarbonOptimizer for scalar/predicate sub query")
         return plan
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fc0fc7d9/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
new file mode 100644
index 0000000..fbc859f
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.spark.carbondata.query
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class SubQueryTestSuite extends QueryTest with BeforeAndAfterAll {
+
+  val tempDirPath = s"$resourcesPath/temp"
+
+  override def beforeAll(){
+    sql("drop table if exists subquery")
+    sql("create table subquery(id int, name string, rating float) stored by 'carbondata'")
+    sql(s"load data local inpath '$tempDirPath/data1.csv' into table subquery")
+  }
+
+  test("test to check if 2nd level subquery gives correct result") {
+    checkAnswer(sql(
+      "select * from subquery where id in(select id from subquery where name in(select name from" +
+      " subquery where rating=2.0))"),
+      Seq(Row(2,"ghj",2.0), Row(3,"ghj",3.0)))
+  }
+
+  override def afterAll() {
+    sql("drop table if exists subquery")
+  }
+}