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 2016/09/01 09:23:32 UTC

[1/2] incubator-carbondata git commit: Fix the bug that subquery with sort and filter the result is empty

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 30988c501 -> ff7a8382e


Fix the bug that subquery with sort and filter the result is empty

Fix the bug that subquery with sort and filter the result is empty

Fix the bug that subquery with sort and filter the result is empty

Fix test case

Fix test case


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

Branch: refs/heads/master
Commit: 7cd362390f84340433a6b79261c9eb66aa2800aa
Parents: 30988c5
Author: Zhangshunyu <zh...@huawei.com>
Authored: Thu Sep 1 15:33:01 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 1 14:51:38 2016 +0530

----------------------------------------------------------------------
 .../spark/sql/optimizer/CarbonOptimizer.scala   |  2 +-
 .../SubqueryWithFilterAndSortTestCase.scala     | 82 ++++++++++++++++++++
 2 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cd36239/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
index 4566261..bb0b33f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
@@ -218,7 +218,7 @@ class ResolveCarbonFunctions(relations: Seq[CarbonDecoderRelation])
         case filter: Filter if !filter.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
           val attrsOnConds = new util.HashSet[AttributeReferenceWrapper]
           // In case the child is join then we cannot push down the filters so decode them earlier
-          if (filter.child.isInstanceOf[Join]) {
+          if (filter.child.isInstanceOf[Join] || filter.child.isInstanceOf[Sort]) {
             filter.condition.collect {
               case attr: AttributeReference =>
                 attrsOnConds.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cd36239/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/SubqueryWithFilterAndSortTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/SubqueryWithFilterAndSortTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/SubqueryWithFilterAndSortTestCase.scala
new file mode 100644
index 0000000..029c355
--- /dev/null
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/SubqueryWithFilterAndSortTestCase.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.spark.testsuite.detailquery
+
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType
+
+class SubqueryWithFilterAndSortTestCase extends QueryTest with BeforeAndAfterAll {
+  val tempDirPath = "./src/test/resources/temp"
+  val tempFilePath = "./src/test/resources/temp/subqueryfilterwithsort.csv"
+
+  override def beforeAll {
+    FileFactory.mkdirs(tempDirPath,FileType.LOCAL)
+    sql("drop table if exists subqueryfilterwithsort")
+    sql("drop table if exists subqueryfilterwithsort_hive")
+    sql("CREATE TABLE subqueryfilterwithsort (name String, id int) STORED BY 'org.apache.carbondata.format'")
+    sql("CREATE TABLE subqueryfilterwithsort_hive (name String, id int)row format delimited fields terminated by ','")
+    val data ="name_a,1\nname_b,2\nname_c,3\nname_d,4\nname_e,5\nname_f,6"
+    writedata(tempFilePath, data)
+    sql(s"LOAD data local inpath '${tempFilePath}' into table subqueryfilterwithsort options('fileheader'='name,id')")
+    sql(s"LOAD data local inpath '${tempFilePath}' into table subqueryfilterwithsort_hive")
+  }
+
+  test("When the query has sub-query with sort and has '=' filter") {
+    try {
+      checkAnswer(sql("select name,id from (select * from subqueryfilterwithsort order by id)t where name='name_c' "),
+        sql("select name,id from (select * from subqueryfilterwithsort_hive order by id)t where name='name_c'"))
+    } catch{
+      case ex:Exception => ex.printStackTrace()
+        assert(false)
+    }
+  }
+
+  test("When the query has sub-query with sort and has 'like' filter") {
+    try {
+      checkAnswer(sql("select name,id from (select * from subqueryfilterwithsort order by id)t where name like 'name%' "),
+        sql("select name,id from (select * from subqueryfilterwithsort_hive order by id)t where name like 'name%'"))
+    } catch{
+      case ex:Exception => ex.printStackTrace()
+        assert(false)
+    }
+  }
+
+  def writedata(filePath: String, data: String) = {
+    val dis = FileFactory.getDataOutputStream(filePath, FileFactory.getFileType(filePath))
+    dis.writeBytes(data.toString())
+    dis.close()
+  }
+  def deleteFile(filePath: String) {
+    val file = FileFactory.getCarbonFile(filePath, FileFactory.getFileType(filePath))
+    file.delete()
+  }
+
+  override def afterAll {
+    sql("drop table if exists subqueryfilterwithsort")
+    sql("drop table if exists subqueryfilterwithsort_hive")
+    deleteFile(tempFilePath)
+    deleteFile(tempDirPath)
+  }
+
+}


[2/2] incubator-carbondata git commit: Fix the bug that when subquery with sort and filter the result is empty This closes #117

Posted by ra...@apache.org.
Fix the bug that when subquery with sort and filter the result is empty This closes #117


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

Branch: refs/heads/master
Commit: ff7a8382e9b2e0c2de54e31fe6e03d9b16985c17
Parents: 30988c5 7cd3623
Author: ravipesala <ra...@gmail.com>
Authored: Thu Sep 1 14:52:43 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 1 14:52:43 2016 +0530

----------------------------------------------------------------------
 .../spark/sql/optimizer/CarbonOptimizer.scala   |  2 +-
 .../SubqueryWithFilterAndSortTestCase.scala     | 82 ++++++++++++++++++++
 2 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------