carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravipes...@apache.org
Subject [1/3] incubator-carbondata git commit: Cast Push Down Optimization
Date Mon, 01 May 2017 01:08:13 GMT
Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 472f9c9d5 -> b99c2c1a1


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
new file mode 100644
index 0000000..eb8a7a0
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.sql
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, ExprId, LeafExpression,
NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.DataType
+
+import org.apache.carbondata.core.scan.expression.ColumnExpression
+
+case class CastExpr(expr: Expression) extends Filter {
+  override def references: Array[String] = null
+}
+
+case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nullable: Boolean)
+  extends LeafExpression with NamedExpression with CodegenFallback {
+
+  type EvaluatedType = Any
+
+  override def toString: String = s"input[" + colExp.getColIndex + "]"
+
+  override def eval(input: InternalRow): Any = input.get(colExp.getColIndex, dataType)
+
+  override def name: String = colExp.getColumnName
+
+  override def toAttribute: Attribute = throw new UnsupportedOperationException
+
+  override def exprId: ExprId = throw new UnsupportedOperationException
+
+  override def qualifier: Option[String] = null
+
+  override def newInstance(): NamedExpression = throw new UnsupportedOperationException
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
index 85046cf..0bcbcfc 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
@@ -46,17 +46,9 @@ case class CarbonScan(
 
   def processFilterExpressions(plan: CarbonQueryPlan) {
     if (dimensionPredicatesRaw.nonEmpty) {
-      val expressionVal = CarbonFilters.processExpression(
-        dimensionPredicatesRaw,
-        attributesNeedToDecode,
-        unprocessedExprs,
-        carbonTable)
-      expressionVal match {
-        case Some(ce) =>
-          // adding dimension used in expression in querystats
-          plan.setFilterExpression(ce)
-        case _ =>
-      }
+      val exps = CarbonFilters.preProcessExpressions(dimensionPredicatesRaw)
+      val expressionVal = CarbonFilters.transformExpression(exps.head)
+      plan.setFilterExpression(expressionVal)
     }
     processExtraAttributes(plan)
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
index 398a0b7..943c2da 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -15,12 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql
+package org.apache.carbondata.spark
 
 import java.util.{ArrayList, List}
 
 import scala.collection.JavaConverters._
 
+import org.apache.spark.sql.CarbonBoundReference
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression, GenericInternalRow}
 
@@ -108,14 +109,30 @@ class SparkUnknownExpression(var sparkExp: SparkExpression)
   }
 
   def getColumnListFromExpressionTree(sparkCurrentExp: SparkExpression,
-      list: java.util.List[ColumnExpression]): Unit = {
-    sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, list))
+      lst: java.util.List[ColumnExpression]): Unit = {
+    sparkCurrentExp match {
+      case carbonBoundRef: CarbonBoundReference =>
+        val foundExp = lst.asScala
+          .find(p => p.getColumnName() == carbonBoundRef.colExp.getColumnName())
+        if (foundExp.isEmpty) {
+          carbonBoundRef.colExp.setColIndex(lst.size)
+          lst.add(carbonBoundRef.colExp)
+        } else {
+          carbonBoundRef.colExp.setColIndex(foundExp.get.getColIndex())
+        }
+      case _ => sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, lst))
+    }
   }
 
 
+
+
   def getAllColumnListFromExpressionTree(sparkCurrentExp: SparkExpression,
       list: List[ColumnExpression]): List[ColumnExpression] = {
-    sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, list))
+    sparkCurrentExp match {
+      case carbonBoundRef: CarbonBoundReference => list.add(carbonBoundRef.colExp)
+      case _ => sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, list))
+    }
     list
   }
 
@@ -128,4 +145,6 @@ class SparkUnknownExpression(var sparkExp: SparkExpression)
       false
     }
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/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 39fbb09..595c173 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
@@ -17,6 +17,9 @@
 
 package org.apache.spark.sql.execution
 
+import java.text.SimpleDateFormat
+import java.util.Date
+
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
@@ -31,13 +34,14 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning,
Partition
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.optimizer.CarbonDecoderRelation
 import org.apache.spark.sql.sources.{BaseRelation, Filter}
-import org.apache.spark.sql.types.{AtomicType, IntegerType}
+import org.apache.spark.sql.types.{AtomicType, DoubleType, IntegerType, StringType, TimestampType}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampDirectDictionaryGenerator
 import org.apache.carbondata.core.metadata.schema.BucketingInfo
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.CarbonAliasDecoderRelation
+import org.apache.carbondata.spark.{CarbonAliasDecoderRelation}
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
@@ -351,7 +355,6 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     }
   }
 
