Skip to content

Commit

Permalink
[SPARK-6740][SQL] Fix NOT operator precedence.
Browse files Browse the repository at this point in the history
NOT has lower precedence than comparison operations.
  • Loading branch information
smola committed Jun 24, 2015
1 parent bba6699 commit fbc1815
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,12 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser {
andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1, e2) })

protected lazy val andExpression: Parser[Expression] =
comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1, e2) })
booleanFactor * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1, e2) })

protected lazy val booleanFactor: Parser[Expression] =
NOT.? ~ comparisonExpression ^^ {
case notOpt ~ expr => notOpt.map(s => Not(expr)).getOrElse(expr)
}

protected lazy val comparisonExpression: Parser[Expression] =
( termExpression ~ ("=" ~> termExpression) ^^ { case e1 ~ e2 => EqualTo(e1, e2) }
Expand Down Expand Up @@ -235,7 +240,6 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser {
}
| termExpression <~ IS ~ NULL ^^ { case e => IsNull(e) }
| termExpression <~ IS ~ NOT ~ NULL ^^ { case e => IsNotNull(e) }
| NOT ~> termExpression ^^ {e => Not(e)}
| termExpression
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,10 @@
package org.apache.spark.sql.catalyst

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.analysis.{UnresolvedStar, UnresolvedRelation}
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.logical.Command

Expand Down Expand Up @@ -51,6 +54,15 @@ private[sql] class CaseInsensitiveTestParser extends AbstractSparkSQLParser {

class SqlParserSuite extends SparkFunSuite {

test("precedence of NOT operator") {
val parser = new SqlParser
val actual = parser.parse("SELECT * FROM t WHERE NOT c IS NULL")
val expected = UnresolvedRelation("t" :: Nil)
.where('c.isNull.unary_!)
.select(UnresolvedStar(None))
assertResult(expected)(actual)
}

test("test long keyword") {
val parser = new SuperLongKeywordTestParser
assert(TestCommand("NotRealCommand") ===
Expand Down

0 comments on commit fbc1815

Please sign in to comment.