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/03/23 04:25:08 UTC

[1/2] incubator-carbondata git commit: Whenever a not equal to filter is applied on dictionary column with numeric datatype, the cast added by spark plan is removed while creating carbon filters from spark filter. Due to this plan modification incorrect

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master c2a3e7d9b -> 9f38a3dde


Whenever a not equal to filter is applied on dictionary column with numeric datatype, the cast added by spark plan is removed while creating carbon filters from spark filter. Due to this plan modification incorrect results are returned by spark.


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

Branch: refs/heads/master
Commit: ae86464dfb34d9556ad62cf30ad3618dc6907961
Parents: c2a3e7d
Author: manishgupta88 <to...@gmail.com>
Authored: Tue Mar 21 23:15:43 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Mar 23 09:53:45 2017 +0530

----------------------------------------------------------------------
 .../test/resources/filter/notEqualToFilter.csv  |  3 +
 .../DateDataTypeDirectDictionaryTest.scala      | 15 +---
 ...TypeDirectDictionaryWithNoDictTestCase.scala |  2 +-
 .../apache/carbondata/spark/CarbonFilters.scala | 42 +++------
 .../execution/CarbonLateDecodeStrategy.scala    | 59 ++++---------
 .../carbondata/query/TestNotEqualToFilter.scala | 90 ++++++++++++++++++++
 6 files changed, 124 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ae86464d/integration/spark-common-test/src/test/resources/filter/notEqualToFilter.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/filter/notEqualToFilter.csv b/integration/spark-common-test/src/test/resources/filter/notEqualToFilter.csv
new file mode 100644
index 0000000..31ed808
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/filter/notEqualToFilter.csv
@@ -0,0 +1,3 @@
+1,2015-07-23 00:00:00,china,aaa1,phone197,ASD69643,15000
+7,2015-07-24 00:00:00,china,aaa2,phone756,ASD42892,15001
+7,2015-07-25 00:00:00,china,aaa3,phone1904,ASD37014,15002

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ae86464d/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryTest.scala
index 2e5eeb0..267bdf8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryTest.scala
@@ -39,25 +39,17 @@ class DateDataTypeDirectDictionaryTest extends QueryTest with BeforeAndAfterAll
     try {
       CarbonProperties.getInstance().addProperty("carbon.direct.dictionary", "true")
       sql("drop table if exists directDictionaryTable ")
-      sql("drop table if exists directDictionaryTable_hive ")
       sql(
         "CREATE TABLE if not exists directDictionaryTable (empno int,doj date, " +
           "salary int) " +
           "STORED BY 'org.apache.carbondata.format'"
       )
 
-      sql(
-        "CREATE TABLE if not exists directDictionaryTable_hive (empno int,doj date, " +
-          "salary int) " +
-          "row format delimited fields terminated by ','"
-      )
-
       CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
       val csvFilePath = s"$resourcesPath/datasamplefordate.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO TABLE directDictionaryTable OPTIONS" +
         "('DELIMITER'= ',', 'QUOTECHAR'= '\"')" )
