diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index d17c9553ac24e..b3b77513d6640 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ -import scala.math.pow case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any @@ -43,7 +42,7 @@ case class Sqrt(child: Expression) extends UnaryExpression { override def toString = s"SQRT($child)" override def eval(input: Row): Any = { - n1(child, input, ((na,a) => math.sqrt(na.toDouble(a)))) + n1(child, input, (na,a) => math.sqrt(na.toDouble(a))) } } @@ -134,7 +133,7 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme case ShortType => (evalE1.asInstanceOf[Short] & evalE2.asInstanceOf[Short]).toShort case IntegerType => evalE1.asInstanceOf[Int] & evalE2.asInstanceOf[Int] case LongType => evalE1.asInstanceOf[Long] & evalE2.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise & operation on ${other}") + case other => sys.error(s"Unsupported bitwise & operation on $other") } } @@ -149,7 +148,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet case ShortType => (evalE1.asInstanceOf[Short] | evalE2.asInstanceOf[Short]).toShort case IntegerType => evalE1.asInstanceOf[Int] | evalE2.asInstanceOf[Int] case LongType => evalE1.asInstanceOf[Long] | evalE2.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise | operation on ${other}") + case other => sys.error(s"Unsupported bitwise | operation on $other") } } @@ -164,7 +163,7 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme case ShortType => (evalE1.asInstanceOf[Short] ^ evalE2.asInstanceOf[Short]).toShort case IntegerType => evalE1.asInstanceOf[Int] ^ evalE2.asInstanceOf[Int] case LongType => evalE1.asInstanceOf[Long] ^ evalE2.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise ^ operation on ${other}") + case other => sys.error(s"Unsupported bitwise ^ operation on $other") } } @@ -177,7 +176,7 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { def dataType = child.dataType override def foldable = child.foldable def nullable = child.nullable - override def toString = s"-$child" + override def toString = s"~$child" override def eval(input: Row): Any = { val evalE = child.eval(input) @@ -185,11 +184,11 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { null } else { dataType match { - case ByteType => (~(evalE.asInstanceOf[Byte])).toByte - case ShortType => (~(evalE.asInstanceOf[Short])).toShort - case IntegerType => ~(evalE.asInstanceOf[Int]) - case LongType => ~(evalE.asInstanceOf[Long]) - case other => sys.error(s"Unsupported bitwise ~ operation on ${other}") + case ByteType => (~evalE.asInstanceOf[Byte]).toByte + case ShortType => (~evalE.asInstanceOf[Short]).toShort + case IntegerType => ~evalE.asInstanceOf[Int] + case LongType => ~evalE.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise ~ operation on $other") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 3f5b9f698f827..c9395ac18a3e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -42,6 +42,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(1) + Literal(1), 2) } + test("unary BitwiseNOT") { + checkEvaluation(BitwiseNot(1), -2) + assert(BitwiseNot(1).dataType === IntegerType) + assert(BitwiseNot(1).eval(EmptyRow).isInstanceOf[Int]) + checkEvaluation(BitwiseNot(1.toLong), -2.toLong) + assert(BitwiseNot(1.toLong).dataType === LongType) + assert(BitwiseNot(1.toLong).eval(EmptyRow).isInstanceOf[Long]) + checkEvaluation(BitwiseNot(1.toShort), -2.toShort) + assert(BitwiseNot(1.toShort).dataType === ShortType) + assert(BitwiseNot(1.toShort).eval(EmptyRow).isInstanceOf[Short]) + checkEvaluation(BitwiseNot(1.toByte), -2.toByte) + assert(BitwiseNot(1.toByte).dataType === ByteType) + assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte]) + } + /** * Checks for three-valued-logic. Based on: * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29