-
   protected[sql] def selectFilters(
       relation: BaseRelation,
       predicates: Seq[Expression]): (Seq[Expression], Seq[Filter]) = {
@@ -397,8 +400,10 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     (unrecognizedPredicates ++ unhandledPredicates, translatedFilters)
   }
 
+
   /**
    * 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, or: Boolean = false): Option[Filter]
= {
@@ -425,15 +430,22 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
         } 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 c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
       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 c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
       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]) =>
@@ -442,22 +454,43 @@ 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 c@Not(In(Cast(a: Attribute, _), list))
+        if !list.exists(!_.isInstanceOf[Literal]) =>
+        Some(CastExpr(c))
+      case c@In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
+        Some(CastExpr(c))
       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 c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
       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 c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
       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 c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
       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 c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
       case others => None
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
new file mode 100644
index 0000000..71a1032
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
@@ -0,0 +1,373 @@
+/*
+ * 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.sql.execution
+
+import java.text.{ParseException, SimpleDateFormat}
+import java.util
+import java.util.{Locale, TimeZone}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, EmptyRow, EqualTo, Expression,
GreaterThan, GreaterThanOrEqual, In, LessThan, LessThanOrEqual, Literal, Not}
+import org.apache.spark.sql.CastExpr
+import org.apache.spark.sql.sources
+import org.apache.spark.sql.types.{DoubleType, IntegerType, StringType, TimestampType}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+object CastExpressionOptimization {
+
+
+  def typeCastStringToLong(v: Any): Any = {
+    val parser: SimpleDateFormat = new SimpleDateFormat(CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+    try {
+      val value = parser.parse(v.toString).getTime() * 1000L
+      value
+    } catch {
+      case e: ParseException =>
+        try {
+          val parsenew: SimpleDateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz")
+          parsenew.parse(v.toString).getTime() * 1000L
+        } catch {
+          case e: ParseException =>
+            val gmtDay = new SimpleDateFormat("yyyy-MM-dd", Locale.US)
+            gmtDay.setTimeZone(TimeZone.getTimeZone("GMT"))
+            try {
+              gmtDay.parse(v.toString).getTime()
+            } catch {
+              case e: ParseException =>
+                v
+              case e: Exception =>
+                v
+            }
+          case e: Exception =>
+            v
+        }
+      case e: Exception =>
+        v
+    }
+  }
+
+  def typeCastStringToLongList(list: Seq[Expression]): Seq[Expression] = {
+    val tempList = new util.ArrayList[Expression]()
+    list.foreach { value =>
+      val output = typeCastStringToLong(value)
+      if (!output.equals(value)) {
+        tempList.add(output.asInstanceOf[Expression])
+      }
+    }
+    if (tempList.size() != list.size) {
+      list
+    } else {
+      tempList.asScala
+    }
+  }
+
+  def typeCastDoubleToIntList(list: Seq[Expression]): Seq[Expression] = {
+    val tempList = new util.ArrayList[Expression]()
+    list.foreach { value =>
+      val output = value.asInstanceOf[Double].toInt
+      if (value.asInstanceOf[Double].toInt.equals(output)) {
+        tempList.add(output.asInstanceOf[Expression])
+      }
+    }
+    if (tempList.size() != list.size) {
+      list
+    } else {
+      tempList.asScala
+    }
+  }
+
+  /**
+   * This routines tries to apply rules on Cast Filter Predicates and if the rules applied
and the
+   * values can be toss back to native datatypes the cast is removed. Current two rules are
applied
+   * a) Left : timestamp column      Right : String Value
+   * Input from Spark : cast (col as string) <> 'String Literal'
+   * Change to        : Column <> 'Long value of Timestamp String'
+   *
+   * b) Left : Integer Column        Right : String Value
+   * Input from Spark : cast (col as double) <> 'Double Literal'
+   * Change to        : Column <> 'Int value'
+   *
+   * @param expr
+   * @return
+   */
+  def checkIfCastCanBeRemove(expr: Expression): Option[sources.Filter] = {
+    expr match {
+      case c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.EqualTo(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.EqualTo(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.EqualTo(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.EqualTo(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.Not(sources.EqualTo(a.name, value)))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.Not(sources.EqualTo(a.name, value)))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.Not(sources.EqualTo(a.name, value)))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.Not(sources.EqualTo(a.name, value)))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@Not(In(Cast(a: Attribute, _), list)) =>
+        a.dataType match {
+          case t: TimestampType if list.head.dataType.sameType(StringType) =>
+            val value = typeCastStringToLongList(list)
+            if (!value.equals(list)) {
+              val hSet = value.map(e => e.eval(EmptyRow))
+              Some(sources.Not(sources.In(a.name, hSet.toArray)))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if list.head.dataType.sameType(DoubleType) =>
+            val value = typeCastDoubleToIntList(list)
+            if (!value.equals(list)) {
+              val hSet = value.map(e => e.eval(EmptyRow))
+              Some(sources.Not(sources.In(a.name, hSet.toArray)))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@In(Cast(a: Attribute, _), list) =>
+        a.dataType match {
+          case t: TimestampType if list.head.dataType.sameType(StringType) =>
+            val value = typeCastStringToLongList(list)
+            if (!value.equals(list)) {
+              val hSet = value.map(e => e.eval(EmptyRow))
+              Some(sources.In(a.name, hSet.toArray))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if list.head.dataType.sameType(DoubleType) =>
+            val value = typeCastDoubleToIntList(list)
+            if (!value.equals(list)) {
+              val hSet = value.map(e => e.eval(EmptyRow))
+              Some(sources.In(a.name, hSet.toArray))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.GreaterThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.GreaterThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.LessThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.LessThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.LessThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.LessThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.GreaterThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.GreaterThan(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.GreaterThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.GreaterThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.LessThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.LessThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.LessThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.LessThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+      case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        a.dataType match {
+          case t: TimestampType if t.sameType(StringType) =>
+            val value = typeCastStringToLong(v)
+            if (!value.equals(v)) {
+              Some(sources.GreaterThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case i: IntegerType if t.sameType(DoubleType) =>
+            val value = v.asInstanceOf[Double].toInt
+            if (value.toDouble.equals(v)) {
+              Some(sources.GreaterThanOrEqual(a.name, value))
+            } else {
+              Some(CastExpr(c))
+            }
+          case _ => Some(CastExpr(c))
+        }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
new file mode 100644
index 0000000..88343c0
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -0,0 +1,386 @@
+/*
+ * 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
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.CastExpressionOptimization
+import org.apache.spark.sql.optimizer.AttributeReferenceWrapper
+import org.apache.spark.sql.CarbonBoundReference
+import org.apache.spark.sql.CastExpr
+import org.apache.spark.sql.sources
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
+import org.apache.carbondata.core.scan.expression.{ColumnExpression => CarbonColumnExpression,
Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
+import org.apache.carbondata.core.scan.expression.conditional._
+import org.apache.carbondata.core.scan.expression.logical.{AndExpression, FalseExpression,
OrExpression}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+
+/**
+ * All filter conversions are done here.
+ */
+object CarbonFilters {
+
+
+  /**
+   * Converts data sources filters to carbon filter predicates.
+   */
+  def createCarbonFilter(schema: StructType,
+      predicate: sources.Filter): Option[CarbonExpression] = {
+    val dataTypeOf = schema.map(f => f.name -> f.dataType).toMap
+
+    def createFilter(predicate: sources.Filter): Option[CarbonExpression] = {
+      predicate match {
+
+        case sources.EqualTo(name, value) =>
+          Some(new EqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.Not(sources.EqualTo(name, value)) =>
+          Some(new NotEqualsExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.EqualNullSafe(name, value) =>
+          Some(new EqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.Not(sources.EqualNullSafe(name, value)) =>
+          Some(new NotEqualsExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.GreaterThan(name, value) =>
+          Some(new GreaterThanExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.LessThan(name, value) =>
+          Some(new LessThanExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.GreaterThanOrEqual(name, value) =>
+          Some(new GreaterThanEqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.LessThanOrEqual(name, value) =>
+          Some(new LessThanEqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.In(name, values) =>
+          Some(new InExpression(getCarbonExpression(name),
+            new ListExpression(
+              convertToJavaList(values.map(f => getCarbonLiteralExpression(name, f)).toList))))
+        case sources.Not(sources.In(name, values)) =>
+          Some(new NotInExpression(getCarbonExpression(name),
+            new ListExpression(
+              convertToJavaList(values.map(f => getCarbonLiteralExpression(name, f)).toList))))
+        case sources.IsNull(name) =>
+          Some(new EqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, null), true))
+        case sources.IsNotNull(name) =>
+          Some(new NotEqualsExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, null), true))
+        case sources.And(lhs, rhs) =>
+          (createFilter(lhs) ++ createFilter(rhs)).reduceOption(new AndExpression(_, _))
+        case sources.Or(lhs, rhs) =>
+          for {
+            lhsFilter <- createFilter(lhs)
+            rhsFilter <- createFilter(rhs)
+          } yield {
+            new OrExpression(lhsFilter, rhsFilter)
+          }
+        case CastExpr(expr: Expression) =>
+          Some(transformExpression(expr))
+        case _ => None
+      }
+    }
+
+    def getCarbonExpression(name: String) = {
+      new CarbonColumnExpression(name,
+        CarbonScalaUtil.convertSparkToCarbonDataType(dataTypeOf(name)))
+    }
+
+    def getCarbonLiteralExpression(name: String, value: Any): CarbonExpression = {
+      val dataTypeOfAttribute = CarbonScalaUtil.convertSparkToCarbonDataType(dataTypeOf(name))
+      val dataType = if (Option(value).isDefined
+                         && dataTypeOfAttribute == DataType.STRING
+                         && value.isInstanceOf[Double]) {
+        DataType.DOUBLE
+      } else {
+        dataTypeOfAttribute
+      }
+      new CarbonLiteralExpression(value, dataType)
+    }
+
+    createFilter(predicate)
+  }
+
+
+  // Check out which filters can be pushed down to carbon, remaining can be handled in spark
layer.
+  // Mostly dimension filters are only pushed down since it is faster in carbon.
+  // TODO - The Filters are first converted Intermediate sources filters expression and then
these
+  // expressions are again converted back to CarbonExpression. Instead of two step process
of
+  // evaluating the filters it can be merged into a single one.
+  def selectFilters(filters: Seq[Expression],
+      attrList: java.util.HashSet[AttributeReferenceWrapper],
+      aliasMap: CarbonAliasDecoderRelation): Unit = {
+    def translate(expr: Expression, or: Boolean = false): Option[sources.Filter] = {
+      expr match {
+        case or@Or(left, right) =>
+
+          val leftFilter = translate(left, or = true)
+          val rightFilter = translate(right, or = true)
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            Some(sources.Or(leftFilter.get, rightFilter.get))
+          } else {
+            or.collect {
+              case attr: AttributeReference =>
+                attrList.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+            None
+          }
+        case And(left, right) =>
+          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 c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        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 c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        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]) =>
+          val hSet = list.map(e => e.eval(EmptyRow))
+          Some(sources.Not(sources.In(a.name, hSet.toArray)))
+        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 c@Not(In(Cast(a: Attribute, _), list)) if !list.exists(!_.isInstanceOf[Literal])
=>
+          Some(CastExpr(c))
+        case c@In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal])
=>
+            Some(CastExpr(c))
+        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 c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        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 c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        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 c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        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 c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+          CastExpressionOptimization.checkIfCastCanBeRemove(c)
+        case c@Cast(a: Attribute, _) =>
+          Some(CastExpr(c))
+        case others =>
+          if (!or) {
+            others.collect {
+              case attr: AttributeReference =>
+                attrList.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+          }
+          None
+      }
+    }
+
+    filters.flatMap(translate(_, false)).toArray
+  }
+
+  def transformExpression(expr: Expression): CarbonExpression = {
+    expr match {
+      case Or(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=> new
+          OrExpression(transformExpression(left), transformExpression(right))
+      case And(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=> new
+          AndExpression(transformExpression(left), transformExpression(right))
+      case EqualTo(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=> new
+          EqualToExpression(transformExpression(left), transformExpression(right))
+      case Not(EqualTo(left, right))
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=> new
+          NotEqualsExpression(transformExpression(left), transformExpression(right))
+      case IsNotNull(child)
+        if (isCarbonSupportedDataTypes(child)) => new
+          NotEqualsExpression(transformExpression(child), transformExpression(Literal(null)),
true)
+      case IsNull(child)
+        if (isCarbonSupportedDataTypes(child)) => new
+          EqualToExpression(transformExpression(child), transformExpression(Literal(null)),
true)
+      case Not(In(left, right)) if (isCarbonSupportedDataTypes(left)) =>
+        if (right.contains(null)) {
+          new FalseExpression(transformExpression(left))
+        }
+        else {
+          new NotInExpression(transformExpression(left),
+            new ListExpression(convertToJavaList(right.map(transformExpression)))
+          )
+        }
+      case In(left, right) if (isCarbonSupportedDataTypes(left)) =>
+        new InExpression(transformExpression(left),
+          new ListExpression(convertToJavaList(right.filter(_ != null).filter(!isNullLiteral(_))
+            .map(transformExpression))))
+      case InSet(left, right) if (isCarbonSupportedDataTypes(left)) =>
+        val validData = right.filter(_ != null).map { x =>
+          val e = Literal(x.toString)
+          transformExpression(e)
+        }.toList
+        new InExpression(transformExpression(left),
+          new ListExpression(convertToJavaList(validData)))
+      case Not(InSet(left, right)) if (isCarbonSupportedDataTypes(left)) =>
+        if (right.contains(null)) {
+          new FalseExpression(transformExpression(left))
+        }
+        else {
+          val r = right.map { x =>
+            val strVal = if (null == x) {
+              x
+            } else {
+              x.toString
+            }
+            val e = Literal(strVal)
+            transformExpression(e)
+          }.toList
+          new NotInExpression(transformExpression(left), new ListExpression(convertToJavaList(r)))
+        }
+      case GreaterThan(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=>
+        new GreaterThanExpression(transformExpression(left), transformExpression(right))
+      case GreaterThanOrEqual(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=>
+        new GreaterThanEqualToExpression(transformExpression(left), transformExpression(right))
+      case LessThan(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=>
+        new LessThanExpression(transformExpression(left), transformExpression(right))
+      case LessThanOrEqual(left, right)
+        if (isCarbonSupportedDataTypes(left) && isCarbonSupportedDataTypes(right))
=>
+        new LessThanEqualToExpression(transformExpression(left), transformExpression(right))
+      case AttributeReference(name, dataType, _, _) =>
+        new CarbonColumnExpression(name.toString,
+          CarbonScalaUtil.convertSparkToCarbonDataType(dataType))
+      case Literal(name, dataType) => new
+          CarbonLiteralExpression(name, CarbonScalaUtil.convertSparkToCarbonDataType(dataType))
+      case StringTrim(child) => transformExpression(child)
+      case _ =>
+        new SparkUnknownExpression(expr.transform {
+          case AttributeReference(name, dataType, _, _) =>
+            CarbonBoundReference(new CarbonColumnExpression(name.toString,
+              CarbonScalaUtil.convertSparkToCarbonDataType(dataType)), dataType, expr.nullable)
+        }
+        )
+    }
+  }
+
+
+  private def isNullLiteral(exp: Expression): Boolean = {
+    if (null != exp
+        && exp.isInstanceOf[Literal]
+        && (exp.asInstanceOf[Literal].dataType == org.apache.spark.sql.types.DataTypes.NullType)
+        || (exp.asInstanceOf[Literal].value == null)) {
+      true
+    } else {
+      false
+    }
+  }
+
+  def isCarbonSupportedDataTypes(expr: Expression): Boolean = {
+    expr.dataType match {
+      case StringType => true
+      case IntegerType => true
+      case LongType => true
+      case DoubleType => true
+      case FloatType => true
+      case BooleanType => true
+      case TimestampType => true
+      case ArrayType(_, _) => true
+      case StructType(_) => true
+      case DecimalType() => true
+      case _ => false
+    }
+  }
+
+  private def getActualCarbonDataType(column: String, carbonTable: CarbonTable) = {
+    var carbonColumn: CarbonColumn =
+      carbonTable.getDimensionByName(carbonTable.getFactTableName, column)
+    val dataType = if (carbonColumn != null) {
+      carbonColumn.getDataType
+    } else {
+      carbonColumn = carbonTable.getMeasureByName(carbonTable.getFactTableName, column)
+      carbonColumn.getDataType match {
+        case DataType.INT => DataType.LONG
+        case DataType.LONG => DataType.LONG
+        case DataType.DECIMAL => DataType.DECIMAL
+        case _ => DataType.DOUBLE
+      }
+    }
+    CarbonScalaUtil.convertCarbonToSparkDataType(dataType)
+  }
+
+  // Convert scala list to java list, Cannot use scalaList.asJava as while deserializing
it is
+  // not able find the classes inside scala list and gives ClassNotFoundException.
+  private def convertToJavaList(
+      scalaList: Seq[CarbonExpression]): java.util.List[CarbonExpression] = {
+    val javaList = new java.util.ArrayList[CarbonExpression]()
+    scalaList.foreach(javaList.add)
+    javaList
+  }
+
+  def preProcessExpressions(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions match {
+      case left :: right :: rest => preProcessExpressions(List(And(left, right)) ::: rest)
+      case List(left, right) => List(And(left, right))
+
+      case _ => expressions
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/96e5b5cc/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
index fb0188c..cc56509 100644
--- 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
@@ -63,6 +63,9 @@ class TestNotEqualToFilter extends QueryTest with BeforeAndAfterAll {
   }
 
   test("select Id from test_not_equal_to_carbon where id != '7'") {
+   // sql("select id from test_not_equal_to_carbon").show(200,false)
+   // sql("select id from test_not_equal_to_hive").show(200,false)
+    sql("select Id from test_not_equal_to_carbon where id > '1.5rre'").show(200, false)
     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'")



Mime
View raw message