-      sql("LOAD DATA local inpath '" + csvFilePath + "' INTO TABLE directDictionaryTable_hive")
     } catch {
       case x: Throwable =>
         x.printStackTrace()
@@ -87,7 +79,7 @@ class DateDataTypeDirectDictionaryTest extends QueryTest with BeforeAndAfterAll
 
   test("test direct dictionary for not equals condition") {
     checkAnswer(
-      sql("select doj from directDictionaryTable where doj != '2016-04-14 00:00:00'"),
+      sql("select doj from directDictionaryTable where doj != '2016-04-14'"),
       Seq(Row(Date.valueOf("2016-03-14"))
       )
     )
@@ -103,7 +95,7 @@ class DateDataTypeDirectDictionaryTest extends QueryTest with BeforeAndAfterAll
 
   test("select doj from directDictionaryTable with equals filter") {
     checkAnswer(
-      sql("select doj from directDictionaryTable where doj = '2016-03-14 00:00:00'"),
+      sql("select doj from directDictionaryTable where doj = '2016-03-14'"),
       Seq(Row(Date.valueOf("2016-03-14")))
     )
 
@@ -119,7 +111,7 @@ class DateDataTypeDirectDictionaryTest extends QueryTest with BeforeAndAfterAll
   test("select doj from directDictionaryTable with regexp_replace NOT IN filter") {
     checkAnswer(
       sql("select doj from directDictionaryTable where regexp_replace(doj, '-', '/') NOT IN ('2016/03/14')"),
-      sql("select doj from directDictionaryTable_hive where regexp_replace(doj, '-', '/') NOT IN ('2016/03/14')")
+      Seq(Row(Date.valueOf("2016-04-14")))
     )
   }
 
@@ -139,7 +131,6 @@ class DateDataTypeDirectDictionaryTest extends QueryTest with BeforeAndAfterAll
 
   override def afterAll {
     sql("drop table directDictionaryTable")
-    sql("drop table directDictionaryTable_hive")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "dd-MM-yyyy")
     CarbonProperties.getInstance().addProperty("carbon.direct.dictionary", "false")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ae86464d/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
index 2ad8592..e8d4a92 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
@@ -72,7 +72,7 @@ class DateDataTypeDirectDictionaryWithNoDictTestCase extends QueryTest with Befo
 
   test("select doj from directDictionaryTable with equals filter") {
     checkAnswer(
-      sql("select doj from directDictionaryTable where doj='2016-03-14 15:00:09'"),
+      sql("select doj from directDictionaryTable where doj='2016-03-14'"),
       Seq(Row(Date.valueOf("2016-03-14")))
     )
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ae86464d/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
index 3d13c6b..7a3c737 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
@@ -141,24 +141,26 @@ object CarbonFilters {
           }
 
         case And(left, right) =>
-          (translate(left) ++ translate(right)).reduceOption(sources.And)
+          val leftFilter = translate(left, or)
+          val rightFilter = translate(right, or)
+          if (or) {
+            if (leftFilter.isDefined && rightFilter.isDefined) {
+              (leftFilter ++ rightFilter).reduceOption(sources.And)
+            } else {
+              None
+            }
+          } else {
+            (leftFilter ++ rightFilter).reduceOption(sources.And)
+          }
 
         case EqualTo(a: Attribute, Literal(v, t)) =>
           Some(sources.EqualTo(a.name, v))
         case EqualTo(l@Literal(v, t), a: Attribute) =>
           Some(sources.EqualTo(a.name, v))
-        case EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
-          Some(sources.EqualTo(a.name, v))
-        case EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
-          Some(sources.EqualTo(a.name, v))
         case Not(EqualTo(a: Attribute, Literal(v, t))) =>
           Some(sources.Not(sources.EqualTo(a.name, v)))
         case Not(EqualTo(Literal(v, t), a: Attribute)) =>
           Some(sources.Not(sources.EqualTo(a.name, v)))
-        case Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
-          Some(sources.Not(sources.EqualTo(a.name, v)))
-        case Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
-          Some(sources.Not(sources.EqualTo(a.name, v)))
         case IsNotNull(a: Attribute) =>
           Some(sources.IsNotNull(a.name))
         case IsNull(a: Attribute) =>
@@ -169,44 +171,22 @@ object CarbonFilters {
         case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
           val hSet = list.map(e => e.eval(EmptyRow))
           Some(sources.In(a.name, hSet.toArray))
-        case Not(In(Cast(a: Attribute, _), list)) if !list.exists(!_.isInstanceOf[Literal]) =>
-          val hSet = list.map(e => e.eval(EmptyRow))
-          Some(sources.Not(sources.In(a.name, hSet.toArray)))
-        case In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
-          val hSet = list.map(e => e.eval(EmptyRow))
-          Some(sources.In(a.name, hSet.toArray))
         case GreaterThan(a: Attribute, Literal(v, t)) =>
           Some(sources.GreaterThan(a.name, v))
         case GreaterThan(Literal(v, t), a: Attribute) =>
           Some(sources.LessThan(a.name, v))
-        case GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
-          Some(sources.GreaterThan(a.name, v))
-        case GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
-          Some(sources.LessThan(a.name, v))
         case LessThan(a: Attribute, Literal(v, t)) =>
           Some(sources.LessThan(a.name, v))
         case LessThan(Literal(v, t), a: Attribute) =>
           Some(sources.GreaterThan(a.name, v))
-        case LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
-          Some(sources.LessThan(a.name, v))
-        case LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
-          Some(sources.GreaterThan(a.name, v))
         case GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
           Some(sources.GreaterThanOrEqual(a.name, v))
         case GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
           Some(sources.LessThanOrEqual(a.name, v))
-        case GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-          Some(sources.GreaterThanOrEqual(a.name, v))
-        case GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-          Some(sources.LessThanOrEqual(a.name, v))
         case LessThanOrEqual(a: Attribute, Literal(v, t)) =>
           Some(sources.LessThanOrEqual(a.name, v))
         case LessThanOrEqual(Literal(v, t), a: Attribute) =>
           Some(sources.GreaterThanOrEqual(a.name, v))
-        case LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-          Some(sources.LessThanOrEqual(a.name, v))
-        case LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-          Some(sources.GreaterThanOrEqual(a.name, v))
 
         case others =>
           if (!or) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ae86464d/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
index 549cc1e..16e8a99 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
@@ -393,38 +393,39 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
    * Tries to translate a Catalyst [[Expression]] into data source [[Filter]].
    * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`.
    */
-  protected[sql] def translateFilter(predicate: Expression): Option[Filter] = {
+  protected[sql] def translateFilter(predicate: Expression, or: Boolean = false): Option[Filter] = {
     predicate match {
-      case or@ Or(left, right) =>
+      case or@Or(left, right) =>
 
-        val leftFilter = translateFilter(left)
-        val rightFilter = translateFilter(right)
+        val leftFilter = translateFilter(left, true)
+        val rightFilter = translateFilter(right, true)
         if (leftFilter.isDefined && rightFilter.isDefined) {
-          Some( sources.Or(leftFilter.get, rightFilter.get))
+          Some(sources.Or(leftFilter.get, rightFilter.get))
         } else {
           None
         }
 
       case And(left, right) =>
-        (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
+        val leftFilter = translateFilter(left, or)
+        val rightFilter = translateFilter(right, or)
+        if (or) {
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
+          } else {
+            None
+          }
+        } else {
+          (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
+        }
 
       case EqualTo(a: Attribute, Literal(v, t)) =>
         Some(sources.EqualTo(a.name, v))
       case EqualTo(l@Literal(v, t), a: Attribute) =>
         Some(sources.EqualTo(a.name, v))
-      case EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
-        Some(sources.EqualTo(a.name, v))
-      case EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
-        Some(sources.EqualTo(a.name, v))
-
       case Not(EqualTo(a: Attribute, Literal(v, t))) =>
           Some(sources.Not(sources.EqualTo(a.name, v)))
       case Not(EqualTo(Literal(v, t), a: Attribute)) =>
           Some(sources.Not(sources.EqualTo(a.name, v)))
-      case Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
-          Some(sources.Not(sources.EqualTo(a.name, v)))
-      case Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
-          Some(sources.Not(sources.EqualTo(a.name, v)))
       case IsNotNull(a: Attribute) => Some(sources.IsNotNull(a.name))
       case IsNull(a: Attribute) => Some(sources.IsNull(a.name))
       case Not(In(a: Attribute, list)) if !list.exists(!_.isInstanceOf[Literal]) =>
@@ -433,50 +434,22 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
       case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
         val hSet = list.map(e => e.eval(EmptyRow))
         Some(sources.In(a.name, hSet.toArray))
-      case Not(In(Cast(a: Attribute, _), list))
-        if !list.exists(!_.isInstanceOf[Literal]) =>
-        val hSet = list.map(e => e.eval(EmptyRow))
-        Some(sources.Not(sources.In(a.name, hSet.toArray)))
-      case In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
-        val hSet = list.map(e => e.eval(EmptyRow))
-        Some(sources.In(a.name, hSet.toArray))
-
       case GreaterThan(a: Attribute, Literal(v, t)) =>
         Some(sources.GreaterThan(a.name, v))
       case GreaterThan(Literal(v, t), a: Attribute) =>
         Some(sources.LessThan(a.name, v))
-      case GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
-        Some(sources.GreaterThan(a.name, v))
-      case GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
-        Some(sources.LessThan(a.name, v))
-
       case LessThan(a: Attribute, Literal(v, t)) =>
         Some(sources.LessThan(a.name, v))
       case LessThan(Literal(v, t), a: Attribute) =>
         Some(sources.GreaterThan(a.name, v))
-      case LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
-        Some(sources.LessThan(a.name, v))
-      case LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
-        Some(sources.GreaterThan(a.name, v))
-
       case GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
         Some(sources.GreaterThanOrEqual(a.name, v))
       case GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
         Some(sources.LessThanOrEqual(a.name, v))
-      case GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-        Some(sources.GreaterThanOrEqual(a.name, v))
-      case GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-        Some(sources.LessThanOrEqual(a.name, v))
-
       case LessThanOrEqual(a: Attribute, Literal(v, t)) =>
         Some(sources.LessThanOrEqual(a.name, v))
       case LessThanOrEqual(Literal(v, t), a: Attribute) =>
         Some(sources.GreaterThanOrEqual(a.name, v))
-      case LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-        Some(sources.LessThanOrEqual(a.name, v))
-      case LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-        Some(sources.GreaterThanOrEqual(a.name, v))
-
       case others => None
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ae86464d/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala
new file mode 100644
index 0000000..fb0188c
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala
@@ -0,0 +1,90 @@
+/*
+ * 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.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * Test cases for testing columns having \N or \null values for non numeric columns
+ */
+class TestNotEqualToFilter extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists test_not_equal_to_carbon")
+    sql("drop table if exists test_not_equal_to_hive")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT
+      )
+    sql(
+      """
+        CREATE TABLE IF NOT EXISTS test_not_equal_to_carbon
+        (ID Int, date Timestamp, country String,
+        name String, phonetype String, serialname String, salary Int)
+        STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+      """)
+    sql(
+      """
+        CREATE TABLE IF NOT EXISTS test_not_equal_to_hive
+        (ID Int, date Timestamp, country String,
+        name String, phonetype String, serialname String, salary Int)
+        row format delimited fields terminated by ','
+      """)
+    sql(
+      s"""
+           LOAD DATA LOCAL INPATH '$resourcesPath/filter/notEqualToFilter.csv' into table
+           test_not_equal_to_carbon
+           OPTIONS('FILEHEADER'='ID,date,country,name,phonetype,serialname,salary')
+           """)
+    sql(
+      s"""
+           LOAD DATA LOCAL INPATH '$resourcesPath/filter/notEqualToFilter.csv' into table
+           test_not_equal_to_hive
+           """)
+  }
+
+  test("select Id from test_not_equal_to_carbon where id != '7'") {
+    checkAnswer(
+      sql("select Id from test_not_equal_to_carbon where id != '7'"),
+      sql("select Id from test_not_equal_to_hive where id != '7'")
+    )
+  }
+
+  test("select Id from test_not_equal_to_carbon where id != 7.0") {
+    checkAnswer(
+      sql("select Id from test_not_equal_to_carbon where id != 7.0"),
+      sql("select Id from test_not_equal_to_hive where id != 7.0")
+    )
+  }
+
+  test("select Id from test_not_equal_to_carbon where id != 7") {
+    checkAnswer(
+      sql("select Id from test_not_equal_to_carbon where id != 7"),
+      sql("select Id from test_not_equal_to_hive where id != 7")
+    )
+  }
+
+  override def afterAll {
+    sql("drop table if exists test_not_equal_to_carbon")
+    sql("drop table if exists test_not_equal_to_hive")
+  }
+}


[2/2] incubator-carbondata git commit: [CARBONDATA-803] Incorrect results returned by not equal to filter on dictionary column with numeric data type This closes #683

Posted by ra...@apache.org.
[CARBONDATA-803] Incorrect results returned by not equal to filter on dictionary column with numeric data type This closes #683


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

Branch: refs/heads/master
Commit: 9f38a3dde5ace4824f2ffc0789ea01488dfbfc16
Parents: c2a3e7d ae86464
Author: ravipesala <ra...@gmail.com>
Authored: Thu Mar 23 09:54:26 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Mar 23 09:54:26 2017 +0530

----------------------------------------------------------------------
 .../test/resources/filter/notEqualToFilter.csv  |  3 +
 .../DateDataTypeDirectDictionaryTest.scala      | 15 +---
 ...TypeDirectDictionaryWithNoDictTestCase.scala |  2 +-
 .../apache/carbondata/spark/CarbonFilters.scala | 42 +++------
 .../execution/CarbonLateDecodeStrategy.scala    | 59 ++++---------
 .../carbondata/query/TestNotEqualToFilter.scala | 90 ++++++++++++++++++++
 6 files changed, 124 insertions(+), 87 deletions(-)
----------------------------------------------------------------------