From b127ff8a0c5fb704da574d101a2d0e27ac5f463a Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sun, 7 Jun 2015 21:42:45 +0100 Subject: [PATCH 1/5] [SPARK-2808] [STREAMING] [KAFKA] cleanup tests from see if requiring producer acks eliminates the need for waitUntilLeaderOffset calls in tests Author: cody koeninger Closes #5921 from koeninger/kafka-0.8.2-test-cleanup and squashes the following commits: 1e89dc8 [cody koeninger] Merge branch 'master' into kafka-0.8.2-test-cleanup 4662828 [cody koeninger] [Streaming][Kafka] filter mima issue for removal of method from private test class af1e083 [cody koeninger] Merge branch 'master' into kafka-0.8.2-test-cleanup 4298ac2 [cody koeninger] [Streaming][Kafka] update comment to trigger jenkins attempt 1274afb [cody koeninger] [Streaming][Kafka] see if requiring producer acks eliminates the need for waitUntilLeaderOffset calls in tests --- .../spark/streaming/kafka/KafkaTestUtils.scala | 17 ++--------------- .../streaming/kafka/JavaKafkaRDDSuite.java | 3 --- .../spark/streaming/kafka/KafkaRDDSuite.scala | 4 ---- project/MimaExcludes.scala | 3 +++ python/pyspark/streaming/tests.py | 5 ----- 5 files changed, 5 insertions(+), 27 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index 6dc4e9517d5a4..b608b75952721 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -195,6 +195,8 @@ private class KafkaTestUtils extends Logging { val props = new Properties() props.put("metadata.broker.list", brokerAddress) props.put("serializer.class", classOf[StringEncoder].getName) + // wait for all in-sync replicas to ack sends + props.put("request.required.acks", "-1") props } @@ -229,21 +231,6 @@ private class KafkaTestUtils extends Logging { tryAgain(1) } - /** Wait until the leader offset for the given topic/partition equals the specified offset */ - def waitUntilLeaderOffset( - topic: String, - partition: Int, - offset: Long): Unit = { - eventually(Time(10000), Time(100)) { - val kc = new KafkaCluster(Map("metadata.broker.list" -> brokerAddress)) - val tp = TopicAndPartition(topic, partition) - val llo = kc.getLatestLeaderOffsets(Set(tp)).right.get.apply(tp).offset - assert( - llo == offset, - s"$topic $partition $offset not reached after timeout") - } - } - private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { case Some(partitionState) => diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 5cf379635354f..a9dc6e50613ca 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -72,9 +72,6 @@ public void testKafkaRDD() throws InterruptedException { HashMap kafkaParams = new HashMap(); kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); - kafkaTestUtils.waitUntilLeaderOffset(topic1, 0, topic1data.length); - kafkaTestUtils.waitUntilLeaderOffset(topic2, 0, topic2data.length); - OffsetRange[] offsetRanges = { OffsetRange.create(topic1, 0, 0, 1), OffsetRange.create(topic2, 0, 0, 1) diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 054487269a935..d5baf5fd89994 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -61,8 +61,6 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt}") - kafkaTestUtils.waitUntilLeaderOffset(topic, 0, messages.size) - val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( @@ -86,7 +84,6 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { // this is the "lots of messages" case kafkaTestUtils.sendMessages(topic, sent) val sentCount = sent.values.sum - kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount) // rdd defined from leaders after sending messages, should get the number sent val rdd = getRdd(kc, Set(topic)) @@ -113,7 +110,6 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { val sentOnlyOne = Map("d" -> 1) kafkaTestUtils.sendMessages(topic, sentOnlyOne) - kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount + 1) assert(rdd2.isDefined) assert(rdd2.get.count === 0, "got messages when there shouldn't be any") diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 73e4bfd78e577..8a93ca2999510 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -47,6 +47,9 @@ object MimaExcludes { // Mima false positive (was a private[spark] class) ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.util.collection.PairIterator"), + // Removing a testing method from a private class + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.kafka.KafkaTestUtils.waitUntilLeaderOffset"), // SQL execution is considered private. excludePackage("org.apache.spark.sql.execution") ) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 46cb18b2e8ef9..57049beea4dba 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -615,7 +615,6 @@ def test_kafka_stream(self): self._kafkaTestUtils.createTopic(topic) self._kafkaTestUtils.sendMessages(topic, sendData) - self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), "test-streaming-consumer", {topic: 1}, @@ -631,7 +630,6 @@ def test_kafka_direct_stream(self): self._kafkaTestUtils.createTopic(topic) self._kafkaTestUtils.sendMessages(topic, sendData) - self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) self._validateStreamResult(sendData, stream) @@ -646,7 +644,6 @@ def test_kafka_direct_stream_from_offset(self): self._kafkaTestUtils.createTopic(topic) self._kafkaTestUtils.sendMessages(topic, sendData) - self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, fromOffsets) self._validateStreamResult(sendData, stream) @@ -661,7 +658,6 @@ def test_kafka_rdd(self): self._kafkaTestUtils.createTopic(topic) self._kafkaTestUtils.sendMessages(topic, sendData) - self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) self._validateRddResult(sendData, rdd) @@ -677,7 +673,6 @@ def test_kafka_rdd_with_leaders(self): self._kafkaTestUtils.createTopic(topic) self._kafkaTestUtils.sendMessages(topic, sendData) - self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders) self._validateRddResult(sendData, rdd) From 5e7b6b67bed9cd0d8c7d4e78df666b807e8f7ef2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 7 Jun 2015 14:11:20 -0700 Subject: [PATCH 2/5] [SPARK-8117] [SQL] Push codegen implementation into each Expression This PR move codegen implementation of expressions into Expression class itself, make it easy to manage. It introduces two APIs in Expression: ``` def gen(ctx: CodeGenContext): GeneratedExpressionCode def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code ``` gen(ctx) will call genSource(ctx, ev) to generate Java source code for the current expression. A expression needs to override genSource(). Here are the types: ``` type Term String type Code String /** * Java source for evaluating an [[Expression]] given a [[Row]] of input. */ case class GeneratedExpressionCode(var code: Code, nullTerm: Term, primitiveTerm: Term, objectTerm: Term) /** * A context for codegen, which is used to bookkeeping the expressions those are not supported * by codegen, then they are evaluated directly. The unsupported expression is appended at the * end of `references`, the position of it is kept in the code, used to access and evaluate it. */ class CodeGenContext { /** * Holding all the expressions those do not support codegen, will be evaluated directly. */ val references: Seq[Expression] = new mutable.ArrayBuffer[Expression]() } ``` This is basically #6660, but fixed style violation and compilation failure. Author: Davies Liu Author: Reynold Xin Closes #6690 from rxin/codegen and squashes the following commits: e1368c2 [Reynold Xin] Fixed tests. 73db80e [Reynold Xin] Fixed compilation failure. 19d6435 [Reynold Xin] Fixed style violation. 9adaeaf [Davies Liu] address comments f42c732 [Davies Liu] improve coverage and tests bad6828 [Davies Liu] address comments e03edaa [Davies Liu] consts fold 86fac2c [Davies Liu] fix style 02262c9 [Davies Liu] address comments b5d3617 [Davies Liu] Merge pull request #5 from rxin/codegen 48c454f [Reynold Xin] Some code gen update. 2344bc0 [Davies Liu] fix test 12ff88a [Davies Liu] fix build c5fb514 [Davies Liu] rename 8c6d82d [Davies Liu] update docs b145047 [Davies Liu] fix style e57959d [Davies Liu] add type alias 3ff25f8 [Davies Liu] refactor 593d617 [Davies Liu] pushing codegen into Expression --- .../catalyst/expressions/BoundAttribute.scala | 9 + .../spark/sql/catalyst/expressions/Cast.scala | 42 + .../sql/catalyst/expressions/Expression.scala | 100 +++ .../sql/catalyst/expressions/arithmetic.scala | 161 +++- .../expressions/codegen/CodeGenerator.scala | 750 ++++-------------- .../codegen/GenerateMutableProjection.scala | 6 +- .../codegen/GenerateOrdering.scala | 20 +- .../codegen/GeneratePredicate.scala | 4 +- .../codegen/GenerateProjection.scala | 26 +- .../expressions/codegen/package.scala | 3 + .../expressions/decimalFunctions.scala | 19 + .../sql/catalyst/expressions/literals.scala | 54 ++ .../expressions/mathfuncs/binary.scala | 24 +- .../expressions/mathfuncs/unary.scala | 30 +- .../expressions/namedExpressions.scala | 6 +- .../catalyst/expressions/nullFunctions.scala | 55 ++ .../sql/catalyst/expressions/predicates.scala | 192 ++++- .../spark/sql/catalyst/expressions/sets.scala | 54 +- .../expressions/stringOperations.scala | 18 + .../ExpressionEvaluationSuite.scala | 87 +- .../GeneratedEvaluationSuite.scala | 27 +- .../GeneratedMutableEvaluationSuite.scala | 61 -- .../ParquetPartitionDiscoverySuite.scala | 6 +- 23 files changed, 1036 insertions(+), 718 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 1ffc95c676f6f..005de3166095f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.attachTree +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext} import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.trees @@ -41,6 +42,14 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) override def qualifiers: Seq[String] = throw new UnsupportedOperationException override def exprId: ExprId = throw new UnsupportedOperationException + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + s""" + boolean ${ev.isNull} = i.isNullAt($ordinal); + ${ctx.javaType(dataType)} ${ev.primitive} = ${ev.isNull} ? + ${ctx.defaultValue(dataType)} : (${ctx.getColumn(dataType, ordinal)}); + """ + } } object BindReferences extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 21adac144112e..5f76a512679a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -21,6 +21,7 @@ import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext} import org.apache.spark.sql.catalyst.util.DateUtils import org.apache.spark.sql.types._ @@ -433,6 +434,47 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w val evaluated = child.eval(input) if (evaluated == null) null else cast(evaluated) } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + // TODO(cg): Add support for more data types. + (child.dataType, dataType) match { + + case (BinaryType, StringType) => + defineCodeGen (ctx, ev, c => + s"new ${ctx.stringType}().set($c)") + case (DateType, StringType) => + defineCodeGen(ctx, ev, c => + s"""new ${ctx.stringType}().set( + org.apache.spark.sql.catalyst.util.DateUtils.toString($c))""") + // Special handling required for timestamps in hive test cases since the toString function + // does not match the expected output. + case (TimestampType, StringType) => + super.genCode(ctx, ev) + case (_, StringType) => + defineCodeGen(ctx, ev, c => s"new ${ctx.stringType}().set(String.valueOf($c))") + + // fallback for DecimalType, this must be before other numeric types + case (_, dt: DecimalType) => + super.genCode(ctx, ev) + + case (BooleanType, dt: NumericType) => + defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dt)})($c ? 1 : 0)") + case (dt: DecimalType, BooleanType) => + defineCodeGen(ctx, ev, c => s"$c.isZero()") + case (dt: NumericType, BooleanType) => + defineCodeGen(ctx, ev, c => s"$c != 0") + + case (_: DecimalType, IntegerType) => + defineCodeGen(ctx, ev, c => s"($c).toInt()") + case (_: DecimalType, dt: NumericType) => + defineCodeGen(ctx, ev, c => s"($c).to${ctx.boxedType(dt)}()") + case (_: NumericType, dt: NumericType) => + defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dt)})($c)") + + case other => + super.genCode(ctx, ev) + } + } } object Cast { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index b2b9d1a5e1581..0ed576b3d5870 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext, Term} import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types._ @@ -51,6 +52,44 @@ abstract class Expression extends TreeNode[Expression] { /** Returns the result of evaluating this expression on a given input Row */ def eval(input: Row = null): Any + /** + * Returns an [[GeneratedExpressionCode]], which contains Java source code that + * can be used to generate the result of evaluating the expression on an input row. + * + * @param ctx a [[CodeGenContext]] + * @return [[GeneratedExpressionCode]] + */ + def gen(ctx: CodeGenContext): GeneratedExpressionCode = { + val isNull = ctx.freshName("isNull") + val primitive = ctx.freshName("primitive") + val ve = GeneratedExpressionCode("", isNull, primitive) + ve.code = genCode(ctx, ve) + ve + } + + /** + * Returns Java source code that can be compiled to evaluate this expression. + * The default behavior is to call the eval method of the expression. Concrete expression + * implementations should override this to do actual code generation. + * + * @param ctx a [[CodeGenContext]] + * @param ev an [[GeneratedExpressionCode]] with unique terms. + * @return Java source code + */ + protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + ctx.references += this + val objectTerm = ctx.freshName("obj") + s""" + /* expression: ${this} */ + Object ${objectTerm} = expressions[${ctx.references.size - 1}].eval(i); + boolean ${ev.isNull} = ${objectTerm} == null; + ${ctx.javaType(this.dataType)} ${ev.primitive} = ${ctx.defaultValue(this.dataType)}; + if (!${ev.isNull}) { + ${ev.primitive} = (${ctx.boxedType(this.dataType)})${objectTerm}; + } + """ + } + /** * Returns `true` if this expression and all its children have been resolved to a specific schema * and input data types checking passed, and `false` if it still contains any unresolved @@ -116,6 +155,41 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def nullable: Boolean = left.nullable || right.nullable override def toString: String = s"($left $symbol $right)" + + /** + * Short hand for generating binary evaluation code, which depends on two sub-evaluations of + * the same type. If either of the sub-expressions is null, the result of this computation + * is assumed to be null. + * + * @param f accepts two variable names and returns Java code to compute the output. + */ + protected def defineCodeGen( + ctx: CodeGenContext, + ev: GeneratedExpressionCode, + f: (Term, Term) => Code): String = { + // TODO: Right now some timestamp tests fail if we enforce this... + if (left.dataType != right.dataType) { + // log.warn(s"${left.dataType} != ${right.dataType}") + } + + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + val resultCode = f(eval1.primitive, eval2.primitive) + + s""" + ${eval1.code} + boolean ${ev.isNull} = ${eval1.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + ${eval2.code} + if(!${eval2.isNull}) { + ${ev.primitive} = $resultCode; + } else { + ${ev.isNull} = true; + } + } + """ + } } private[sql] object BinaryExpression { @@ -128,6 +202,32 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => + + /** + * Called by unary expressions to generate a code block that returns null if its parent returns + * null, and if not not null, use `f` to generate the expression. + * + * As an example, the following does a boolean inversion (i.e. NOT). + * {{{ + * defineCodeGen(ctx, ev, c => s"!($c)") + * }}} + * + * @param f function that accepts a variable name and returns Java code to compute the output. + */ + protected def defineCodeGen( + ctx: CodeGenContext, + ev: GeneratedExpressionCode, + f: Term => Code): Code = { + val eval = child.gen(ctx) + // reuse the previous isNull + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + ${ev.primitive} = ${f(eval.primitive)}; + } + """ + } } // TODO Semantically we probably not need GroupExpression 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 a3770f998d94d..3ac7c92dcd009 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.codegen.{Code, GeneratedExpressionCode, CodeGenContext} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -49,6 +50,11 @@ case class UnaryMinus(child: Expression) extends UnaryArithmetic { private lazy val numeric = TypeUtils.getNumeric(dataType) + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = dataType match { + case dt: DecimalType => defineCodeGen(ctx, ev, c => s"c.unary_$$minus()") + case dt: NumericType => defineCodeGen(ctx, ev, c => s"-($c)") + } + protected override def evalInternal(evalE: Any) = numeric.negate(evalE) } @@ -67,6 +73,21 @@ case class Sqrt(child: Expression) extends UnaryArithmetic { if (value < 0) null else math.sqrt(value) } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval = child.gen(ctx) + eval.code + s""" + boolean ${ev.isNull} = ${eval.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + if (${eval.primitive} < 0.0) { + ${ev.isNull} = true; + } else { + ${ev.primitive} = java.lang.Math.sqrt(${eval.primitive}); + } + } + """ + } } /** @@ -86,6 +107,9 @@ case class Abs(child: Expression) extends UnaryArithmetic { abstract class BinaryArithmetic extends BinaryExpression { self: Product => + /** Name of the function for this expression on a [[Decimal]] type. */ + def decimalMethod: String = "" + override def dataType: DataType = left.dataType override def checkInputDataTypes(): TypeCheckResult = { @@ -114,6 +138,17 @@ abstract class BinaryArithmetic extends BinaryExpression { } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = dataType match { + case dt: DecimalType => + defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)") + // byte and short are casted into int when add, minus, times or divide + case ByteType | ShortType => + defineCodeGen(ctx, ev, (eval1, eval2) => + s"(${ctx.javaType(dataType)})($eval1 $symbol $eval2)") + case _ => + defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1 $symbol $eval2") + } + protected def evalInternal(evalE1: Any, evalE2: Any): Any = sys.error(s"BinaryArithmetics must override either eval or evalInternal") } @@ -124,6 +159,7 @@ private[sql] object BinaryArithmetic { case class Add(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "+" + override def decimalMethod: String = "$plus" override lazy val resolved = childrenResolved && checkInputDataTypes().isSuccess && !DecimalType.isFixed(dataType) @@ -138,6 +174,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic { case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "-" + override def decimalMethod: String = "$minus" override lazy val resolved = childrenResolved && checkInputDataTypes().isSuccess && !DecimalType.isFixed(dataType) @@ -152,6 +189,7 @@ case class Subtract(left: Expression, right: Expression) extends BinaryArithmeti case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "*" + override def decimalMethod: String = "$times" override lazy val resolved = childrenResolved && checkInputDataTypes().isSuccess && !DecimalType.isFixed(dataType) @@ -166,6 +204,8 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "/" + override def decimalMethod: String = "$divide" + override def nullable: Boolean = true override lazy val resolved = @@ -192,10 +232,40 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic } } } + + /** + * Special case handling due to division by 0 => null. + */ + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + val test = if (left.dataType.isInstanceOf[DecimalType]) { + s"${eval2.primitive}.isZero()" + } else { + s"${eval2.primitive} == 0" + } + val method = if (left.dataType.isInstanceOf[DecimalType]) { + s".$decimalMethod" + } else { + s"$symbol" + } + eval1.code + eval2.code + + s""" + boolean ${ev.isNull} = false; + ${ctx.javaType(left.dataType)} ${ev.primitive} = ${ctx.defaultValue(left.dataType)}; + if (${eval1.isNull} || ${eval2.isNull} || $test) { + ${ev.isNull} = true; + } else { + ${ev.primitive} = ${eval1.primitive}$method(${eval2.primitive}); + } + """ + } } case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "%" + override def decimalMethod: String = "reminder" + override def nullable: Boolean = true override lazy val resolved = @@ -222,6 +292,34 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet } } } + + /** + * Special case handling for x % 0 ==> null. + */ + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + val test = if (left.dataType.isInstanceOf[DecimalType]) { + s"${eval2.primitive}.isZero()" + } else { + s"${eval2.primitive} == 0" + } + val method = if (left.dataType.isInstanceOf[DecimalType]) { + s".$decimalMethod" + } else { + s"$symbol" + } + eval1.code + eval2.code + + s""" + boolean ${ev.isNull} = false; + ${ctx.javaType(left.dataType)} ${ev.primitive} = ${ctx.defaultValue(left.dataType)}; + if (${eval1.isNull} || ${eval2.isNull} || $test) { + ${ev.isNull} = true; + } else { + ${ev.primitive} = ${eval1.primitive}$method(${eval2.primitive}); + } + """ + } } /** @@ -271,7 +369,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet } /** - * A function that calculates bitwise xor(^) of two numbers. + * A function that calculates bitwise xor of two numbers. */ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "^" @@ -313,6 +411,10 @@ case class BitwiseNot(child: Expression) extends UnaryArithmetic { ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any] } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)})~($c)") + } + protected override def evalInternal(evalE: Any) = not(evalE) } @@ -340,6 +442,33 @@ case class MaxOf(left: Expression, right: Expression) extends BinaryArithmetic { } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + if (ctx.isNativeType(left.dataType)) { + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + eval1.code + eval2.code + s""" + boolean ${ev.isNull} = false; + ${ctx.javaType(left.dataType)} ${ev.primitive} = + ${ctx.defaultValue(left.dataType)}; + + if (${eval1.isNull}) { + ${ev.isNull} = ${eval2.isNull}; + ${ev.primitive} = ${eval2.primitive}; + } else if (${eval2.isNull}) { + ${ev.isNull} = ${eval1.isNull}; + ${ev.primitive} = ${eval1.primitive}; + } else { + if (${eval1.primitive} > ${eval2.primitive}) { + ${ev.primitive} = ${eval1.primitive}; + } else { + ${ev.primitive} = ${eval2.primitive}; + } + } + """ + } else { + super.genCode(ctx, ev) + } + } override def toString: String = s"MaxOf($left, $right)" } @@ -367,5 +496,35 @@ case class MinOf(left: Expression, right: Expression) extends BinaryArithmetic { } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + if (ctx.isNativeType(left.dataType)) { + + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + + eval1.code + eval2.code + s""" + boolean ${ev.isNull} = false; + ${ctx.javaType(left.dataType)} ${ev.primitive} = + ${ctx.defaultValue(left.dataType)}; + + if (${eval1.isNull}) { + ${ev.isNull} = ${eval2.isNull}; + ${ev.primitive} = ${eval2.primitive}; + } else if (${eval2.isNull}) { + ${ev.isNull} = ${eval1.isNull}; + ${ev.primitive} = ${eval1.primitive}; + } else { + if (${eval1.primitive} < ${eval2.primitive}) { + ${ev.primitive} = ${eval1.primitive}; + } else { + ${ev.primitive} = ${eval2.primitive}; + } + } + """ + } else { + super.genCode(ctx, ev) + } + } + override def toString: String = s"MinOf($left, $right)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index cd604121b7dd9..c8d0aaf79f5f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -24,7 +24,6 @@ import com.google.common.cache.{CacheBuilder, CacheLoader} import org.codehaus.janino.ClassBodyEvaluator import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -32,6 +31,157 @@ import org.apache.spark.sql.types._ class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int] class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long] +/** + * Java source for evaluating an [[Expression]] given a [[Row]] of input. + * + * @param code The sequence of statements required to evaluate the expression. + * @param isNull A term that holds a boolean value representing whether the expression evaluated + * to null. + * @param primitive A term for a possible primitive value of the result of the evaluation. Not + * valid if `isNull` is set to `true`. + */ +case class GeneratedExpressionCode(var code: Code, var isNull: Term, var primitive: Term) + +/** + * A context for codegen, which is used to bookkeeping the expressions those are not supported + * by codegen, then they are evaluated directly. The unsupported expression is appended at the + * end of `references`, the position of it is kept in the code, used to access and evaluate it. + */ +class CodeGenContext { + + /** + * Holding all the expressions those do not support codegen, will be evaluated directly. + */ + val references: mutable.ArrayBuffer[Expression] = new mutable.ArrayBuffer[Expression]() + + val stringType: String = classOf[UTF8String].getName + val decimalType: String = classOf[Decimal].getName + + private val curId = new java.util.concurrent.atomic.AtomicInteger() + + /** + * Returns a term name that is unique within this instance of a `CodeGenerator`. + * + * (Since we aren't in a macro context we do not seem to have access to the built in `freshName` + * function.) + */ + def freshName(prefix: String): Term = { + s"$prefix${curId.getAndIncrement}" + } + + /** + * Return the code to access a column for given DataType + */ + def getColumn(dataType: DataType, ordinal: Int): Code = { + if (isNativeType(dataType)) { + s"i.${accessorForType(dataType)}($ordinal)" + } else { + s"(${boxedType(dataType)})i.apply($ordinal)" + } + } + + /** + * Return the code to update a column in Row for given DataType + */ + def setColumn(dataType: DataType, ordinal: Int, value: Term): Code = { + if (isNativeType(dataType)) { + s"${mutatorForType(dataType)}($ordinal, $value)" + } else { + s"update($ordinal, $value)" + } + } + + /** + * Return the name of accessor in Row for a DataType + */ + def accessorForType(dt: DataType): Term = dt match { + case IntegerType => "getInt" + case other => s"get${boxedType(dt)}" + } + + /** + * Return the name of mutator in Row for a DataType + */ + def mutatorForType(dt: DataType): Term = dt match { + case IntegerType => "setInt" + case other => s"set${boxedType(dt)}" + } + + /** + * Return the Java type for a DataType + */ + def javaType(dt: DataType): Term = dt match { + case IntegerType => "int" + case LongType => "long" + case ShortType => "short" + case ByteType => "byte" + case DoubleType => "double" + case FloatType => "float" + case BooleanType => "boolean" + case dt: DecimalType => decimalType + case BinaryType => "byte[]" + case StringType => stringType + case DateType => "int" + case TimestampType => "java.sql.Timestamp" + case dt: OpenHashSetUDT if dt.elementType == IntegerType => classOf[IntegerHashSet].getName + case dt: OpenHashSetUDT if dt.elementType == LongType => classOf[LongHashSet].getName + case _ => "Object" + } + + /** + * Return the boxed type in Java + */ + def boxedType(dt: DataType): Term = dt match { + case IntegerType => "Integer" + case LongType => "Long" + case ShortType => "Short" + case ByteType => "Byte" + case DoubleType => "Double" + case FloatType => "Float" + case BooleanType => "Boolean" + case DateType => "Integer" + case _ => javaType(dt) + } + + /** + * Return the representation of default value for given DataType + */ + def defaultValue(dt: DataType): Term = dt match { + case BooleanType => "false" + case FloatType => "-1.0f" + case ShortType => "(short)-1" + case LongType => "-1L" + case ByteType => "(byte)-1" + case DoubleType => "-1.0" + case IntegerType => "-1" + case DateType => "-1" + case _ => "null" + } + + /** + * Returns a function to generate equal expression in Java + */ + def equalFunc(dataType: DataType): ((Term, Term) => Code) = dataType match { + case BinaryType => { case (eval1, eval2) => + s"java.util.Arrays.equals($eval1, $eval2)" } + case IntegerType | BooleanType | LongType | DoubleType | FloatType | ShortType | ByteType => + { case (eval1, eval2) => s"$eval1 == $eval2" } + case other => + { case (eval1, eval2) => s"$eval1.equals($eval2)" } + } + + /** + * List of data types that have special accessors and setters in [[Row]]. + */ + val nativeTypes = + Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType) + + /** + * Returns true if the data type has a special accessor and setter in [[Row]]. + */ + def isNativeType(dt: DataType): Boolean = nativeTypes.contains(dt) +} + /** * A base class for generators of byte code to perform expression evaluation. Includes a set of * helpers for referring to Catalyst types and building trees that perform evaluation of individual @@ -39,14 +189,9 @@ class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long] */ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Logging { - protected val rowType = classOf[Row].getName - protected val stringType = classOf[UTF8String].getName - protected val decimalType = classOf[Decimal].getName - protected val exprType = classOf[Expression].getName - protected val mutableRowType = classOf[MutableRow].getName - protected val genericMutableRowType = classOf[GenericMutableRow].getName - - private val curId = new java.util.concurrent.atomic.AtomicInteger() + protected val exprType: String = classOf[Expression].getName + protected val mutableRowType: String = classOf[MutableRow].getName + protected val genericMutableRowType: String = classOf[GenericMutableRow].getName /** * Can be flipped on manually in the console to add (expensive) expression evaluation trace code. @@ -75,10 +220,16 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin */ protected def compile(code: String): Class[_] = { val startTime = System.nanoTime() - val clazz = new ClassBodyEvaluator(code).getClazz() + val clazz = try { + new ClassBodyEvaluator(code).getClazz() + } catch { + case e: Exception => + logError(s"failed to compile:\n $code", e) + throw e + } val endTime = System.nanoTime() def timeMs: Double = (endTime - startTime).toDouble / 1000000 - logDebug(s"Compiled Java code (${code.size} bytes) in $timeMs ms") + logDebug(s"Code (${code.size} bytes) compiled in $timeMs ms") clazz } @@ -112,586 +263,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin /** Generates the requested evaluator given already bound expression(s). */ def generate(expressions: InType): OutType = cache.get(canonicalize(expressions)) - /** - * Returns a term name that is unique within this instance of a `CodeGenerator`. - * - * (Since we aren't in a macro context we do not seem to have access to the built in `freshName` - * function.) - */ - protected def freshName(prefix: String): String = { - s"$prefix${curId.getAndIncrement}" - } - - /** - * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input. - * - * @param code The sequence of statements required to evaluate the expression. - * @param nullTerm A term that holds a boolean value representing whether the expression evaluated - * to null. - * @param primitiveTerm A term for a possible primitive value of the result of the evaluation. Not - * valid if `nullTerm` is set to `true`. - * @param objectTerm A possibly boxed version of the result of evaluating this expression. - */ - protected case class EvaluatedExpression( - code: String, - nullTerm: String, - primitiveTerm: String, - objectTerm: String) - - /** - * A context for codegen, which is used to bookkeeping the expressions those are not supported - * by codegen, then they are evaluated directly. The unsupported expression is appended at the - * end of `references`, the position of it is kept in the code, used to access and evaluate it. - */ - protected class CodeGenContext { - /** - * Holding all the expressions those do not support codegen, will be evaluated directly. - */ - val references: mutable.ArrayBuffer[Expression] = new mutable.ArrayBuffer[Expression]() - } - /** * Create a new codegen context for expression evaluator, used to store those * expressions that don't support codegen */ def newCodeGenContext(): CodeGenContext = { - new CodeGenContext() - } - - /** - * Given an expression tree returns an [[EvaluatedExpression]], which contains Scala trees that - * can be used to determine the result of evaluating the expression on an input row. - */ - def expressionEvaluator(e: Expression, ctx: CodeGenContext): EvaluatedExpression = { - val primitiveTerm = freshName("primitiveTerm") - val nullTerm = freshName("nullTerm") - val objectTerm = freshName("objectTerm") - - implicit class Evaluate1(e: Expression) { - def castOrNull(f: String => String, dataType: DataType): String = { - val eval = expressionEvaluator(e, ctx) - eval.code + - s""" - boolean $nullTerm = ${eval.nullTerm}; - ${primitiveForType(dataType)} $primitiveTerm = ${defaultPrimitive(dataType)}; - if (!$nullTerm) { - $primitiveTerm = ${f(eval.primitiveTerm)}; - } - """ - } - } - - implicit class Evaluate2(expressions: (Expression, Expression)) { - - /** - * Short hand for generating binary evaluation code, which depends on two sub-evaluations of - * the same type. If either of the sub-expressions is null, the result of this computation - * is assumed to be null. - * - * @param f a function from two primitive term names to a tree that evaluates them. - */ - def evaluate(f: (String, String) => String): String = - evaluateAs(expressions._1.dataType)(f) - - def evaluateAs(resultType: DataType)(f: (String, String) => String): String = { - // TODO: Right now some timestamp tests fail if we enforce this... - if (expressions._1.dataType != expressions._2.dataType) { - log.warn(s"${expressions._1.dataType} != ${expressions._2.dataType}") - } - - val eval1 = expressionEvaluator(expressions._1, ctx) - val eval2 = expressionEvaluator(expressions._2, ctx) - val resultCode = f(eval1.primitiveTerm, eval2.primitiveTerm) - - eval1.code + eval2.code + - s""" - boolean $nullTerm = ${eval1.nullTerm} || ${eval2.nullTerm}; - ${primitiveForType(resultType)} $primitiveTerm = ${defaultPrimitive(resultType)}; - if(!$nullTerm) { - $primitiveTerm = (${primitiveForType(resultType)})($resultCode); - } - """ - } - } - - val inputTuple = "i" - - // TODO: Skip generation of null handling code when expression are not nullable. - val primitiveEvaluation: PartialFunction[Expression, String] = { - case b @ BoundReference(ordinal, dataType, nullable) => - s""" - final boolean $nullTerm = $inputTuple.isNullAt($ordinal); - final ${primitiveForType(dataType)} $primitiveTerm = $nullTerm ? - ${defaultPrimitive(dataType)} : (${getColumn(inputTuple, dataType, ordinal)}); - """ - - case expressions.Literal(null, dataType) => - s""" - final boolean $nullTerm = true; - ${primitiveForType(dataType)} $primitiveTerm = ${defaultPrimitive(dataType)}; - """ - - case expressions.Literal(value: UTF8String, StringType) => - val arr = s"new byte[]{${value.getBytes.map(_.toString).mkString(", ")}}" - s""" - final boolean $nullTerm = false; - ${stringType} $primitiveTerm = - new ${stringType}().set(${arr}); - """ - - case expressions.Literal(value, FloatType) => - s""" - final boolean $nullTerm = false; - float $primitiveTerm = ${value}f; - """ - - case expressions.Literal(value, dt @ DecimalType()) => - s""" - final boolean $nullTerm = false; - ${primitiveForType(dt)} $primitiveTerm = new ${primitiveForType(dt)}().set($value); - """ - - case expressions.Literal(value, dataType) => - s""" - final boolean $nullTerm = false; - ${primitiveForType(dataType)} $primitiveTerm = $value; - """ - - case Cast(child @ BinaryType(), StringType) => - child.castOrNull(c => - s"new ${stringType}().set($c)", - StringType) - - case Cast(child @ DateType(), StringType) => - child.castOrNull(c => - s"""new ${stringType}().set( - org.apache.spark.sql.catalyst.util.DateUtils.toString($c))""", - StringType) - - case Cast(child @ BooleanType(), dt: NumericType) if !dt.isInstanceOf[DecimalType] => - child.castOrNull(c => s"(${primitiveForType(dt)})($c?1:0)", dt) - - case Cast(child @ DecimalType(), IntegerType) => - child.castOrNull(c => s"($c).toInt()", IntegerType) - - case Cast(child @ DecimalType(), dt: NumericType) if !dt.isInstanceOf[DecimalType] => - child.castOrNull(c => s"($c).to${termForType(dt)}()", dt) - - case Cast(child @ NumericType(), dt: NumericType) if !dt.isInstanceOf[DecimalType] => - child.castOrNull(c => s"(${primitiveForType(dt)})($c)", dt) - - // Special handling required for timestamps in hive test cases since the toString function - // does not match the expected output. - case Cast(e, StringType) if e.dataType != TimestampType => - e.castOrNull(c => - s"new ${stringType}().set(String.valueOf($c))", - StringType) - - case EqualTo(e1 @ BinaryType(), e2 @ BinaryType()) => - (e1, e2).evaluateAs (BooleanType) { - case (eval1, eval2) => - s"java.util.Arrays.equals((byte[])$eval1, (byte[])$eval2)" - } - - case EqualTo(e1, e2) => - (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => s"$eval1 == $eval2" } - - case GreaterThan(e1 @ NumericType(), e2 @ NumericType()) => - (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => s"$eval1 > $eval2" } - case GreaterThanOrEqual(e1 @ NumericType(), e2 @ NumericType()) => - (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => s"$eval1 >= $eval2" } - case LessThan(e1 @ NumericType(), e2 @ NumericType()) => - (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => s"$eval1 < $eval2" } - case LessThanOrEqual(e1 @ NumericType(), e2 @ NumericType()) => - (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => s"$eval1 <= $eval2" } - - case And(e1, e2) => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - s""" - ${eval1.code} - boolean $nullTerm = false; - boolean $primitiveTerm = false; - - if (!${eval1.nullTerm} && !${eval1.primitiveTerm}) { - } else { - ${eval2.code} - if (!${eval2.nullTerm} && !${eval2.primitiveTerm}) { - } else if (!${eval1.nullTerm} && !${eval2.nullTerm}) { - $primitiveTerm = true; - } else { - $nullTerm = true; - } - } - """ - - case Or(e1, e2) => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - - s""" - ${eval1.code} - boolean $nullTerm = false; - boolean $primitiveTerm = false; - - if (!${eval1.nullTerm} && ${eval1.primitiveTerm}) { - $primitiveTerm = true; - } else { - ${eval2.code} - if (!${eval2.nullTerm} && ${eval2.primitiveTerm}) { - $primitiveTerm = true; - } else if (!${eval1.nullTerm} && !${eval2.nullTerm}) { - $primitiveTerm = false; - } else { - $nullTerm = true; - } - } - """ - - case Not(child) => - // Uh, bad function name... - child.castOrNull(c => s"!$c", BooleanType) - - case Add(e1 @ DecimalType(), e2 @ DecimalType()) => - (e1, e2) evaluate { case (eval1, eval2) => s"$eval1.$$plus($eval2)" } - case Subtract(e1 @ DecimalType(), e2 @ DecimalType()) => - (e1, e2) evaluate { case (eval1, eval2) => s"$eval1.$$minus($eval2)" } - case Multiply(e1 @ DecimalType(), e2 @ DecimalType()) => - (e1, e2) evaluate { case (eval1, eval2) => s"$eval1.$$times($eval2)" } - case Divide(e1 @ DecimalType(), e2 @ DecimalType()) => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - eval1.code + eval2.code + - s""" - boolean $nullTerm = false; - ${primitiveForType(e1.dataType)} $primitiveTerm = null; - if (${eval1.nullTerm} || ${eval2.nullTerm} || ${eval2.primitiveTerm}.isZero()) { - $nullTerm = true; - } else { - $primitiveTerm = ${eval1.primitiveTerm}.$$div${eval2.primitiveTerm}); - } - """ - case Remainder(e1 @ DecimalType(), e2 @ DecimalType()) => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - eval1.code + eval2.code + - s""" - boolean $nullTerm = false; - ${primitiveForType(e1.dataType)} $primitiveTerm = 0; - if (${eval1.nullTerm} || ${eval2.nullTerm} || ${eval2.primitiveTerm}.isZero()) { - $nullTerm = true; - } else { - $primitiveTerm = ${eval1.primitiveTerm}.remainder(${eval2.primitiveTerm}); - } - """ - - case Add(e1, e2) => - (e1, e2) evaluate { case (eval1, eval2) => s"$eval1 + $eval2" } - case Subtract(e1, e2) => - (e1, e2) evaluate { case (eval1, eval2) => s"$eval1 - $eval2" } - case Multiply(e1, e2) => - (e1, e2) evaluate { case (eval1, eval2) => s"$eval1 * $eval2" } - case Divide(e1, e2) => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - eval1.code + eval2.code + - s""" - boolean $nullTerm = false; - ${primitiveForType(e1.dataType)} $primitiveTerm = 0; - if (${eval1.nullTerm} || ${eval2.nullTerm} || ${eval2.primitiveTerm} == 0) { - $nullTerm = true; - } else { - $primitiveTerm = ${eval1.primitiveTerm} / ${eval2.primitiveTerm}; - } - """ - case Remainder(e1, e2) => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - eval1.code + eval2.code + - s""" - boolean $nullTerm = false; - ${primitiveForType(e1.dataType)} $primitiveTerm = 0; - if (${eval1.nullTerm} || ${eval2.nullTerm} || ${eval2.primitiveTerm} == 0) { - $nullTerm = true; - } else { - $primitiveTerm = ${eval1.primitiveTerm} % ${eval2.primitiveTerm}; - } - """ - - case IsNotNull(e) => - val eval = expressionEvaluator(e, ctx) - s""" - ${eval.code} - boolean $nullTerm = false; - boolean $primitiveTerm = !${eval.nullTerm}; - """ - - case IsNull(e) => - val eval = expressionEvaluator(e, ctx) - s""" - ${eval.code} - boolean $nullTerm = false; - boolean $primitiveTerm = ${eval.nullTerm}; - """ - - case e @ Coalesce(children) => - s""" - boolean $nullTerm = true; - ${primitiveForType(e.dataType)} $primitiveTerm = ${defaultPrimitive(e.dataType)}; - """ + - children.map { c => - val eval = expressionEvaluator(c, ctx) - s""" - if($nullTerm) { - ${eval.code} - if(!${eval.nullTerm}) { - $nullTerm = false; - $primitiveTerm = ${eval.primitiveTerm}; - } - } - """ - }.mkString("\n") - - case e @ expressions.If(condition, trueValue, falseValue) => - val condEval = expressionEvaluator(condition, ctx) - val trueEval = expressionEvaluator(trueValue, ctx) - val falseEval = expressionEvaluator(falseValue, ctx) - - s""" - boolean $nullTerm = false; - ${primitiveForType(e.dataType)} $primitiveTerm = ${defaultPrimitive(e.dataType)}; - ${condEval.code} - if(!${condEval.nullTerm} && ${condEval.primitiveTerm}) { - ${trueEval.code} - $nullTerm = ${trueEval.nullTerm}; - $primitiveTerm = ${trueEval.primitiveTerm}; - } else { - ${falseEval.code} - $nullTerm = ${falseEval.nullTerm}; - $primitiveTerm = ${falseEval.primitiveTerm}; - } - """ - - case NewSet(elementType) => - s""" - boolean $nullTerm = false; - ${hashSetForType(elementType)} $primitiveTerm = new ${hashSetForType(elementType)}(); - """ - - case AddItemToSet(item, set) => - val itemEval = expressionEvaluator(item, ctx) - val setEval = expressionEvaluator(set, ctx) - - val elementType = set.dataType.asInstanceOf[OpenHashSetUDT].elementType - val htype = hashSetForType(elementType) - - itemEval.code + setEval.code + - s""" - if (!${itemEval.nullTerm} && !${setEval.nullTerm}) { - (($htype)${setEval.primitiveTerm}).add(${itemEval.primitiveTerm}); - } - boolean $nullTerm = false; - ${htype} $primitiveTerm = ($htype)${setEval.primitiveTerm}; - """ - - case CombineSets(left, right) => - val leftEval = expressionEvaluator(left, ctx) - val rightEval = expressionEvaluator(right, ctx) - - val elementType = left.dataType.asInstanceOf[OpenHashSetUDT].elementType - val htype = hashSetForType(elementType) - - leftEval.code + rightEval.code + - s""" - boolean $nullTerm = false; - ${htype} $primitiveTerm = - (${htype})${leftEval.primitiveTerm}; - $primitiveTerm.union((${htype})${rightEval.primitiveTerm}); - """ - - case MaxOf(e1, e2) if !e1.dataType.isInstanceOf[DecimalType] => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - - eval1.code + eval2.code + - s""" - boolean $nullTerm = false; - ${primitiveForType(e1.dataType)} $primitiveTerm = ${defaultPrimitive(e1.dataType)}; - - if (${eval1.nullTerm}) { - $nullTerm = ${eval2.nullTerm}; - $primitiveTerm = ${eval2.primitiveTerm}; - } else if (${eval2.nullTerm}) { - $nullTerm = ${eval1.nullTerm}; - $primitiveTerm = ${eval1.primitiveTerm}; - } else { - if (${eval1.primitiveTerm} > ${eval2.primitiveTerm}) { - $primitiveTerm = ${eval1.primitiveTerm}; - } else { - $primitiveTerm = ${eval2.primitiveTerm}; - } - } - """ - - case MinOf(e1, e2) if !e1.dataType.isInstanceOf[DecimalType] => - val eval1 = expressionEvaluator(e1, ctx) - val eval2 = expressionEvaluator(e2, ctx) - - eval1.code + eval2.code + - s""" - boolean $nullTerm = false; - ${primitiveForType(e1.dataType)} $primitiveTerm = ${defaultPrimitive(e1.dataType)}; - - if (${eval1.nullTerm}) { - $nullTerm = ${eval2.nullTerm}; - $primitiveTerm = ${eval2.primitiveTerm}; - } else if (${eval2.nullTerm}) { - $nullTerm = ${eval1.nullTerm}; - $primitiveTerm = ${eval1.primitiveTerm}; - } else { - if (${eval1.primitiveTerm} < ${eval2.primitiveTerm}) { - $primitiveTerm = ${eval1.primitiveTerm}; - } else { - $primitiveTerm = ${eval2.primitiveTerm}; - } - } - """ - - case UnscaledValue(child) => - val childEval = expressionEvaluator(child, ctx) - - childEval.code + - s""" - boolean $nullTerm = ${childEval.nullTerm}; - long $primitiveTerm = $nullTerm ? -1 : ${childEval.primitiveTerm}.toUnscaledLong(); - """ - - case MakeDecimal(child, precision, scale) => - val eval = expressionEvaluator(child, ctx) - - eval.code + - s""" - boolean $nullTerm = ${eval.nullTerm}; - org.apache.spark.sql.types.Decimal $primitiveTerm = ${defaultPrimitive(DecimalType())}; - - if (!$nullTerm) { - $primitiveTerm = new org.apache.spark.sql.types.Decimal(); - $primitiveTerm = $primitiveTerm.setOrNull(${eval.primitiveTerm}, $precision, $scale); - $nullTerm = $primitiveTerm == null; - } - """ - } - - // If there was no match in the partial function above, we fall back on calling the interpreted - // expression evaluator. - val code: String = - primitiveEvaluation.lift.apply(e).getOrElse { - logError(s"No rules to generate $e") - ctx.references += e - s""" - /* expression: ${e} */ - Object $objectTerm = expressions[${ctx.references.size - 1}].eval(i); - boolean $nullTerm = $objectTerm == null; - ${primitiveForType(e.dataType)} $primitiveTerm = ${defaultPrimitive(e.dataType)}; - if (!$nullTerm) $primitiveTerm = (${termForType(e.dataType)})$objectTerm; - """ - } - - EvaluatedExpression(code, nullTerm, primitiveTerm, objectTerm) - } - - protected def getColumn(inputRow: String, dataType: DataType, ordinal: Int) = { - dataType match { - case StringType => s"(${stringType})$inputRow.apply($ordinal)" - case dt: DataType if isNativeType(dt) => s"$inputRow.${accessorForType(dt)}($ordinal)" - case _ => s"(${termForType(dataType)})$inputRow.apply($ordinal)" - } - } - - protected def setColumn( - destinationRow: String, - dataType: DataType, - ordinal: Int, - value: String): String = { - dataType match { - case StringType => s"$destinationRow.update($ordinal, $value)" - case dt: DataType if isNativeType(dt) => - s"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" - case _ => s"$destinationRow.update($ordinal, $value)" - } - } - - protected def accessorForType(dt: DataType) = dt match { - case IntegerType => "getInt" - case other => s"get${termForType(dt)}" - } - - protected def mutatorForType(dt: DataType) = dt match { - case IntegerType => "setInt" - case other => s"set${termForType(dt)}" - } - - protected def hashSetForType(dt: DataType): String = dt match { - case IntegerType => classOf[IntegerHashSet].getName - case LongType => classOf[LongHashSet].getName - case unsupportedType => - sys.error(s"Code generation not support for hashset of type $unsupportedType") + new CodeGenContext } - - protected def primitiveForType(dt: DataType): String = dt match { - case IntegerType => "int" - case LongType => "long" - case ShortType => "short" - case ByteType => "byte" - case DoubleType => "double" - case FloatType => "float" - case BooleanType => "boolean" - case dt: DecimalType => decimalType - case BinaryType => "byte[]" - case StringType => stringType - case DateType => "int" - case TimestampType => "java.sql.Timestamp" - case _ => "Object" - } - - protected def defaultPrimitive(dt: DataType): String = dt match { - case BooleanType => "false" - case FloatType => "-1.0f" - case ShortType => "-1" - case LongType => "-1" - case ByteType => "-1" - case DoubleType => "-1.0" - case IntegerType => "-1" - case DateType => "-1" - case dt: DecimalType => "null" - case StringType => "null" - case _ => "null" - } - - protected def termForType(dt: DataType): String = dt match { - case IntegerType => "Integer" - case LongType => "Long" - case ShortType => "Short" - case ByteType => "Byte" - case DoubleType => "Double" - case FloatType => "Float" - case BooleanType => "Boolean" - case dt: DecimalType => decimalType - case BinaryType => "byte[]" - case StringType => stringType - case DateType => "Integer" - case TimestampType => "java.sql.Timestamp" - case _ => "Object" - } - - /** - * List of data types that have special accessors and setters in [[Row]]. - */ - protected val nativeTypes = - Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType) - - /** - * Returns true if the data type has a special accessor and setter in [[Row]]. - */ - protected def isNativeType(dt: DataType) = nativeTypes.contains(dt) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 638b53fe0fe2f..e5ee2accd8a84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -37,13 +37,13 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu protected def create(expressions: Seq[Expression]): (() => MutableProjection) = { val ctx = newCodeGenContext() val projectionCode = expressions.zipWithIndex.map { case (e, i) => - val evaluationCode = expressionEvaluator(e, ctx) + val evaluationCode = e.gen(ctx) evaluationCode.code + s""" - if(${evaluationCode.nullTerm}) + if(${evaluationCode.isNull}) mutableRow.setNullAt($i); else - ${setColumn("mutableRow", e.dataType, i, evaluationCode.primitiveTerm)}; + mutableRow.${ctx.setColumn(e.dataType, i, evaluationCode.primitive)}; """ }.mkString("\n") val code = s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 0ff840dab393c..36e155d164a40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -52,15 +52,15 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit val ctx = newCodeGenContext() val comparisons = ordering.zipWithIndex.map { case (order, i) => - val evalA = expressionEvaluator(order.child, ctx) - val evalB = expressionEvaluator(order.child, ctx) + val evalA = order.child.gen(ctx) + val evalB = order.child.gen(ctx) val asc = order.direction == Ascending val compare = order.child.dataType match { case BinaryType => s""" { - byte[] x = ${if (asc) evalA.primitiveTerm else evalB.primitiveTerm}; - byte[] y = ${if (!asc) evalB.primitiveTerm else evalA.primitiveTerm}; + byte[] x = ${if (asc) evalA.primitive else evalB.primitive}; + byte[] y = ${if (!asc) evalB.primitive else evalA.primitive}; int j = 0; while (j < x.length && j < y.length) { if (x[j] != y[j]) return x[j] - y[j]; @@ -73,8 +73,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit }""" case _: NumericType => s""" - if (${evalA.primitiveTerm} != ${evalB.primitiveTerm}) { - if (${evalA.primitiveTerm} > ${evalB.primitiveTerm}) { + if (${evalA.primitive} != ${evalB.primitive}) { + if (${evalA.primitive} > ${evalB.primitive}) { return ${if (asc) "1" else "-1"}; } else { return ${if (asc) "-1" else "1"}; @@ -82,7 +82,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit }""" case _ => s""" - int comp = ${evalA.primitiveTerm}.compare(${evalB.primitiveTerm}); + int comp = ${evalA.primitive}.compare(${evalB.primitive}); if (comp != 0) { return ${if (asc) "comp" else "-comp"}; }""" @@ -93,11 +93,11 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit ${evalA.code} i = $b; ${evalB.code} - if (${evalA.nullTerm} && ${evalB.nullTerm}) { + if (${evalA.isNull} && ${evalB.isNull}) { // Nothing - } else if (${evalA.nullTerm}) { + } else if (${evalA.isNull}) { return ${if (order.direction == Ascending) "-1" else "1"}; - } else if (${evalB.nullTerm}) { + } else if (${evalB.isNull}) { return ${if (order.direction == Ascending) "1" else "-1"}; } else { $compare diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index fb18769f00da3..4a547b5ce9543 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -38,7 +38,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] { protected def create(predicate: Expression): ((Row) => Boolean) = { val ctx = newCodeGenContext() - val eval = expressionEvaluator(predicate, ctx) + val eval = predicate.gen(ctx) val code = s""" import org.apache.spark.sql.Row; @@ -55,7 +55,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] { @Override public boolean eval(Row i) { ${eval.code} - return !${eval.nullTerm} && ${eval.primitiveTerm}; + return !${eval.isNull} && ${eval.primitive}; } }""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index d5be1fc12e0f0..7caf4aaab88bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -45,19 +45,19 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { val ctx = newCodeGenContext() val columns = expressions.zipWithIndex.map { case (e, i) => - s"private ${primitiveForType(e.dataType)} c$i = ${defaultPrimitive(e.dataType)};\n" + s"private ${ctx.javaType(e.dataType)} c$i = ${ctx.defaultValue(e.dataType)};\n" }.mkString("\n ") val initColumns = expressions.zipWithIndex.map { case (e, i) => - val eval = expressionEvaluator(e, ctx) + val eval = e.gen(ctx) s""" { // column$i ${eval.code} - nullBits[$i] = ${eval.nullTerm}; - if(!${eval.nullTerm}) { - c$i = ${eval.primitiveTerm}; + nullBits[$i] = ${eval.isNull}; + if (!${eval.isNull}) { + c$i = ${eval.primitive}; } } """ @@ -68,10 +68,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { }.mkString("\n ") val updateCases = expressions.zipWithIndex.map { case (e, i) => - s"case $i: { c$i = (${termForType(e.dataType)})value; return;}" + s"case $i: { c$i = (${ctx.boxedType(e.dataType)})value; return;}" }.mkString("\n ") - val specificAccessorFunctions = nativeTypes.map { dataType => + val specificAccessorFunctions = ctx.nativeTypes.map { dataType => val cases = expressions.zipWithIndex.map { case (e, i) if e.dataType == dataType => s"case $i: return c$i;" @@ -80,21 +80,21 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { if (cases.count(_ != '\n') > 0) { s""" @Override - public ${primitiveForType(dataType)} ${accessorForType(dataType)}(int i) { + public ${ctx.javaType(dataType)} ${ctx.accessorForType(dataType)}(int i) { if (isNullAt(i)) { - return ${defaultPrimitive(dataType)}; + return ${ctx.defaultValue(dataType)}; } switch (i) { $cases } - return ${defaultPrimitive(dataType)}; + return ${ctx.defaultValue(dataType)}; }""" } else { "" } }.mkString("\n") - val specificMutatorFunctions = nativeTypes.map { dataType => + val specificMutatorFunctions = ctx.nativeTypes.map { dataType => val cases = expressions.zipWithIndex.map { case (e, i) if e.dataType == dataType => s"case $i: { c$i = value; return; }" @@ -103,7 +103,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { if (cases.count(_ != '\n') > 0) { s""" @Override - public void ${mutatorForType(dataType)}(int i, ${primitiveForType(dataType)} value) { + public void ${ctx.mutatorForType(dataType)}(int i, ${ctx.javaType(dataType)} value) { nullBits[i] = false; switch (i) { $cases @@ -122,7 +122,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { case LongType => s"$col ^ ($col >>> 32)" case FloatType => s"Float.floatToIntBits($col)" case DoubleType => - s"Double.doubleToLongBits($col) ^ (Double.doubleToLongBits($col) >>> 32)" + s"(int)(Double.doubleToLongBits($col) ^ (Double.doubleToLongBits($col) >>> 32))" case _ => s"$col.hashCode()" } s"isNullAt($i) ? 0 : ($nonNull)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala index 7f1b12cdd5800..6f9589d20445e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala @@ -27,6 +27,9 @@ import org.apache.spark.util.Utils */ package object codegen { + type Term = String + type Code = String + /** Canonicalizes an expression so those that differ only by names can reuse the same code. */ object ExpressionCanonicalizer extends rules.RuleExecutor[Expression] { val batches = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index 65ba18924afe1..ddfadf314f838 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext} import org.apache.spark.sql.types._ /** Return the unscaled Long value of a Decimal, assuming it fits in a Long */ @@ -35,6 +36,10 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { childResult.asInstanceOf[Decimal].toUnscaledLong } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, c => s"$c.toUnscaledLong()") + } } /** Create a Decimal from an unscaled Long value */ @@ -53,4 +58,18 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un new Decimal().setOrNull(childResult.asInstanceOf[Long], precision, scale) } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval = child.gen(ctx) + eval.code + s""" + boolean ${ev.isNull} = ${eval.isNull}; + ${ctx.decimalType} ${ev.primitive} = null; + + if (!${ev.isNull}) { + ${ev.primitive} = (new ${ctx.decimalType}()).setOrNull( + ${eval.primitive}, $precision, $scale); + ${ev.isNull} = ${ev.primitive} == null; + } + """ + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index d3ca3d9a4b18b..3a9271678bc9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.codegen.{Code, CodeGenContext, GeneratedExpressionCode} import org.apache.spark.sql.catalyst.util.DateUtils import org.apache.spark.sql.types._ @@ -78,7 +79,60 @@ case class Literal protected (value: Any, dataType: DataType) extends LeafExpres override def toString: String = if (value != null) value.toString else "null" + override def equals(other: Any): Boolean = other match { + case o: Literal => + dataType.equals(o.dataType) && + (value == null && null == o.value || value != null && value.equals(o.value)) + case _ => false + } + override def eval(input: Row): Any = value + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + // change the isNull and primitive to consts, to inline them + if (value == null) { + ev.isNull = "true" + ev.primitive = ctx.defaultValue(dataType) + "" + } else { + dataType match { + case BooleanType => + ev.isNull = "false" + ev.primitive = value.toString + "" + case FloatType => // This must go before NumericType + val v = value.asInstanceOf[Float] + if (v.isNaN || v.isInfinite) { + super.genCode(ctx, ev) + } else { + ev.isNull = "false" + ev.primitive = s"${value}f" + "" + } + case DoubleType => // This must go before NumericType + val v = value.asInstanceOf[Double] + if (v.isNaN || v.isInfinite) { + super.genCode(ctx, ev) + } else { + ev.isNull = "false" + ev.primitive = s"${value}" + "" + } + + case ByteType | ShortType => // This must go before NumericType + ev.isNull = "false" + ev.primitive = s"(${ctx.javaType(dataType)})$value" + "" + case dt: NumericType if !dt.isInstanceOf[DecimalType] => + ev.isNull = "false" + ev.primitive = value.toString + "" + // eval() version may be faster for non-primitive types + case other => + super.genCode(ctx, ev) + } + } + } } // TODO: Specialize diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala index db853a2b97fad..88211acd7713c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.mathfuncs +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, BinaryExpression, Expression, Row} import org.apache.spark.sql.types._ @@ -49,6 +50,10 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) } } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)") + } } case class Atan2(left: Expression, right: Expression) @@ -70,9 +75,26 @@ case class Atan2(left: Expression, right: Expression) } } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") + s""" + if (Double.valueOf(${ev.primitive}).isNaN()) { + ${ev.isNull} = true; + } + """ + } } case class Hypot(left: Expression, right: Expression) extends BinaryMathExpression(math.hypot, "HYPOT") -case class Pow(left: Expression, right: Expression) extends BinaryMathExpression(math.pow, "POWER") +case class Pow(left: Expression, right: Expression) + extends BinaryMathExpression(math.pow, "POWER") { + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") + s""" + if (Double.valueOf(${ev.primitive}).isNaN()) { + ${ev.isNull} = true; + } + """ + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala index 41b422346a02d..5563cd94bf86d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.mathfuncs +import org.apache.spark.sql.catalyst.expressions.codegen.{Code, CodeGenContext, GeneratedExpressionCode} import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, Row, UnaryExpression} import org.apache.spark.sql.types._ @@ -44,6 +45,23 @@ abstract class UnaryMathExpression(f: Double => Double, name: String) if (result.isNaN) null else result } } + + // name of function in java.lang.Math + def funcName: String = name.toLowerCase + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval = child.gen(ctx) + eval.code + s""" + boolean ${ev.isNull} = ${eval.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + ${ev.primitive} = java.lang.Math.${funcName}(${eval.primitive}); + if (Double.valueOf(${ev.primitive}).isNaN()) { + ${ev.isNull} = true; + } + } + """ + } } case class Acos(child: Expression) extends UnaryMathExpression(math.acos, "ACOS") @@ -72,7 +90,9 @@ case class Log10(child: Expression) extends UnaryMathExpression(math.log10, "LOG case class Log1p(child: Expression) extends UnaryMathExpression(math.log1p, "LOG1P") -case class Rint(child: Expression) extends UnaryMathExpression(math.rint, "ROUND") +case class Rint(child: Expression) extends UnaryMathExpression(math.rint, "ROUND") { + override def funcName: String = "rint" +} case class Signum(child: Expression) extends UnaryMathExpression(math.signum, "SIGNUM") @@ -84,6 +104,10 @@ case class Tan(child: Expression) extends UnaryMathExpression(math.tan, "TAN") case class Tanh(child: Expression) extends UnaryMathExpression(math.tanh, "TANH") -case class ToDegrees(child: Expression) extends UnaryMathExpression(math.toDegrees, "DEGREES") +case class ToDegrees(child: Expression) extends UnaryMathExpression(math.toDegrees, "DEGREES") { + override def funcName: String = "toDegrees" +} -case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadians, "RADIANS") +case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadians, "RADIANS") { + override def funcName: String = "toRadians" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 00565ec651a59..2e4b9ba678433 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.trees.LeafNode +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode} +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.types._ object NamedExpression { @@ -116,6 +116,8 @@ case class Alias(child: Expression, name: String)( override def eval(input: Row): Any = child.eval(input) + override def gen(ctx: CodeGenContext): GeneratedExpressionCode = child.gen(ctx) + override def dataType: DataType = child.dataType override def nullable: Boolean = child.nullable override def metadata: Metadata = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 5070570b4740d..9ecfb3ccc262f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext} import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.types.DataType @@ -51,6 +52,25 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } result } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + s""" + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + """ + + children.map { e => + val eval = e.gen(ctx) + s""" + if (${ev.isNull}) { + ${eval.code} + if (!${eval.isNull}) { + ${ev.isNull} = false; + ${ev.primitive} = ${eval.primitive}; + } + } + """ + }.mkString("\n") + } } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { @@ -61,6 +81,13 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr child.eval(input) == null } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval = child.gen(ctx) + ev.isNull = "false" + ev.primitive = eval.isNull + eval.code + } + override def toString: String = s"IS NULL $child" } @@ -72,6 +99,13 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E override def eval(input: Row): Any = { child.eval(input) != null } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval = child.gen(ctx) + ev.isNull = "false" + ev.primitive = s"(!(${eval.isNull}))" + eval.code + } } /** @@ -95,4 +129,25 @@ case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate } numNonNulls >= n } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val nonnull = ctx.freshName("nonnull") + val code = children.map { e => + val eval = e.gen(ctx) + s""" + if ($nonnull < $n) { + ${eval.code} + if (!${eval.isNull}) { + $nonnull += 1; + } + } + """ + }.mkString("\n") + s""" + int $nonnull = 0; + $code + boolean ${ev.isNull} = false; + boolean ${ev.primitive} = $nonnull >= $n; + """ + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 58273b166fe91..1d0f19a400d63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.TypeUtils -import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType} +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ object InterpretedPredicate { def create(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = @@ -82,6 +83,10 @@ case class Not(child: Expression) extends UnaryExpression with Predicate with Ex case b: Boolean => !b } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, c => s"!($c)") + } } /** @@ -141,6 +146,29 @@ case class And(left: Expression, right: Expression) } } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + + // The result should be `false`, if any of them is `false` whenever the other is null or not. + s""" + ${eval1.code} + boolean ${ev.isNull} = false; + boolean ${ev.primitive} = false; + + if (!${eval1.isNull} && !${eval1.primitive}) { + } else { + ${eval2.code} + if (!${eval2.isNull} && !${eval2.primitive}) { + } else if (!${eval1.isNull} && !${eval2.isNull}) { + ${ev.primitive} = true; + } else { + ${ev.isNull} = true; + } + } + """ + } } case class Or(left: Expression, right: Expression) @@ -167,6 +195,29 @@ case class Or(left: Expression, right: Expression) } } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + + // The result should be `true`, if any of them is `true` whenever the other is null or not. + s""" + ${eval1.code} + boolean ${ev.isNull} = false; + boolean ${ev.primitive} = true; + + if (!${eval1.isNull} && ${eval1.primitive}) { + } else { + ${eval2.code} + if (!${eval2.isNull} && ${eval2.primitive}) { + } else if (!${eval1.isNull} && !${eval2.isNull}) { + ${ev.primitive} = false; + } else { + ${ev.isNull} = true; + } + } + """ + } } abstract class BinaryComparison extends BinaryExpression with Predicate { @@ -198,6 +249,20 @@ abstract class BinaryComparison extends BinaryExpression with Predicate { } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + left.dataType match { + case dt: NumericType if ctx.isNativeType(dt) => defineCodeGen (ctx, ev, { + (c1, c3) => s"$c1 $symbol $c3" + }) + case TimestampType => + // java.sql.Timestamp does not have compare() + super.genCode(ctx, ev) + case other => defineCodeGen (ctx, ev, { + (c1, c2) => s"$c1.compare($c2) $symbol 0" + }) + } + } + protected def evalInternal(evalE1: Any, evalE2: Any): Any = sys.error(s"BinaryComparisons must override either eval or evalInternal") } @@ -215,6 +280,9 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison if (left.dataType != BinaryType) l == r else java.util.Arrays.equals(l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, ctx.equalFunc(left.dataType)) + } } case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison { @@ -235,6 +303,17 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp l == r } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + val equalCode = ctx.equalFunc(left.dataType)(eval1.primitive, eval2.primitive) + ev.isNull = "false" + eval1.code + eval2.code + s""" + boolean ${ev.primitive} = (${eval1.isNull} && ${eval2.isNull}) || + (!${eval1.isNull} && $equalCode); + """ + } } case class LessThan(left: Expression, right: Expression) extends BinaryComparison { @@ -309,6 +388,27 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val condEval = predicate.gen(ctx) + val trueEval = trueValue.gen(ctx) + val falseEval = falseValue.gen(ctx) + + s""" + ${condEval.code} + boolean ${ev.isNull} = false; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${condEval.isNull} && ${condEval.primitive}) { + ${trueEval.code} + ${ev.isNull} = ${trueEval.isNull}; + ${ev.primitive} = ${trueEval.primitive}; + } else { + ${falseEval.code} + ${ev.isNull} = ${falseEval.isNull}; + ${ev.primitive} = ${falseEval.primitive}; + } + """ + } + override def toString: String = s"if ($predicate) $trueValue else $falseValue" } @@ -393,6 +493,48 @@ case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { return res } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val len = branchesArr.length + val got = ctx.freshName("got") + + val cases = (0 until len/2).map { i => + val cond = branchesArr(i * 2).gen(ctx) + val res = branchesArr(i * 2 + 1).gen(ctx) + s""" + if (!$got) { + ${cond.code} + if (!${cond.isNull} && ${cond.primitive}) { + $got = true; + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + } + """ + }.mkString("\n") + + val other = if (len % 2 == 1) { + val res = branchesArr(len - 1).gen(ctx) + s""" + if (!$got) { + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + """ + } else { + "" + } + + s""" + boolean $got = false; + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + $cases + $other + """ + } + override def toString: String = { "CASE" + branches.sliding(2, 2).map { case Seq(cond, value) => s" WHEN $cond THEN $value" @@ -444,6 +586,52 @@ case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseW return res } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val keyEval = key.gen(ctx) + val len = branchesArr.length + val got = ctx.freshName("got") + + val cases = (0 until len/2).map { i => + val cond = branchesArr(i * 2).gen(ctx) + val res = branchesArr(i * 2 + 1).gen(ctx) + s""" + if (!$got) { + ${cond.code} + if (${keyEval.isNull} && ${cond.isNull} || + !${keyEval.isNull} && !${cond.isNull} + && ${ctx.equalFunc(key.dataType)(keyEval.primitive, cond.primitive)}) { + $got = true; + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + } + """ + }.mkString("\n") + + val other = if (len % 2 == 1) { + val res = branchesArr(len - 1).gen(ctx) + s""" + if (!$got) { + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + """ + } else { + "" + } + + s""" + boolean $got = false; + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + ${keyEval.code} + $cases + $other + """ + } + private def equalNullSafe(l: Any, r: Any) = { if (l == null && r == null) { true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index b65bf165f21db..b39349b988389 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, Code, CodeGenContext} import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashSet @@ -60,6 +61,17 @@ case class NewSet(elementType: DataType) extends LeafExpression { new OpenHashSet[Any]() } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + elementType match { + case IntegerType | LongType => + ev.isNull = "false" + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = new ${ctx.javaType(dataType)}(); + """ + case _ => super.genCode(ctx, ev) + } + } + override def toString: String = s"new Set($dataType)" } @@ -91,6 +103,25 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val elementType = set.dataType.asInstanceOf[OpenHashSetUDT].elementType + elementType match { + case IntegerType | LongType => + val itemEval = item.gen(ctx) + val setEval = set.gen(ctx) + val htype = ctx.javaType(dataType) + + ev.isNull = "false" + ev.primitive = setEval.primitive + itemEval.code + setEval.code + s""" + if (!${itemEval.isNull} && !${setEval.isNull}) { + (($htype)${setEval.primitive}).add(${itemEval.primitive}); + } + """ + case _ => super.genCode(ctx, ev) + } + } + override def toString: String = s"$set += $item" } @@ -116,14 +147,31 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres val rightValue = iterator.next() leftEval.add(rightValue) } - leftEval - } else { - null } + leftEval } else { null } } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val elementType = left.dataType.asInstanceOf[OpenHashSetUDT].elementType + elementType match { + case IntegerType | LongType => + val leftEval = left.gen(ctx) + val rightEval = right.gen(ctx) + val htype = ctx.javaType(dataType) + + ev.isNull = leftEval.isNull + ev.primitive = leftEval.primitive + leftEval.code + rightEval.code + s""" + if (!${leftEval.isNull} && !${rightEval.isNull}) { + ${leftEval.primitive}.union((${htype})${rightEval.primitive}); + } + """ + case _ => super.genCode(ctx, ev) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index c4ef9c30907f1..78adb509b470b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.types._ trait StringRegexExpression extends ExpectsInputTypes { @@ -137,6 +138,10 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE override def convert(v: UTF8String): UTF8String = v.toUpperCase() override def toString: String = s"Upper($child)" + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, c => s"($c).toUpperCase()") + } } /** @@ -147,6 +152,10 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE override def convert(v: UTF8String): UTF8String = v.toLowerCase() override def toString: String = s"Lower($child)" + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, c => s"($c).toLowerCase()") + } } /** A base trait for functions that compare two strings, returning a boolean. */ @@ -181,6 +190,9 @@ trait StringComparison extends ExpectsInputTypes { case class Contains(left: Expression, right: Expression) extends BinaryExpression with Predicate with StringComparison { override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"($c1).contains($c2)") + } } /** @@ -189,6 +201,9 @@ case class Contains(left: Expression, right: Expression) case class StartsWith(left: Expression, right: Expression) extends BinaryExpression with Predicate with StringComparison { override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"($c1).startsWith($c2)") + } } /** @@ -197,6 +212,9 @@ case class StartsWith(left: Expression, right: Expression) case class EndsWith(left: Expression, right: Expression) extends BinaryExpression with Predicate with StringComparison { override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"($c1).endsWith($c2)") + } } /** 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 5df528770ca6e..eea2edc323eea 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 @@ -28,6 +28,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateProjection, GenerateMutableProjection} import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.catalyst.util.DateUtils import org.apache.spark.sql.types._ @@ -35,11 +36,20 @@ import org.apache.spark.sql.types._ class ExpressionEvaluationBaseSuite extends SparkFunSuite { + def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + checkEvaluationWithoutCodegen(expression, expected, inputRow) + checkEvaluationWithGeneratedMutableProjection(expression, expected, inputRow) + checkEvaluationWithGeneratedProjection(expression, expected, inputRow) + } + def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { expression.eval(inputRow) } - def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + def checkEvaluationWithoutCodegen( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { val actual = try evaluate(expression, inputRow) catch { case e: Exception => fail(s"Exception evaluating $expression", e) } @@ -49,6 +59,68 @@ class ExpressionEvaluationBaseSuite extends SparkFunSuite { } } + def checkEvaluationWithGeneratedMutableProjection( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + + val plan = try { + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)() + } catch { + case e: Throwable => + val ctx = GenerateProjection.newCodeGenContext() + val evaluated = expression.gen(ctx) + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code} + |$e + """.stripMargin) + } + + val actual = plan(inputRow).apply(0) + if (actual != expected) { + val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + def checkEvaluationWithGeneratedProjection( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val ctx = GenerateProjection.newCodeGenContext() + lazy val evaluated = expression.gen(ctx) + + val plan = try { + GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil) + } catch { + case e: Throwable => + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code} + |$e + """.stripMargin) + } + + val actual = plan(inputRow) + val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) + if (actual.hashCode() != expectedRow.hashCode()) { + fail( + s""" + |Mismatched hashCodes for values: $actual, $expectedRow + |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()} + |Expressions: ${expression} + |Code: ${evaluated} + """.stripMargin) + } + if (actual != expectedRow) { + val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + def checkDoubleEvaluation( expression: Expression, expected: Spread[Double], @@ -69,8 +141,16 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("literals") { checkEvaluation(Literal(1), 1) checkEvaluation(Literal(true), true) + checkEvaluation(Literal(false), false) checkEvaluation(Literal(0L), 0L) + List(0.0, -0.0, Double.NegativeInfinity, Double.PositiveInfinity).foreach { + d => { + checkEvaluation(Literal(d), d) + checkEvaluation(Literal(d.toFloat), d.toFloat) + } + } checkEvaluation(Literal("test"), "test") + checkEvaluation(Literal.create(null, StringType), null) checkEvaluation(Literal(1) + Literal(1), 2) } @@ -1367,6 +1447,11 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { // TODO: Make the tests work with codegen. class ExpressionEvaluationWithoutCodeGenSuite extends ExpressionEvaluationBaseSuite { + override def checkEvaluation( + expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + checkEvaluationWithoutCodegen(expression, expected, inputRow) + } + test("CreateStruct") { val row = Row(1, 2, 3) val c1 = 'a.int.at(0).as("a") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala index 8cfd853afa35f..371a73181dad7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -21,34 +21,9 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ /** - * Overrides our expression evaluation tests to use code generation for evaluation. + * Additional tests for code generation. */ class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { - override def checkEvaluation( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { - val plan = try { - GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)() - } catch { - case e: Throwable => - val ctx = GenerateProjection.newCodeGenContext() - val evaluated = GenerateProjection.expressionEvaluator(expression, ctx) - fail( - s""" - |Code generation of $expression failed: - |${evaluated.code} - |$e - """.stripMargin) - } - - val actual = plan(inputRow).apply(0) - if (actual != expected) { - val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } - test("multithreaded eval") { import scala.concurrent._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala deleted file mode 100644 index 9ab1f7d7ad0db..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.catalyst.expressions - -import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.codegen._ - -/** - * Overrides our expression evaluation tests to use generated code on mutable rows. - */ -class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { - override def checkEvaluation( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { - val ctx = GenerateProjection.newCodeGenContext() - lazy val evaluated = GenerateProjection.expressionEvaluator(expression, ctx) - - val plan = try { - GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil) - } catch { - case e: Throwable => - fail( - s""" - |Code generation of $expression failed: - |${evaluated.code} - |$e - """.stripMargin) - } - - val actual = plan(inputRow) - val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) - if (actual.hashCode() != expectedRow.hashCode()) { - fail( - s""" - |Mismatched hashCodes for values: $actual, $expectedRow - |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()} - |${evaluated.code} - """.stripMargin) - } - if (actual != expectedRow) { - val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index 8979a0a210a42..d9a010a9815a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -53,7 +53,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { check("10", Literal.create(10, IntegerType)) check("1000000000000000", Literal.create(1000000000000000L, LongType)) - check("1.5", Literal.create(1.5, FloatType)) + check("1.5", Literal.create(1.5f, FloatType)) check("hello", Literal.create("hello", StringType)) check(defaultPartitionName, Literal.create(null, NullType)) } @@ -83,13 +83,13 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { ArrayBuffer( Literal.create(10, IntegerType), Literal.create("hello", StringType), - Literal.create(1.5, FloatType))) + Literal.create(1.5f, FloatType))) }) check("file://path/a=10/b_hello/c=1.5", Some { PartitionValues( ArrayBuffer("c"), - ArrayBuffer(Literal.create(1.5, FloatType))) + ArrayBuffer(Literal.create(1.5f, FloatType))) }) check("file:///", None) From b3a40f8592aabebdb5dc5a9ab0cfcb8580aa6faf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Jun 2015 01:23:55 -0700 Subject: [PATCH 3/5] Refactored expression test suites. --- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 6 +- .../sql/catalyst/expressions/arithmetic.scala | 96 -- .../sql/catalyst/expressions/bitwise.scala | 120 ++ .../catalyst/expressions/conditionals.scala | 313 ++++ .../{mathfuncs/unary.scala => math.scala} | 99 +- .../expressions/mathfuncs/binary.scala | 100 -- .../sql/catalyst/expressions/predicates.scala | 290 ---- .../ArithmeticExpressionSuite.scala | 144 ++ .../expressions/BitwiseFunctionsSuite.scala | 80 + .../sql/catalyst/expressions/CastSuite.scala | 532 ++++++ ...nSuite.scala => CodeGenerationSuite.scala} | 3 +- .../expressions/ComplexTypeSuite.scala | 122 ++ .../ConditionalExpressionSuite.scala | 96 ++ .../expressions/ExpressionEvalHelper.scala | 134 ++ .../ExpressionEvaluationSuite.scala | 1461 ----------------- .../expressions/LiteralExpressionSuite.scala | 55 + .../expressions/MathFunctionsSuite.scala | 179 ++ .../expressions/NullFunctionsSuite.scala | 65 + .../catalyst/expressions/PredicateSuite.scala | 179 ++ .../expressions/StringFunctionsSuite.scala | 218 +++ .../ExpressionOptimizationSuite.scala | 3 +- 22 files changed, 2340 insertions(+), 1957 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{mathfuncs/unary.scala => math.scala} (54%) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/{GeneratedEvaluationSuite.scala => CodeGenerationSuite.scala} (94%) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 5f76a512679a4..2a1f96409daf4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -161,7 +161,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null } }) case BooleanType => - buildCast[Boolean](_, b => new Timestamp((if (b) 1 else 0))) + buildCast[Boolean](_, b => new Timestamp(if (b) 1 else 0)) case LongType => buildCast[Long](_, l => new Timestamp(l)) case IntegerType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 0ed576b3d5870..432d65eee54fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -81,11 +81,11 @@ abstract class Expression extends TreeNode[Expression] { val objectTerm = ctx.freshName("obj") s""" /* expression: ${this} */ - Object ${objectTerm} = expressions[${ctx.references.size - 1}].eval(i); - boolean ${ev.isNull} = ${objectTerm} == null; + Object $objectTerm = expressions[${ctx.references.size - 1}].eval(i); + boolean ${ev.isNull} = $objectTerm == null; ${ctx.javaType(this.dataType)} ${ev.primitive} = ${ctx.defaultValue(this.dataType)}; if (!${ev.isNull}) { - ${ev.primitive} = (${ctx.boxedType(this.dataType)})${objectTerm}; + ${ev.primitive} = (${ctx.boxedType(this.dataType)}) $objectTerm; } """ } 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 3ac7c92dcd009..d4efda2e04c29 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 @@ -322,102 +322,6 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet } } -/** - * A function that calculates bitwise and(&) of two numbers. - */ -case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { - override def symbol: String = "&" - - protected def checkTypesInternal(t: DataType) = - TypeUtils.checkForBitwiseExpr(t, "operator " + symbol) - - private lazy val and: (Any, Any) => Any = dataType match { - case ByteType => - ((evalE1: Byte, evalE2: Byte) => (evalE1 & evalE2).toByte).asInstanceOf[(Any, Any) => Any] - case ShortType => - ((evalE1: Short, evalE2: Short) => (evalE1 & evalE2).toShort).asInstanceOf[(Any, Any) => Any] - case IntegerType => - ((evalE1: Int, evalE2: Int) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any] - case LongType => - ((evalE1: Long, evalE2: Long) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any] - } - - protected override def evalInternal(evalE1: Any, evalE2: Any) = and(evalE1, evalE2) -} - -/** - * A function that calculates bitwise or(|) of two numbers. - */ -case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { - override def symbol: String = "|" - - protected def checkTypesInternal(t: DataType) = - TypeUtils.checkForBitwiseExpr(t, "operator " + symbol) - - private lazy val or: (Any, Any) => Any = dataType match { - case ByteType => - ((evalE1: Byte, evalE2: Byte) => (evalE1 | evalE2).toByte).asInstanceOf[(Any, Any) => Any] - case ShortType => - ((evalE1: Short, evalE2: Short) => (evalE1 | evalE2).toShort).asInstanceOf[(Any, Any) => Any] - case IntegerType => - ((evalE1: Int, evalE2: Int) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any] - case LongType => - ((evalE1: Long, evalE2: Long) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any] - } - - protected override def evalInternal(evalE1: Any, evalE2: Any) = or(evalE1, evalE2) -} - -/** - * A function that calculates bitwise xor of two numbers. - */ -case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { - override def symbol: String = "^" - - protected def checkTypesInternal(t: DataType) = - TypeUtils.checkForBitwiseExpr(t, "operator " + symbol) - - private lazy val xor: (Any, Any) => Any = dataType match { - case ByteType => - ((evalE1: Byte, evalE2: Byte) => (evalE1 ^ evalE2).toByte).asInstanceOf[(Any, Any) => Any] - case ShortType => - ((evalE1: Short, evalE2: Short) => (evalE1 ^ evalE2).toShort).asInstanceOf[(Any, Any) => Any] - case IntegerType => - ((evalE1: Int, evalE2: Int) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any] - case LongType => - ((evalE1: Long, evalE2: Long) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any] - } - - protected override def evalInternal(evalE1: Any, evalE2: Any): Any = xor(evalE1, evalE2) -} - -/** - * A function that calculates bitwise not(~) of a number. - */ -case class BitwiseNot(child: Expression) extends UnaryArithmetic { - override def toString: String = s"~$child" - - override def checkInputDataTypes(): TypeCheckResult = - TypeUtils.checkForBitwiseExpr(child.dataType, "operator ~") - - private lazy val not: (Any) => Any = dataType match { - case ByteType => - ((evalE: Byte) => (~evalE).toByte).asInstanceOf[(Any) => Any] - case ShortType => - ((evalE: Short) => (~evalE).toShort).asInstanceOf[(Any) => Any] - case IntegerType => - ((evalE: Int) => ~evalE).asInstanceOf[(Any) => Any] - case LongType => - ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any] - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)})~($c)") - } - - protected override def evalInternal(evalE: Any) = not(evalE) -} - case class MaxOf(left: Expression, right: Expression) extends BinaryArithmetic { override def nullable: Boolean = left.nullable && right.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala new file mode 100644 index 0000000000000..ef34586261e70 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala @@ -0,0 +1,120 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.types._ + + +/** + * A function that calculates bitwise and(&) of two numbers. + */ +case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { + override def symbol: String = "&" + + protected def checkTypesInternal(t: DataType) = + TypeUtils.checkForBitwiseExpr(t, "operator " + symbol) + + private lazy val and: (Any, Any) => Any = dataType match { + case ByteType => + ((evalE1: Byte, evalE2: Byte) => (evalE1 & evalE2).toByte).asInstanceOf[(Any, Any) => Any] + case ShortType => + ((evalE1: Short, evalE2: Short) => (evalE1 & evalE2).toShort).asInstanceOf[(Any, Any) => Any] + case IntegerType => + ((evalE1: Int, evalE2: Int) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any] + case LongType => + ((evalE1: Long, evalE2: Long) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any] + } + + protected override def evalInternal(evalE1: Any, evalE2: Any) = and(evalE1, evalE2) +} + +/** + * A function that calculates bitwise or(|) of two numbers. + */ +case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { + override def symbol: String = "|" + + protected def checkTypesInternal(t: DataType) = + TypeUtils.checkForBitwiseExpr(t, "operator " + symbol) + + private lazy val or: (Any, Any) => Any = dataType match { + case ByteType => + ((evalE1: Byte, evalE2: Byte) => (evalE1 | evalE2).toByte).asInstanceOf[(Any, Any) => Any] + case ShortType => + ((evalE1: Short, evalE2: Short) => (evalE1 | evalE2).toShort).asInstanceOf[(Any, Any) => Any] + case IntegerType => + ((evalE1: Int, evalE2: Int) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any] + case LongType => + ((evalE1: Long, evalE2: Long) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any] + } + + protected override def evalInternal(evalE1: Any, evalE2: Any) = or(evalE1, evalE2) +} + +/** + * A function that calculates bitwise xor of two numbers. + */ +case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { + override def symbol: String = "^" + + protected def checkTypesInternal(t: DataType) = + TypeUtils.checkForBitwiseExpr(t, "operator " + symbol) + + private lazy val xor: (Any, Any) => Any = dataType match { + case ByteType => + ((evalE1: Byte, evalE2: Byte) => (evalE1 ^ evalE2).toByte).asInstanceOf[(Any, Any) => Any] + case ShortType => + ((evalE1: Short, evalE2: Short) => (evalE1 ^ evalE2).toShort).asInstanceOf[(Any, Any) => Any] + case IntegerType => + ((evalE1: Int, evalE2: Int) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any] + case LongType => + ((evalE1: Long, evalE2: Long) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any] + } + + protected override def evalInternal(evalE1: Any, evalE2: Any): Any = xor(evalE1, evalE2) +} + +/** + * A function that calculates bitwise not(~) of a number. + */ +case class BitwiseNot(child: Expression) extends UnaryArithmetic { + override def toString: String = s"~$child" + + override def checkInputDataTypes(): TypeCheckResult = + TypeUtils.checkForBitwiseExpr(child.dataType, "operator ~") + + private lazy val not: (Any) => Any = dataType match { + case ByteType => + ((evalE: Byte) => (~evalE).toByte).asInstanceOf[(Any) => Any] + case ShortType => + ((evalE: Short) => (~evalE).toShort).asInstanceOf[(Any) => Any] + case IntegerType => + ((evalE: Int) => ~evalE).asInstanceOf[(Any) => Any] + case LongType => + ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any] + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)})~($c)") + } + + protected override def evalInternal(evalE: Any) = not(evalE) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala new file mode 100644 index 0000000000000..3aa86edd7ab20 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala @@ -0,0 +1,313 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.types.{BooleanType, DataType} + + +case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) + extends Expression { + + override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil + override def nullable: Boolean = trueValue.nullable || falseValue.nullable + + override def checkInputDataTypes(): TypeCheckResult = { + if (predicate.dataType != BooleanType) { + TypeCheckResult.TypeCheckFailure( + s"type of predicate expression in If should be boolean, not ${predicate.dataType}") + } else if (trueValue.dataType != falseValue.dataType) { + TypeCheckResult.TypeCheckFailure( + s"differing types in If (${trueValue.dataType} and ${falseValue.dataType}).") + } else { + TypeCheckResult.TypeCheckSuccess + } + } + + override def dataType: DataType = trueValue.dataType + + override def eval(input: Row): Any = { + if (true == predicate.eval(input)) { + trueValue.eval(input) + } else { + falseValue.eval(input) + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val condEval = predicate.gen(ctx) + val trueEval = trueValue.gen(ctx) + val falseEval = falseValue.gen(ctx) + + s""" + ${condEval.code} + boolean ${ev.isNull} = false; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${condEval.isNull} && ${condEval.primitive}) { + ${trueEval.code} + ${ev.isNull} = ${trueEval.isNull}; + ${ev.primitive} = ${trueEval.primitive}; + } else { + ${falseEval.code} + ${ev.isNull} = ${falseEval.isNull}; + ${ev.primitive} = ${falseEval.primitive}; + } + """ + } + + override def toString: String = s"if ($predicate) $trueValue else $falseValue" +} + +trait CaseWhenLike extends Expression { + self: Product => + + // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last + // element is the value for the default catch-all case (if provided). + // Hence, `branches` consists of at least two elements, and can have an odd or even length. + def branches: Seq[Expression] + + @transient lazy val whenList = + branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq + @transient lazy val thenList = + branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq + val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) + + // both then and else expressions should be considered. + def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) + def valueTypesEqual: Boolean = valueTypes.distinct.size == 1 + + override def checkInputDataTypes(): TypeCheckResult = { + if (valueTypesEqual) { + checkTypesInternal() + } else { + TypeCheckResult.TypeCheckFailure( + "THEN and ELSE expressions should all be same type or coercible to a common type") + } + } + + protected def checkTypesInternal(): TypeCheckResult + + override def dataType: DataType = thenList.head.dataType + + override def nullable: Boolean = { + // If no value is nullable and no elseValue is provided, the whole statement defaults to null. + thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) + } +} + +// scalastyle:off +/** + * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". + * Refer to this link for the corresponding semantics: + * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions + */ +// scalastyle:on +case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. + @transient private[this] lazy val branchesArr = branches.toArray + + override def children: Seq[Expression] = branches + + override protected def checkTypesInternal(): TypeCheckResult = { + if (whenList.forall(_.dataType == BooleanType)) { + TypeCheckResult.TypeCheckSuccess + } else { + val index = whenList.indexWhere(_.dataType != BooleanType) + TypeCheckResult.TypeCheckFailure( + s"WHEN expressions in CaseWhen should all be boolean type, " + + s"but the ${index + 1}th when expression's type is ${whenList(index)}") + } + } + + /** Written in imperative fashion for performance considerations. */ + override def eval(input: Row): Any = { + val len = branchesArr.length + var i = 0 + // If all branches fail and an elseVal is not provided, the whole statement + // defaults to null, according to Hive's semantics. + while (i < len - 1) { + if (branchesArr(i).eval(input) == true) { + return branchesArr(i + 1).eval(input) + } + i += 2 + } + var res: Any = null + if (i == len - 1) { + res = branchesArr(i).eval(input) + } + return res + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val len = branchesArr.length + val got = ctx.freshName("got") + + val cases = (0 until len/2).map { i => + val cond = branchesArr(i * 2).gen(ctx) + val res = branchesArr(i * 2 + 1).gen(ctx) + s""" + if (!$got) { + ${cond.code} + if (!${cond.isNull} && ${cond.primitive}) { + $got = true; + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + } + """ + }.mkString("\n") + + val other = if (len % 2 == 1) { + val res = branchesArr(len - 1).gen(ctx) + s""" + if (!$got) { + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + """ + } else { + "" + } + + s""" + boolean $got = false; + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + $cases + $other + """ + } + + override def toString: String = { + "CASE" + branches.sliding(2, 2).map { + case Seq(cond, value) => s" WHEN $cond THEN $value" + case Seq(elseValue) => s" ELSE $elseValue" + }.mkString + } +} + +// scalastyle:off +/** + * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". + * Refer to this link for the corresponding semantics: + * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions + */ +// scalastyle:on +case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { + + // Use private[this] Array to speed up evaluation. + @transient private[this] lazy val branchesArr = branches.toArray + + override def children: Seq[Expression] = key +: branches + + override protected def checkTypesInternal(): TypeCheckResult = { + if ((key +: whenList).map(_.dataType).distinct.size > 1) { + TypeCheckResult.TypeCheckFailure( + "key and WHEN expressions should all be same type or coercible to a common type") + } else { + TypeCheckResult.TypeCheckSuccess + } + } + + /** Written in imperative fashion for performance considerations. */ + override def eval(input: Row): Any = { + val evaluatedKey = key.eval(input) + val len = branchesArr.length + var i = 0 + // If all branches fail and an elseVal is not provided, the whole statement + // defaults to null, according to Hive's semantics. + while (i < len - 1) { + if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) { + return branchesArr(i + 1).eval(input) + } + i += 2 + } + var res: Any = null + if (i == len - 1) { + res = branchesArr(i).eval(input) + } + return res + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + val keyEval = key.gen(ctx) + val len = branchesArr.length + val got = ctx.freshName("got") + + val cases = (0 until len/2).map { i => + val cond = branchesArr(i * 2).gen(ctx) + val res = branchesArr(i * 2 + 1).gen(ctx) + s""" + if (!$got) { + ${cond.code} + if (${keyEval.isNull} && ${cond.isNull} || + !${keyEval.isNull} && !${cond.isNull} + && ${ctx.equalFunc(key.dataType)(keyEval.primitive, cond.primitive)}) { + $got = true; + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + } + """ + }.mkString("\n") + + val other = if (len % 2 == 1) { + val res = branchesArr(len - 1).gen(ctx) + s""" + if (!$got) { + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.primitive} = ${res.primitive}; + } + """ + } else { + "" + } + + s""" + boolean $got = false; + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + ${keyEval.code} + $cases + $other + """ + } + + private def equalNullSafe(l: Any, r: Any) = { + if (l == null && r == null) { + true + } else if (l == null || r == null) { + false + } else { + l == r + } + } + + override def toString: String = { + s"CASE $key" + branches.sliding(2, 2).map { + case Seq(cond, value) => s" WHEN $cond THEN $value" + case Seq(elseValue) => s" ELSE $elseValue" + }.mkString + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala similarity index 54% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala index 5563cd94bf86d..a18067e4a58f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions.mathfuncs +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.codegen.{Code, CodeGenContext, GeneratedExpressionCode} -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, Row, UnaryExpression} -import org.apache.spark.sql.types._ +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.types.{DataType, DoubleType} /** * A unary expression specifically for math functions. Math Functions expect a specific type of @@ -64,6 +63,47 @@ abstract class UnaryMathExpression(f: Double => Double, name: String) } } +/** + * A binary expression specifically for math functions that take two `Double`s as input and returns + * a `Double`. + * @param f The math function. + * @param name The short name of the function + */ +abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) + extends BinaryExpression with Serializable with ExpectsInputTypes { self: Product => + + override def expectedChildTypes: Seq[DataType] = Seq(DoubleType, DoubleType) + + override def toString: String = s"$name($left, $right)" + + override def dataType: DataType = DoubleType + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if (evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + val result = f(evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) + if (result.isNaN) null else result + } + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)") + } +} + +//////////////////////////////////////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////////////////////////////////////// +// Unary math functions +//////////////////////////////////////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////////////////////////////////////// + case class Acos(child: Expression) extends UnaryMathExpression(math.acos, "ACOS") case class Asin(child: Expression) extends UnaryMathExpression(math.asin, "ASIN") @@ -111,3 +151,54 @@ case class ToDegrees(child: Expression) extends UnaryMathExpression(math.toDegre case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadians, "RADIANS") { override def funcName: String = "toRadians" } + + +//////////////////////////////////////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////////////////////////////////////// +// Binary math functions +//////////////////////////////////////////////////////////////////////////////////////////////////// +//////////////////////////////////////////////////////////////////////////////////////////////////// + + +case class Atan2(left: Expression, right: Expression) + extends BinaryMathExpression(math.atan2, "ATAN2") { + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if (evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + // With codegen, the values returned by -0.0 and 0.0 are different. Handled with +0.0 + val result = math.atan2(evalE1.asInstanceOf[Double] + 0.0, + evalE2.asInstanceOf[Double] + 0.0) + if (result.isNaN) null else result + } + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") + s""" + if (Double.valueOf(${ev.primitive}).isNaN()) { + ${ev.isNull} = true; + } + """ + } +} + +case class Hypot(left: Expression, right: Expression) + extends BinaryMathExpression(math.hypot, "HYPOT") + +case class Pow(left: Expression, right: Expression) + extends BinaryMathExpression(math.pow, "POWER") { + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { + defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") + s""" + if (Double.valueOf(${ev.primitive}).isNaN()) { + ${ev.isNull} = true; + } + """ + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala deleted file mode 100644 index 88211acd7713c..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.catalyst.expressions.mathfuncs - -import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, BinaryExpression, Expression, Row} -import org.apache.spark.sql.types._ - -/** - * A binary expression specifically for math functions that take two `Double`s as input and returns - * a `Double`. - * @param f The math function. - * @param name The short name of the function - */ -abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) - extends BinaryExpression with Serializable with ExpectsInputTypes { self: Product => - - override def expectedChildTypes: Seq[DataType] = Seq(DoubleType, DoubleType) - - override def toString: String = s"$name($left, $right)" - - override def dataType: DataType = DoubleType - - override def eval(input: Row): Any = { - val evalE1 = left.eval(input) - if (evalE1 == null) { - null - } else { - val evalE2 = right.eval(input) - if (evalE2 == null) { - null - } else { - val result = f(evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) - if (result.isNaN) null else result - } - } - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)") - } -} - -case class Atan2(left: Expression, right: Expression) - extends BinaryMathExpression(math.atan2, "ATAN2") { - - override def eval(input: Row): Any = { - val evalE1 = left.eval(input) - if (evalE1 == null) { - null - } else { - val evalE2 = right.eval(input) - if (evalE2 == null) { - null - } else { - // With codegen, the values returned by -0.0 and 0.0 are different. Handled with +0.0 - val result = math.atan2(evalE1.asInstanceOf[Double] + 0.0, - evalE2.asInstanceOf[Double] + 0.0) - if (result.isNaN) null else result - } - } - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") + s""" - if (Double.valueOf(${ev.primitive}).isNaN()) { - ${ev.isNull} = true; - } - """ - } -} - -case class Hypot(left: Expression, right: Expression) - extends BinaryMathExpression(math.hypot, "HYPOT") - -case class Pow(left: Expression, right: Expression) - extends BinaryMathExpression(math.pow, "POWER") { - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") + s""" - if (Double.valueOf(${ev.primitive}).isNaN()) { - ${ev.isNull} = true; - } - """ - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 1d0f19a400d63..5edcf3bd77d20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -359,293 +359,3 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar protected override def evalInternal(evalE1: Any, evalE2: Any) = ordering.gteq(evalE1, evalE2) } - -case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) - extends Expression { - - override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil - override def nullable: Boolean = trueValue.nullable || falseValue.nullable - - override def checkInputDataTypes(): TypeCheckResult = { - if (predicate.dataType != BooleanType) { - TypeCheckResult.TypeCheckFailure( - s"type of predicate expression in If should be boolean, not ${predicate.dataType}") - } else if (trueValue.dataType != falseValue.dataType) { - TypeCheckResult.TypeCheckFailure( - s"differing types in If (${trueValue.dataType} and ${falseValue.dataType}).") - } else { - TypeCheckResult.TypeCheckSuccess - } - } - - override def dataType: DataType = trueValue.dataType - - override def eval(input: Row): Any = { - if (true == predicate.eval(input)) { - trueValue.eval(input) - } else { - falseValue.eval(input) - } - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - val condEval = predicate.gen(ctx) - val trueEval = trueValue.gen(ctx) - val falseEval = falseValue.gen(ctx) - - s""" - ${condEval.code} - boolean ${ev.isNull} = false; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${condEval.isNull} && ${condEval.primitive}) { - ${trueEval.code} - ${ev.isNull} = ${trueEval.isNull}; - ${ev.primitive} = ${trueEval.primitive}; - } else { - ${falseEval.code} - ${ev.isNull} = ${falseEval.isNull}; - ${ev.primitive} = ${falseEval.primitive}; - } - """ - } - - override def toString: String = s"if ($predicate) $trueValue else $falseValue" -} - -trait CaseWhenLike extends Expression { - self: Product => - - // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last - // element is the value for the default catch-all case (if provided). - // Hence, `branches` consists of at least two elements, and can have an odd or even length. - def branches: Seq[Expression] - - @transient lazy val whenList = - branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq - @transient lazy val thenList = - branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq - val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) - - // both then and else expressions should be considered. - def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) - def valueTypesEqual: Boolean = valueTypes.distinct.size == 1 - - override def checkInputDataTypes(): TypeCheckResult = { - if (valueTypesEqual) { - checkTypesInternal() - } else { - TypeCheckResult.TypeCheckFailure( - "THEN and ELSE expressions should all be same type or coercible to a common type") - } - } - - protected def checkTypesInternal(): TypeCheckResult - - override def dataType: DataType = thenList.head.dataType - - override def nullable: Boolean = { - // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) - } -} - -// scalastyle:off -/** - * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". - * Refer to this link for the corresponding semantics: - * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - */ -// scalastyle:on -case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray - - override def children: Seq[Expression] = branches - - override protected def checkTypesInternal(): TypeCheckResult = { - if (whenList.forall(_.dataType == BooleanType)) { - TypeCheckResult.TypeCheckSuccess - } else { - val index = whenList.indexWhere(_.dataType != BooleanType) - TypeCheckResult.TypeCheckFailure( - s"WHEN expressions in CaseWhen should all be boolean type, " + - s"but the ${index + 1}th when expression's type is ${whenList(index)}") - } - } - - /** Written in imperative fashion for performance considerations. */ - override def eval(input: Row): Any = { - val len = branchesArr.length - var i = 0 - // If all branches fail and an elseVal is not provided, the whole statement - // defaults to null, according to Hive's semantics. - while (i < len - 1) { - if (branchesArr(i).eval(input) == true) { - return branchesArr(i + 1).eval(input) - } - i += 2 - } - var res: Any = null - if (i == len - 1) { - res = branchesArr(i).eval(input) - } - return res - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - val len = branchesArr.length - val got = ctx.freshName("got") - - val cases = (0 until len/2).map { i => - val cond = branchesArr(i * 2).gen(ctx) - val res = branchesArr(i * 2 + 1).gen(ctx) - s""" - if (!$got) { - ${cond.code} - if (!${cond.isNull} && ${cond.primitive}) { - $got = true; - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.primitive} = ${res.primitive}; - } - } - """ - }.mkString("\n") - - val other = if (len % 2 == 1) { - val res = branchesArr(len - 1).gen(ctx) - s""" - if (!$got) { - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.primitive} = ${res.primitive}; - } - """ - } else { - "" - } - - s""" - boolean $got = false; - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - $cases - $other - """ - } - - override def toString: String = { - "CASE" + branches.sliding(2, 2).map { - case Seq(cond, value) => s" WHEN $cond THEN $value" - case Seq(elseValue) => s" ELSE $elseValue" - }.mkString - } -} - -// scalastyle:off -/** - * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". - * Refer to this link for the corresponding semantics: - * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - */ -// scalastyle:on -case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray - - override def children: Seq[Expression] = key +: branches - - override protected def checkTypesInternal(): TypeCheckResult = { - if ((key +: whenList).map(_.dataType).distinct.size > 1) { - TypeCheckResult.TypeCheckFailure( - "key and WHEN expressions should all be same type or coercible to a common type") - } else { - TypeCheckResult.TypeCheckSuccess - } - } - - /** Written in imperative fashion for performance considerations. */ - override def eval(input: Row): Any = { - val evaluatedKey = key.eval(input) - val len = branchesArr.length - var i = 0 - // If all branches fail and an elseVal is not provided, the whole statement - // defaults to null, according to Hive's semantics. - while (i < len - 1) { - if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) { - return branchesArr(i + 1).eval(input) - } - i += 2 - } - var res: Any = null - if (i == len - 1) { - res = branchesArr(i).eval(input) - } - return res - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = { - val keyEval = key.gen(ctx) - val len = branchesArr.length - val got = ctx.freshName("got") - - val cases = (0 until len/2).map { i => - val cond = branchesArr(i * 2).gen(ctx) - val res = branchesArr(i * 2 + 1).gen(ctx) - s""" - if (!$got) { - ${cond.code} - if (${keyEval.isNull} && ${cond.isNull} || - !${keyEval.isNull} && !${cond.isNull} - && ${ctx.equalFunc(key.dataType)(keyEval.primitive, cond.primitive)}) { - $got = true; - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.primitive} = ${res.primitive}; - } - } - """ - }.mkString("\n") - - val other = if (len % 2 == 1) { - val res = branchesArr(len - 1).gen(ctx) - s""" - if (!$got) { - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.primitive} = ${res.primitive}; - } - """ - } else { - "" - } - - s""" - boolean $got = false; - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - ${keyEval.code} - $cases - $other - """ - } - - private def equalNullSafe(l: Any, r: Any) = { - if (l == null && r == null) { - true - } else if (l == null || r == null) { - false - } else { - l == r - } - } - - override def toString: String = { - s"CASE $key" + branches.sliding(2, 2).map { - case Seq(cond, value) => s" WHEN $cond THEN $value" - case Seq(elseValue) => s" ELSE $elseValue" - }.mkString - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala new file mode 100644 index 0000000000000..e1afa81a7a82f --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -0,0 +1,144 @@ +/* + * 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.catalyst.expressions + +import org.scalatest.Matchers._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types.{DoubleType, IntegerType} + + +class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("arithmetic") { + val row = create_row(1, 2, 3, null) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(UnaryMinus(c1), -1, row) + checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100) + + checkEvaluation(Add(c1, c4), null, row) + checkEvaluation(Add(c1, c2), 3, row) + checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) + checkEvaluation( + Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + + checkEvaluation(-c1, -1, row) + checkEvaluation(c1 + c2, 3, row) + checkEvaluation(c1 - c2, -1, row) + checkEvaluation(c1 * c2, 2, row) + checkEvaluation(c1 / c2, 0, row) + checkEvaluation(c1 % c2, 1, row) + } + + test("fractional arithmetic") { + val row = create_row(1.1, 2.0, 3.1, null) + val c1 = 'a.double.at(0) + val c2 = 'a.double.at(1) + val c3 = 'a.double.at(2) + val c4 = 'a.double.at(3) + + checkEvaluation(UnaryMinus(c1), -1.1, row) + checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0) + checkEvaluation(Add(c1, c4), null, row) + checkEvaluation(Add(c1, c2), 3.1, row) + checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) + checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) + checkEvaluation( + Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) + + checkEvaluation(-c1, -1.1, row) + checkEvaluation(c1 + c2, 3.1, row) + checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row) + checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row) + checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row) + checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row) + } + + test("Divide") { + checkEvaluation(Divide(Literal(2), Literal(1)), 2) + checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) + checkEvaluation(Divide(Literal(1), Literal(2)), 0) + checkEvaluation(Divide(Literal(1), Literal(0)), null) + checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null) + checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null) + checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) + checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) + } + + test("Remainder") { + checkEvaluation(Remainder(Literal(2), Literal(1)), 0) + checkEvaluation(Remainder(Literal(1.0), Literal(2.0)), 1.0) + checkEvaluation(Remainder(Literal(1), Literal(2)), 1) + checkEvaluation(Remainder(Literal(1), Literal(0)), null) + checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null) + checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null) + checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) + checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) + } + + test("MaxOf") { + checkEvaluation(MaxOf(1, 2), 2) + checkEvaluation(MaxOf(2, 1), 2) + checkEvaluation(MaxOf(1L, 2L), 2L) + checkEvaluation(MaxOf(2L, 1L), 2L) + + checkEvaluation(MaxOf(Literal.create(null, IntegerType), 2), 2) + checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) + } + + test("MinOf") { + checkEvaluation(MinOf(1, 2), 1) + checkEvaluation(MinOf(2, 1), 1) + checkEvaluation(MinOf(1L, 2L), 1L) + checkEvaluation(MinOf(2L, 1L), 1L) + + checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1) + checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1) + } + + test("SQRT") { + val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24)) + val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) + val rowSequence = inputSequence.map(l => create_row(l.toDouble)) + val d = 'a.double.at(0) + + for ((row, expected) <- rowSequence zip expectedResults) { + checkEvaluation(Sqrt(d), expected, row) + } + + checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null)) + checkEvaluation(Sqrt(-1), null, EmptyRow) + checkEvaluation(Sqrt(-1.5), null, EmptyRow) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala new file mode 100644 index 0000000000000..c9bbc7a8b8c14 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala @@ -0,0 +1,80 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types._ + + +class BitwiseFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("Bitwise operations") { + val row = create_row(1, 2, 3, null) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(BitwiseAnd(c1, c4), null, row) + checkEvaluation(BitwiseAnd(c1, c2), 0, row) + checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + + checkEvaluation(BitwiseOr(c1, c4), null, row) + checkEvaluation(BitwiseOr(c1, c2), 3, row) + checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + + checkEvaluation(BitwiseXor(c1, c4), null, row) + checkEvaluation(BitwiseXor(c1, c2), 3, row) + checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + + checkEvaluation(BitwiseNot(c4), null, row) + checkEvaluation(BitwiseNot(c1), -2, row) + checkEvaluation(BitwiseNot(Literal.create(null, IntegerType)), null, row) + + checkEvaluation(c1 & c2, 0, row) + checkEvaluation(c1 | c2, 3, row) + checkEvaluation(c1 ^ c2, 3, row) + checkEvaluation(~c1, -2, row) + } + + 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]) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala new file mode 100644 index 0000000000000..eededbfeef4f5 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -0,0 +1,532 @@ +/* + * 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.catalyst.expressions + +import java.sql.{Timestamp, Date} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types._ + +/** + * Test suite for data type casting expression [[Cast]]. + */ +class CastSuite extends SparkFunSuite with ExpressionEvalHelper { + + private def cast(v: Any, targetType: DataType): Cast = { + v match { + case lit: Expression => Cast(lit, targetType) + case _ => Cast(Literal(v), targetType) + } + } + + // expected cannot be null + private def checkCast(v: Any, expected: Any): Unit = { + checkEvaluation(cast(v, Literal(expected).dataType), expected) + } + + test("cast from int") { + checkCast(0, false) + checkCast(1, true) + checkCast(5, true) + checkCast(1, 1.toByte) + checkCast(1, 1.toShort) + checkCast(1, 1) + checkCast(1, 1.toLong) + checkCast(1, 1.0f) + checkCast(1, 1.0) + checkCast(123, "123") + + checkEvaluation(cast(123, DecimalType.Unlimited), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 1)), null) + checkEvaluation(cast(123, DecimalType(2, 0)), null) + } + + test("cast from long") { + checkCast(0L, false) + checkCast(1L, true) + checkCast(5L, true) + checkCast(1L, 1.toByte) + checkCast(1L, 1.toShort) + checkCast(1L, 1) + checkCast(1L, 1.toLong) + checkCast(1L, 1.0f) + checkCast(1L, 1.0) + checkCast(123L, "123") + + checkEvaluation(cast(123L, DecimalType.Unlimited), Decimal(123)) + checkEvaluation(cast(123L, DecimalType(3, 0)), Decimal(123)) + checkEvaluation(cast(123L, DecimalType(3, 1)), Decimal(123.0)) + + // TODO: Fix the following bug and re-enable it. + //checkEvaluation(cast(123L, DecimalType(2, 0)), null) + } + + test("cast from boolean") { + checkEvaluation(cast(true, IntegerType), 1) + checkEvaluation(cast(false, IntegerType), 0) + checkEvaluation(cast(true, StringType), "true") + checkEvaluation(cast(false, StringType), "false") + checkEvaluation(cast(cast(1, BooleanType), IntegerType), 1) + checkEvaluation(cast(cast(0, BooleanType), IntegerType), 0) + } + + test("cast from int 2") { + checkEvaluation(cast(1, LongType), 1.toLong) + checkEvaluation(cast(cast(1000, TimestampType), LongType), 1.toLong) + checkEvaluation(cast(cast(-1200, TimestampType), LongType), -2.toLong) + + checkEvaluation(cast(123, DecimalType.Unlimited), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) + checkEvaluation(cast(123, DecimalType(3, 1)), null) + checkEvaluation(cast(123, DecimalType(2, 0)), null) + } + + test("cast from float") { + + } + + test("cast from double") { + checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + } + + test("cast from string") { + assert(cast("abcdef", StringType).nullable === false) + assert(cast("abcdef", BinaryType).nullable === false) + assert(cast("abcdef", BooleanType).nullable === false) + assert(cast("abcdef", TimestampType).nullable === true) + assert(cast("abcdef", LongType).nullable === true) + assert(cast("abcdef", IntegerType).nullable === true) + assert(cast("abcdef", ShortType).nullable === true) + assert(cast("abcdef", ByteType).nullable === true) + assert(cast("abcdef", DecimalType.Unlimited).nullable === true) + assert(cast("abcdef", DecimalType(4, 2)).nullable === true) + assert(cast("abcdef", DoubleType).nullable === true) + assert(cast("abcdef", FloatType).nullable === true) + } + + test("data type casting") { + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = Timestamp.valueOf(nts) + + checkEvaluation(cast("abdef", StringType), "abdef") + checkEvaluation(cast("abdef", DecimalType.Unlimited), null) + checkEvaluation(cast("abdef", TimestampType), null) + checkEvaluation(cast("12.65", DecimalType.Unlimited), Decimal(12.65)) + + checkEvaluation(cast(cast(sd, DateType), StringType), sd) + checkEvaluation(cast(cast(d, StringType), DateType), 0) + checkEvaluation(cast(cast(nts, TimestampType), StringType), nts) + checkEvaluation(cast(cast(ts, StringType), TimestampType), ts) + + // all convert to string type to check + checkEvaluation(cast(cast(cast(nts, TimestampType), DateType), StringType), sd) + checkEvaluation(cast(cast(cast(ts, DateType), TimestampType), StringType), zts) + + checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") + + checkEvaluation(cast(cast(cast(cast( + cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType), + 5.toLong) + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), + DecimalType.Unlimited), LongType), StringType), ShortType), + 0.toShort) + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", TimestampType),ByteType), + DecimalType.Unlimited), LongType), StringType), ShortType), + null) + checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.Unlimited), + ByteType), TimestampType), LongType), StringType), ShortType), + 0.toShort) + + checkEvaluation(cast("23", DoubleType), 23d) + checkEvaluation(cast("23", IntegerType), 23) + checkEvaluation(cast("23", FloatType), 23f) + checkEvaluation(cast("23", DecimalType.Unlimited), Decimal(23)) + checkEvaluation(cast("23", ByteType), 23.toByte) + checkEvaluation(cast("23", ShortType), 23.toShort) + checkEvaluation(cast("2012-12-11", DoubleType), null) + checkEvaluation(cast(123, IntegerType), 123) + + + checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) + } + + test("cast and add") { + checkEvaluation(Add(Literal(23d), cast(true, DoubleType)), 24d) + checkEvaluation(Add(Literal(23), cast(true, IntegerType)), 24) + checkEvaluation(Add(Literal(23f), cast(true, FloatType)), 24f) + checkEvaluation(Add(Literal(Decimal(23)), cast(true, DecimalType.Unlimited)), Decimal(24)) + checkEvaluation(Add(Literal(23.toByte), cast(true, ByteType)), 24.toByte) + checkEvaluation(Add(Literal(23.toShort), cast(true, ShortType)), 24.toShort) + } + + test("casting to fixed-precision decimals") { + // Overflow and rounding for casting to fixed-precision decimals: + // - Values should round with HALF_UP mode by default when you lower scale + // - Values that would overflow the target precision should turn into null + // - Because of this, casts to fixed-precision decimals should be nullable + + assert(cast(123, DecimalType.Unlimited).nullable === false) + assert(cast(10.03f, DecimalType.Unlimited).nullable === true) + assert(cast(10.03, DecimalType.Unlimited).nullable === true) + assert(cast(Decimal(10.03), DecimalType.Unlimited).nullable === false) + + assert(cast(123, DecimalType(2, 1)).nullable === true) + assert(cast(10.03f, DecimalType(2, 1)).nullable === true) + assert(cast(10.03, DecimalType(2, 1)).nullable === true) + assert(cast(Decimal(10.03), DecimalType(2, 1)).nullable === true) + + + checkEvaluation(cast(10.03, DecimalType.Unlimited), Decimal(10.03)) + checkEvaluation(cast(10.03, DecimalType(4, 2)), Decimal(10.03)) + checkEvaluation(cast(10.03, DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(10.03, DecimalType(2, 0)), Decimal(10)) + checkEvaluation(cast(10.03, DecimalType(1, 0)), null) + checkEvaluation(cast(10.03, DecimalType(2, 1)), null) + checkEvaluation(cast(10.03, DecimalType(3, 2)), null) + checkEvaluation(cast(Decimal(10.03), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(Decimal(10.03), DecimalType(3, 2)), null) + + checkEvaluation(cast(10.05, DecimalType.Unlimited), Decimal(10.05)) + checkEvaluation(cast(10.05, DecimalType(4, 2)), Decimal(10.05)) + checkEvaluation(cast(10.05, DecimalType(3, 1)), Decimal(10.1)) + checkEvaluation(cast(10.05, DecimalType(2, 0)), Decimal(10)) + checkEvaluation(cast(10.05, DecimalType(1, 0)), null) + checkEvaluation(cast(10.05, DecimalType(2, 1)), null) + checkEvaluation(cast(10.05, DecimalType(3, 2)), null) + checkEvaluation(cast(Decimal(10.05), DecimalType(3, 1)), Decimal(10.1)) + checkEvaluation(cast(Decimal(10.05), DecimalType(3, 2)), null) + + checkEvaluation(cast(9.95, DecimalType(3, 2)), Decimal(9.95)) + checkEvaluation(cast(9.95, DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(9.95, DecimalType(2, 0)), Decimal(10)) + checkEvaluation(cast(9.95, DecimalType(2, 1)), null) + checkEvaluation(cast(9.95, DecimalType(1, 0)), null) + checkEvaluation(cast(Decimal(9.95), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(cast(Decimal(9.95), DecimalType(1, 0)), null) + + checkEvaluation(cast(-9.95, DecimalType(3, 2)), Decimal(-9.95)) + checkEvaluation(cast(-9.95, DecimalType(3, 1)), Decimal(-10.0)) + checkEvaluation(cast(-9.95, DecimalType(2, 0)), Decimal(-10)) + checkEvaluation(cast(-9.95, DecimalType(2, 1)), null) + checkEvaluation(cast(-9.95, DecimalType(1, 0)), null) + checkEvaluation(cast(Decimal(-9.95), DecimalType(3, 1)), Decimal(-10.0)) + checkEvaluation(cast(Decimal(-9.95), DecimalType(1, 0)), null) + + checkEvaluation(cast(Double.NaN, DecimalType.Unlimited), null) + checkEvaluation(cast(1.0 / 0.0, DecimalType.Unlimited), null) + checkEvaluation(cast(Float.NaN, DecimalType.Unlimited), null) + checkEvaluation(cast(1.0f / 0.0f, DecimalType.Unlimited), null) + + checkEvaluation(cast(Double.NaN, DecimalType(2, 1)), null) + checkEvaluation(cast(1.0 / 0.0, DecimalType(2, 1)), null) + checkEvaluation(cast(Float.NaN, DecimalType(2, 1)), null) + checkEvaluation(cast(1.0f / 0.0f, DecimalType(2, 1)), null) + } + + test("cast from date") { + val d = Date.valueOf("1970-01-01") + checkEvaluation(cast(d, ShortType), null) + checkEvaluation(cast(d, IntegerType), null) + checkEvaluation(cast(d, LongType), null) + checkEvaluation(cast(d, FloatType), null) + checkEvaluation(cast(d, DoubleType), null) + checkEvaluation(cast(d, DecimalType.Unlimited), null) + checkEvaluation(cast(d, DecimalType(10, 2)), null) + checkEvaluation(cast(d, StringType), "1970-01-01") + checkEvaluation(cast(cast(d, TimestampType), StringType), "1970-01-01 00:00:00") + } + + test("cast from timestamp") { + val millis = 15 * 1000 + 2 + val seconds = millis * 1000 + 2 + val ts = new Timestamp(millis) + val tss = new Timestamp(seconds) + checkEvaluation(cast(ts, ShortType), 15.toShort) + checkEvaluation(cast(ts, IntegerType), 15) + checkEvaluation(cast(ts, LongType), 15.toLong) + checkEvaluation(cast(ts, FloatType), 15.002f) + checkEvaluation(cast(ts, DoubleType), 15.002) + checkEvaluation(cast(cast(tss, ShortType), TimestampType), ts) + checkEvaluation(cast(cast(tss, IntegerType), TimestampType), ts) + checkEvaluation(cast(cast(tss, LongType), TimestampType), ts) + checkEvaluation( + cast(cast(millis.toFloat / 1000, TimestampType), FloatType), + millis.toFloat / 1000) + checkEvaluation( + cast(cast(millis.toDouble / 1000, TimestampType), DoubleType), + millis.toDouble / 1000) + checkEvaluation( + cast(cast(Decimal(1), TimestampType), DecimalType.Unlimited), + Decimal(1)) + + // A test for higher precision than millis + checkEvaluation(cast(cast(0.00000001, TimestampType), DoubleType), 0.00000001) + + checkEvaluation(cast(Double.NaN, TimestampType), null) + checkEvaluation(cast(1.0 / 0.0, TimestampType), null) + checkEvaluation(cast(Float.NaN, TimestampType), null) + checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + } + + test("cast from array") { + val array = Literal.create(Seq("123", "abc", "", null), + ArrayType(StringType, containsNull = true)) + val array_notNull = Literal.create(Seq("123", "abc", ""), + ArrayType(StringType, containsNull = false)) + + { + val ret = cast(array, ArrayType(IntegerType, containsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Seq(123, null, null, null)) + } + { + val ret = cast(array, ArrayType(IntegerType, containsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(array, ArrayType(BooleanType, containsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Seq(true, true, false, null)) + } + { + val ret = cast(array, ArrayType(BooleanType, containsNull = false)) + assert(ret.resolved === false) + } + + { + val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Seq(123, null, null)) + } + { + val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Seq(true, true, false)) + } + { + val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = false)) + assert(ret.resolved === true) + checkEvaluation(ret, Seq(true, true, false)) + } + + { + val ret = cast(array, IntegerType) + assert(ret.resolved === false) + } + } + + test("cast from map") { + val map = Literal.create( + Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null), + MapType(StringType, StringType, valueContainsNull = true)) + val map_notNull = Literal.create( + Map("a" -> "123", "b" -> "abc", "c" -> ""), + MapType(StringType, StringType, valueContainsNull = false)) + + { + val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null)) + } + { + val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(map, MapType(StringType, BooleanType, valueContainsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Map("a" -> true, "b" -> true, "c" -> false, "d" -> null)) + } + { + val ret = cast(map, MapType(StringType, BooleanType, valueContainsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(map, MapType(IntegerType, StringType, valueContainsNull = true)) + assert(ret.resolved === false) + } + + { + val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null)) + } + { + val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true)) + assert(ret.resolved === true) + checkEvaluation(ret, Map("a" -> true, "b" -> true, "c" -> false)) + } + { + val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false)) + assert(ret.resolved === true) + checkEvaluation(ret, Map("a" -> true, "b" -> true, "c" -> false)) + } + { + val ret = cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true)) + assert(ret.resolved === false) + } + + { + val ret = cast(map, IntegerType) + assert(ret.resolved === false) + } + } + + test("cast from struct") { + val struct = Literal.create( + Row("123", "abc", "", null), + StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true), + StructField("c", StringType, nullable = true), + StructField("d", StringType, nullable = true)))) + val struct_notNull = Literal.create( + Row("123", "abc", ""), + StructType(Seq( + StructField("a", StringType, nullable = false), + StructField("b", StringType, nullable = false), + StructField("c", StringType, nullable = false)))) + + { + val ret = cast(struct, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true), + StructField("d", IntegerType, nullable = true)))) + assert(ret.resolved === true) + checkEvaluation(ret, Row(123, null, null, null)) + } + { + val ret = cast(struct, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = true)))) + assert(ret.resolved === false) + } + { + val ret = cast(struct, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = true), + StructField("d", BooleanType, nullable = true)))) + assert(ret.resolved === true) + checkEvaluation(ret, Row(true, true, false, null)) + } + { + val ret = cast(struct, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = false), + StructField("d", BooleanType, nullable = true)))) + assert(ret.resolved === false) + } + + { + val ret = cast(struct_notNull, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true)))) + assert(ret.resolved === true) + checkEvaluation(ret, Row(123, null, null)) + } + { + val ret = cast(struct_notNull, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false)))) + assert(ret.resolved === false) + } + { + val ret = cast(struct_notNull, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = true)))) + assert(ret.resolved === true) + checkEvaluation(ret, Row(true, true, false)) + } + { + val ret = cast(struct_notNull, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = false)))) + assert(ret.resolved === true) + checkEvaluation(ret, Row(true, true, false)) + } + + { + val ret = cast(struct, StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true), + StructField("c", StringType, nullable = true)))) + assert(ret.resolved === false) + } + { + val ret = cast(struct, IntegerType) + assert(ret.resolved === false) + } + } + + test("complex casting") { + val complex = Literal.create( + Row( + Seq("123", "abc", ""), + Map("a" -> "123", "b" -> "abc", "c" -> ""), + Row(0)), + StructType(Seq( + StructField("a", + ArrayType(StringType, containsNull = false), nullable = true), + StructField("m", + MapType(StringType, StringType, valueContainsNull = false), nullable = true), + StructField("s", + StructType(Seq( + StructField("i", IntegerType, nullable = true))))))) + + val ret = cast(complex, StructType(Seq( + StructField("a", + ArrayType(IntegerType, containsNull = true), nullable = true), + StructField("m", + MapType(StringType, BooleanType, valueContainsNull = false), nullable = true), + StructField("s", + StructType(Seq( + StructField("l", LongType, nullable = true))))))) + + assert(ret.resolved === true) + checkEvaluation(ret, Row( + Seq(123, null, null), + Map("a" -> true, "b" -> true, "c" -> false), + Row(0L))) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala similarity index 94% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 371a73181dad7..481b335d15dfd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ /** * Additional tests for code generation. */ -class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { +class CodeGenerationSuite extends SparkFunSuite { test("multithreaded eval") { import scala.concurrent._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala new file mode 100644 index 0000000000000..f151dd2a47f78 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -0,0 +1,122 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types._ + + +class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("CreateStruct") { + val row = Row(1, 2, 3) + val c1 = 'a.int.at(0).as("a") + val c3 = 'c.int.at(2).as("c") + checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row) + } + + test("complex type") { + val row = create_row( + "^Ba*n", // 0 + null.asInstanceOf[UTF8String], // 1 + create_row("aa", "bb"), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + ) + + val typeS = StructType( + StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil + ) + val typeMap = MapType(StringType, StringType) + val typeArray = ArrayType(StringType) + + checkEvaluation(GetMapValue(BoundReference(3, typeMap, true), + Literal("aa")), "bb", row) + checkEvaluation(GetMapValue(Literal.create(null, typeMap), Literal("aa")), null, row) + checkEvaluation( + GetMapValue(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) + checkEvaluation(GetMapValue(BoundReference(3, typeMap, true), + Literal.create(null, StringType)), null, row) + + checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true), + Literal(1)), "bb", row) + checkEvaluation(GetArrayItem(Literal.create(null, typeArray), Literal(1)), null, row) + checkEvaluation( + GetArrayItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) + checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true), + Literal.create(null, IntegerType)), null, row) + + def getStructField(expr: Expression, fieldName: String): ExtractValue = { + expr.dataType match { + case StructType(fields) => + val field = fields.find(_.name == fieldName).get + GetStructField(expr, field, fields.indexOf(field)) + } + } + + def quickResolve(u: UnresolvedExtractValue): ExtractValue = { + ExtractValue(u.child, u.extraction, _ == _) + } + + checkEvaluation(getStructField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) + checkEvaluation(getStructField(Literal.create(null, typeS), "a"), null, row) + + val typeS_notNullable = StructType( + StructField("a", StringType, nullable = false) + :: StructField("b", StringType, nullable = false) :: Nil + ) + + assert(getStructField(BoundReference(2, typeS, nullable = true), "a").nullable === true) + assert(getStructField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable + === false) + + assert(getStructField(Literal.create(null, typeS), "a").nullable === true) + assert(getStructField(Literal.create(null, typeS_notNullable), "a").nullable === true) + + checkEvaluation(quickResolve('c.map(typeMap).at(3).getItem("aa")), "bb", row) + checkEvaluation(quickResolve('c.array(typeArray.elementType).at(4).getItem(1)), "bb", row) + checkEvaluation(quickResolve('c.struct(typeS).at(2).getField("a")), "aa", row) + } + + test("error message of ExtractValue") { + val structType = StructType(StructField("a", StringType, true) :: Nil) + val arrayStructType = ArrayType(structType) + val arrayType = ArrayType(StringType) + val otherType = StringType + + def checkErrorMessage( + childDataType: DataType, + fieldDataType: DataType, + errorMesage: String): Unit = { + val e = intercept[org.apache.spark.sql.AnalysisException] { + ExtractValue( + Literal.create(null, childDataType), + Literal.create(null, fieldDataType), + _ == _) + } + assert(e.getMessage().contains(errorMesage)) + } + + checkErrorMessage(structType, IntegerType, "Field name should be String Literal") + checkErrorMessage(arrayStructType, BooleanType, "Field name should be String Literal") + checkErrorMessage(arrayType, StringType, "Array index should be integral type") + checkErrorMessage(otherType, StringType, "Can't extract value from") + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala new file mode 100644 index 0000000000000..152c4e4111244 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -0,0 +1,96 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types.{IntegerType, BooleanType} + + +class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("case when") { + val row = create_row(null, false, true, "a", "b", "c") + val c1 = 'a.boolean.at(0) + val c2 = 'a.boolean.at(1) + val c3 = 'a.boolean.at(2) + val c4 = 'a.string.at(3) + val c5 = 'a.string.at(4) + val c6 = 'a.string.at(5) + + checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row) + checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row) + + checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row) + checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row) + checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row) + + assert(CaseWhen(Seq(c2, c4, c6)).nullable === true) + assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true) + assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true) + + val c4_notNull = 'a.boolean.notNull.at(3) + val c5_notNull = 'a.boolean.notNull.at(4) + val c6_notNull = 'a.boolean.notNull.at(5) + + assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false) + assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true) + + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false) + assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true) + + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) + } + + test("case key when") { + val row = create_row(null, 1, 2, "a", "b", "c") + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.string.at(3) + val c5 = 'a.string.at(4) + val c6 = 'a.string.at(5) + + val literalNull = Literal.create(null, IntegerType) + val literalInt = Literal(1) + val literalString = Literal("a") + + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row) + checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row) + checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row) + checkEvaluation(CaseKeyWhen(c4, Seq(c6, c3, c5, c2, Literal(3))), 3, row) + + checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row) + checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row) + checkEvaluation(CaseKeyWhen(c6, Seq(c5, c2, c4, c3)), null, row) + checkEvaluation(CaseKeyWhen(literalNull, Seq(c2, c5, c1, c6)), "c", row) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala new file mode 100644 index 0000000000000..87a92b87962f8 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -0,0 +1,134 @@ +/* + * 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.catalyst.expressions + +import org.scalactic.TripleEqualsSupport.Spread +import org.scalatest.Matchers._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateProjection, GenerateMutableProjection} + +/** + * A few helper functions for expression evaluation testing. Mixin this trait to use them. + */ +trait ExpressionEvalHelper { + self: SparkFunSuite => + + protected def create_row(values: Any*): Row = { + new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray) + } + + protected def checkEvaluation( + expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + checkEvaluationWithoutCodegen(expression, expected, inputRow) + checkEvaluationWithGeneratedMutableProjection(expression, expected, inputRow) + checkEvaluationWithGeneratedProjection(expression, expected, inputRow) + } + + protected def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { + expression.eval(inputRow) + } + + protected def checkEvaluationWithoutCodegen( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + if (actual != expected) { + val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect evaluation (codegen off): $expression, " + + s"actual: $actual, " + + s"expected: $expected$input") + } + } + + protected def checkEvaluationWithGeneratedMutableProjection( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + + val plan = try { + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)() + } catch { + case e: Throwable => + val ctx = GenerateProjection.newCodeGenContext() + val evaluated = expression.gen(ctx) + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code} + |$e + """.stripMargin) + } + + val actual = plan(inputRow).apply(0) + if (actual != expected) { + val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + protected def checkEvaluationWithGeneratedProjection( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val ctx = GenerateProjection.newCodeGenContext() + lazy val evaluated = expression.gen(ctx) + + val plan = try { + GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil) + } catch { + case e: Throwable => + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code} + |$e + """.stripMargin) + } + + val actual = plan(inputRow) + val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) + if (actual.hashCode() != expectedRow.hashCode()) { + fail( + s""" + |Mismatched hashCodes for values: $actual, $expectedRow + |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()} + |Expressions: $expression + |Code: $evaluated + """.stripMargin) + } + if (actual != expectedRow) { + val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + protected def checkDoubleEvaluation( + expression: Expression, + expected: Spread[Double], + inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + actual.asInstanceOf[Double] shouldBe expected + } +} 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 deleted file mode 100644 index eea2edc323eea..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ /dev/null @@ -1,1461 +0,0 @@ -/* - * 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.catalyst.expressions - -import java.sql.{Date, Timestamp} - -import scala.collection.immutable.HashSet - -import org.scalactic.TripleEqualsSupport.Spread -import org.scalatest.Matchers._ - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateProjection, GenerateMutableProjection} -import org.apache.spark.sql.catalyst.expressions.mathfuncs._ -import org.apache.spark.sql.catalyst.util.DateUtils -import org.apache.spark.sql.types._ - - -class ExpressionEvaluationBaseSuite extends SparkFunSuite { - - def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { - checkEvaluationWithoutCodegen(expression, expected, inputRow) - checkEvaluationWithGeneratedMutableProjection(expression, expected, inputRow) - checkEvaluationWithGeneratedProjection(expression, expected, inputRow) - } - - def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { - expression.eval(inputRow) - } - - def checkEvaluationWithoutCodegen( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { - val actual = try evaluate(expression, inputRow) catch { - case e: Exception => fail(s"Exception evaluating $expression", e) - } - if (actual != expected) { - val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } - - def checkEvaluationWithGeneratedMutableProjection( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { - - val plan = try { - GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)() - } catch { - case e: Throwable => - val ctx = GenerateProjection.newCodeGenContext() - val evaluated = expression.gen(ctx) - fail( - s""" - |Code generation of $expression failed: - |${evaluated.code} - |$e - """.stripMargin) - } - - val actual = plan(inputRow).apply(0) - if (actual != expected) { - val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } - - def checkEvaluationWithGeneratedProjection( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { - val ctx = GenerateProjection.newCodeGenContext() - lazy val evaluated = expression.gen(ctx) - - val plan = try { - GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil) - } catch { - case e: Throwable => - fail( - s""" - |Code generation of $expression failed: - |${evaluated.code} - |$e - """.stripMargin) - } - - val actual = plan(inputRow) - val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) - if (actual.hashCode() != expectedRow.hashCode()) { - fail( - s""" - |Mismatched hashCodes for values: $actual, $expectedRow - |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()} - |Expressions: ${expression} - |Code: ${evaluated} - """.stripMargin) - } - if (actual != expectedRow) { - val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } - - def checkDoubleEvaluation( - expression: Expression, - expected: Spread[Double], - inputRow: Row = EmptyRow): Unit = { - val actual = try evaluate(expression, inputRow) catch { - case e: Exception => fail(s"Exception evaluating $expression", e) - } - actual.asInstanceOf[Double] shouldBe expected - } -} - -class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { - - def create_row(values: Any*): Row = { - new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray) - } - - test("literals") { - checkEvaluation(Literal(1), 1) - checkEvaluation(Literal(true), true) - checkEvaluation(Literal(false), false) - checkEvaluation(Literal(0L), 0L) - List(0.0, -0.0, Double.NegativeInfinity, Double.PositiveInfinity).foreach { - d => { - checkEvaluation(Literal(d), d) - checkEvaluation(Literal(d.toFloat), d.toFloat) - } - } - checkEvaluation(Literal("test"), "test") - checkEvaluation(Literal.create(null, StringType), null) - 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]) - } - - // scalastyle:off - /** - * Checks for three-valued-logic. Based on: - * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 - * I.e. in flat cpo "False -> Unknown -> True", - * OR is lowest upper bound, - * AND is greatest lower bound. - * p q p OR q p AND q p = q - * True True True True True - * True False True False False - * True Unknown True Unknown Unknown - * False True True False False - * False False False False True - * False Unknown Unknown False Unknown - * Unknown True True Unknown Unknown - * Unknown False Unknown False Unknown - * Unknown Unknown Unknown Unknown Unknown - * - * p NOT p - * True False - * False True - * Unknown Unknown - */ - // scalastyle:on - val notTrueTable = - (true, false) :: - (false, true) :: - (null, null) :: Nil - - test("3VL Not") { - notTrueTable.foreach { - case (v, answer) => - checkEvaluation(!Literal.create(v, BooleanType), answer) - } - } - - booleanLogicTest("AND", _ && _, - (true, true, true) :: - (true, false, false) :: - (true, null, null) :: - (false, true, false) :: - (false, false, false) :: - (false, null, false) :: - (null, true, null) :: - (null, false, false) :: - (null, null, null) :: Nil) - - booleanLogicTest("OR", _ || _, - (true, true, true) :: - (true, false, true) :: - (true, null, true) :: - (false, true, true) :: - (false, false, false) :: - (false, null, null) :: - (null, true, true) :: - (null, false, null) :: - (null, null, null) :: Nil) - - booleanLogicTest("=", _ === _, - (true, true, true) :: - (true, false, false) :: - (true, null, null) :: - (false, true, false) :: - (false, false, true) :: - (false, null, null) :: - (null, true, null) :: - (null, false, null) :: - (null, null, null) :: Nil) - - def booleanLogicTest( - name: String, - op: (Expression, Expression) => Expression, - truthTable: Seq[(Any, Any, Any)]) { - test(s"3VL $name") { - truthTable.foreach { - case (l, r, answer) => - val expr = op(Literal.create(l, BooleanType), Literal.create(r, BooleanType)) - checkEvaluation(expr, answer) - } - } - } - - test("IN") { - checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) - checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) - checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) - checkEvaluation( - In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), - true) - } - - test("Divide") { - checkEvaluation(Divide(Literal(2), Literal(1)), 2) - checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) - checkEvaluation(Divide(Literal(1), Literal(2)), 0) - checkEvaluation(Divide(Literal(1), Literal(0)), null) - checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null) - checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null) - checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) - checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), - null) - } - - test("Remainder") { - checkEvaluation(Remainder(Literal(2), Literal(1)), 0) - checkEvaluation(Remainder(Literal(1.0), Literal(2.0)), 1.0) - checkEvaluation(Remainder(Literal(1), Literal(2)), 1) - checkEvaluation(Remainder(Literal(1), Literal(0)), null) - checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null) - checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) - checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), - null) - } - - test("INSET") { - val hS = HashSet[Any]() + 1 + 2 - val nS = HashSet[Any]() + 1 + 2 + null - val one = Literal(1) - val two = Literal(2) - val three = Literal(3) - val nl = Literal(null) - val s = Seq(one, two) - val nullS = Seq(one, two, null) - checkEvaluation(InSet(one, hS), true) - checkEvaluation(InSet(two, hS), true) - checkEvaluation(InSet(two, nS), true) - checkEvaluation(InSet(nl, nS), true) - checkEvaluation(InSet(three, hS), false) - checkEvaluation(InSet(three, nS), false) - checkEvaluation(InSet(one, hS) && InSet(two, hS), true) - } - - test("MaxOf") { - checkEvaluation(MaxOf(1, 2), 2) - checkEvaluation(MaxOf(2, 1), 2) - checkEvaluation(MaxOf(1L, 2L), 2L) - checkEvaluation(MaxOf(2L, 1L), 2L) - - checkEvaluation(MaxOf(Literal.create(null, IntegerType), 2), 2) - checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) - } - - test("MinOf") { - checkEvaluation(MinOf(1, 2), 1) - checkEvaluation(MinOf(2, 1), 1) - checkEvaluation(MinOf(1L, 2L), 1L) - checkEvaluation(MinOf(2L, 1L), 1L) - - checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1) - checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1) - } - - test("LIKE literal Regular Expression") { - checkEvaluation(Literal.create(null, StringType).like("a"), null) - checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) - checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null) - checkEvaluation("abdef" like "abdef", true) - checkEvaluation("a_%b" like "a\\__b", true) - checkEvaluation("addb" like "a_%b", true) - checkEvaluation("addb" like "a\\__b", false) - checkEvaluation("addb" like "a%\\%b", false) - checkEvaluation("a_%b" like "a%\\%b", true) - checkEvaluation("addb" like "a%", true) - checkEvaluation("addb" like "**", false) - checkEvaluation("abc" like "a%", true) - checkEvaluation("abc" like "b%", false) - checkEvaluation("abc" like "bc%", false) - checkEvaluation("a\nb" like "a_b", true) - checkEvaluation("ab" like "a%b", true) - checkEvaluation("a\nb" like "a%b", true) - } - - test("LIKE Non-literal Regular Expression") { - val regEx = 'a.string.at(0) - checkEvaluation("abcd" like regEx, null, create_row(null)) - checkEvaluation("abdef" like regEx, true, create_row("abdef")) - checkEvaluation("a_%b" like regEx, true, create_row("a\\__b")) - checkEvaluation("addb" like regEx, true, create_row("a_%b")) - checkEvaluation("addb" like regEx, false, create_row("a\\__b")) - checkEvaluation("addb" like regEx, false, create_row("a%\\%b")) - checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b")) - checkEvaluation("addb" like regEx, true, create_row("a%")) - checkEvaluation("addb" like regEx, false, create_row("**")) - checkEvaluation("abc" like regEx, true, create_row("a%")) - checkEvaluation("abc" like regEx, false, create_row("b%")) - checkEvaluation("abc" like regEx, false, create_row("bc%")) - checkEvaluation("a\nb" like regEx, true, create_row("a_b")) - checkEvaluation("ab" like regEx, true, create_row("a%b")) - checkEvaluation("a\nb" like regEx, true, create_row("a%b")) - - checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%")) - } - - test("RLIKE literal Regular Expression") { - checkEvaluation(Literal.create(null, StringType) rlike "abdef", null) - checkEvaluation("abdef" rlike Literal.create(null, StringType), null) - checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null) - checkEvaluation("abdef" rlike "abdef", true) - checkEvaluation("abbbbc" rlike "a.*c", true) - - checkEvaluation("fofo" rlike "^fo", true) - checkEvaluation("fo\no" rlike "^fo\no$", true) - checkEvaluation("Bn" rlike "^Ba*n", true) - checkEvaluation("afofo" rlike "fo", true) - checkEvaluation("afofo" rlike "^fo", false) - checkEvaluation("Baan" rlike "^Ba?n", false) - checkEvaluation("axe" rlike "pi|apa", false) - checkEvaluation("pip" rlike "^(pi)*$", false) - - checkEvaluation("abc" rlike "^ab", true) - checkEvaluation("abc" rlike "^bc", false) - checkEvaluation("abc" rlike "^ab", true) - checkEvaluation("abc" rlike "^bc", false) - - intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike "**") - } - } - - test("RLIKE Non-literal Regular Expression") { - val regEx = 'a.string.at(0) - checkEvaluation("abdef" rlike regEx, true, create_row("abdef")) - checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c")) - checkEvaluation("fofo" rlike regEx, true, create_row("^fo")) - checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$")) - checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n")) - - intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike regEx, create_row("**")) - } - } - - test("data type casting") { - - val sd = "1970-01-01" - val d = Date.valueOf(sd) - val zts = sd + " 00:00:00" - val sts = sd + " 00:00:02" - val nts = sts + ".1" - val ts = Timestamp.valueOf(nts) - - checkEvaluation("abdef" cast StringType, "abdef") - checkEvaluation("abdef" cast DecimalType.Unlimited, null) - checkEvaluation("abdef" cast TimestampType, null) - checkEvaluation("12.65" cast DecimalType.Unlimited, Decimal(12.65)) - - checkEvaluation(Literal(1) cast LongType, 1.toLong) - checkEvaluation(Cast(Literal(1000) cast TimestampType, LongType), 1.toLong) - checkEvaluation(Cast(Literal(-1200) cast TimestampType, LongType), -2.toLong) - checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) - checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) - - checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd) - checkEvaluation(Cast(Literal(d) cast StringType, DateType), 0) - checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts) - checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) - // all convert to string type to check - checkEvaluation( - Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd) - checkEvaluation( - Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), zts) - - checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") - - checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType), - 5.toLong) - checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast - ByteType, TimestampType), DecimalType.Unlimited), LongType), StringType), ShortType), - 0.toShort) - checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast - TimestampType, ByteType), DecimalType.Unlimited), LongType), StringType), ShortType), null) - checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast - DecimalType.Unlimited, ByteType), TimestampType), LongType), StringType), ShortType), - 0.toShort) - checkEvaluation(Literal(true) cast IntegerType, 1) - checkEvaluation(Literal(false) cast IntegerType, 0) - checkEvaluation(Literal(true) cast StringType, "true") - checkEvaluation(Literal(false) cast StringType, "false") - checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) - checkEvaluation(Cast(Literal(0) cast BooleanType, IntegerType), 0) - checkEvaluation("23" cast DoubleType, 23d) - checkEvaluation("23" cast IntegerType, 23) - checkEvaluation("23" cast FloatType, 23f) - checkEvaluation("23" cast DecimalType.Unlimited, Decimal(23)) - checkEvaluation("23" cast ByteType, 23.toByte) - checkEvaluation("23" cast ShortType, 23.toShort) - checkEvaluation("2012-12-11" cast DoubleType, null) - checkEvaluation(Literal(123) cast IntegerType, 123) - - checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24d) - checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) - checkEvaluation(Literal(23f) + Cast(true, FloatType), 24f) - checkEvaluation(Literal(Decimal(23)) + Cast(true, DecimalType.Unlimited), Decimal(24)) - checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24.toByte) - checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24.toShort) - - intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} - - assert(("abcdef" cast StringType).nullable === false) - assert(("abcdef" cast BinaryType).nullable === false) - assert(("abcdef" cast BooleanType).nullable === false) - assert(("abcdef" cast TimestampType).nullable === true) - assert(("abcdef" cast LongType).nullable === true) - assert(("abcdef" cast IntegerType).nullable === true) - assert(("abcdef" cast ShortType).nullable === true) - assert(("abcdef" cast ByteType).nullable === true) - assert(("abcdef" cast DecimalType.Unlimited).nullable === true) - assert(("abcdef" cast DecimalType(4, 2)).nullable === true) - assert(("abcdef" cast DoubleType).nullable === true) - assert(("abcdef" cast FloatType).nullable === true) - - checkEvaluation(Cast(Literal.create(null, IntegerType), ShortType), null) - } - - test("date") { - val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01")) - val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02")) - checkEvaluation(Literal(d1) < Literal(d2), true) - } - - test("casting to fixed-precision decimals") { - // Overflow and rounding for casting to fixed-precision decimals: - // - Values should round with HALF_UP mode by default when you lower scale - // - Values that would overflow the target precision should turn into null - // - Because of this, casts to fixed-precision decimals should be nullable - - assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false) - assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === true) - assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === true) - assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false) - - assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true) - assert(Cast(Literal(10.03f), DecimalType(2, 1)).nullable === true) - assert(Cast(Literal(10.03), DecimalType(2, 1)).nullable === true) - assert(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable === true) - - checkEvaluation(Cast(Literal(123), DecimalType.Unlimited), Decimal(123)) - checkEvaluation(Cast(Literal(123), DecimalType(3, 0)), Decimal(123)) - checkEvaluation(Cast(Literal(123), DecimalType(3, 1)), null) - checkEvaluation(Cast(Literal(123), DecimalType(2, 0)), null) - - checkEvaluation(Cast(Literal(10.03), DecimalType.Unlimited), Decimal(10.03)) - checkEvaluation(Cast(Literal(10.03), DecimalType(4, 2)), Decimal(10.03)) - checkEvaluation(Cast(Literal(10.03), DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(Cast(Literal(10.03), DecimalType(2, 0)), Decimal(10)) - checkEvaluation(Cast(Literal(10.03), DecimalType(1, 0)), null) - checkEvaluation(Cast(Literal(10.03), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(10.03), DecimalType(3, 2)), null) - checkEvaluation(Cast(Literal(Decimal(10.03)), DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(Cast(Literal(Decimal(10.03)), DecimalType(3, 2)), null) - - checkEvaluation(Cast(Literal(10.05), DecimalType.Unlimited), Decimal(10.05)) - checkEvaluation(Cast(Literal(10.05), DecimalType(4, 2)), Decimal(10.05)) - checkEvaluation(Cast(Literal(10.05), DecimalType(3, 1)), Decimal(10.1)) - checkEvaluation(Cast(Literal(10.05), DecimalType(2, 0)), Decimal(10)) - checkEvaluation(Cast(Literal(10.05), DecimalType(1, 0)), null) - checkEvaluation(Cast(Literal(10.05), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(10.05), DecimalType(3, 2)), null) - checkEvaluation(Cast(Literal(Decimal(10.05)), DecimalType(3, 1)), Decimal(10.1)) - checkEvaluation(Cast(Literal(Decimal(10.05)), DecimalType(3, 2)), null) - - checkEvaluation(Cast(Literal(9.95), DecimalType(3, 2)), Decimal(9.95)) - checkEvaluation(Cast(Literal(9.95), DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(Cast(Literal(9.95), DecimalType(2, 0)), Decimal(10)) - checkEvaluation(Cast(Literal(9.95), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(9.95), DecimalType(1, 0)), null) - checkEvaluation(Cast(Literal(Decimal(9.95)), DecimalType(3, 1)), Decimal(10.0)) - checkEvaluation(Cast(Literal(Decimal(9.95)), DecimalType(1, 0)), null) - - checkEvaluation(Cast(Literal(-9.95), DecimalType(3, 2)), Decimal(-9.95)) - checkEvaluation(Cast(Literal(-9.95), DecimalType(3, 1)), Decimal(-10.0)) - checkEvaluation(Cast(Literal(-9.95), DecimalType(2, 0)), Decimal(-10)) - checkEvaluation(Cast(Literal(-9.95), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(-9.95), DecimalType(1, 0)), null) - checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(3, 1)), Decimal(-10.0)) - checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(1, 0)), null) - - checkEvaluation(Cast(Literal(Double.NaN), DecimalType.Unlimited), null) - checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType.Unlimited), null) - checkEvaluation(Cast(Literal(Float.NaN), DecimalType.Unlimited), null) - checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType.Unlimited), null) - - checkEvaluation(Cast(Literal(Double.NaN), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(Float.NaN), DecimalType(2, 1)), null) - checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType(2, 1)), null) - } - - test("timestamp") { - val ts1 = new Timestamp(12) - val ts2 = new Timestamp(123) - checkEvaluation(Literal("ab") < Literal("abc"), true) - checkEvaluation(Literal(ts1) < Literal(ts2), true) - } - - test("date casting") { - val d = Date.valueOf("1970-01-01") - checkEvaluation(Cast(Literal(d), ShortType), null) - checkEvaluation(Cast(Literal(d), IntegerType), null) - checkEvaluation(Cast(Literal(d), LongType), null) - checkEvaluation(Cast(Literal(d), FloatType), null) - checkEvaluation(Cast(Literal(d), DoubleType), null) - checkEvaluation(Cast(Literal(d), DecimalType.Unlimited), null) - checkEvaluation(Cast(Literal(d), DecimalType(10, 2)), null) - checkEvaluation(Cast(Literal(d), StringType), "1970-01-01") - checkEvaluation(Cast(Cast(Literal(d), TimestampType), StringType), "1970-01-01 00:00:00") - } - - test("timestamp casting") { - val millis = 15 * 1000 + 2 - val seconds = millis * 1000 + 2 - val ts = new Timestamp(millis) - val tss = new Timestamp(seconds) - checkEvaluation(Cast(ts, ShortType), 15.toShort) - checkEvaluation(Cast(ts, IntegerType), 15) - checkEvaluation(Cast(ts, LongType), 15.toLong) - checkEvaluation(Cast(ts, FloatType), 15.002f) - checkEvaluation(Cast(ts, DoubleType), 15.002) - checkEvaluation(Cast(Cast(tss, ShortType), TimestampType), ts) - checkEvaluation(Cast(Cast(tss, IntegerType), TimestampType), ts) - checkEvaluation(Cast(Cast(tss, LongType), TimestampType), ts) - checkEvaluation(Cast(Cast(millis.toFloat / 1000, TimestampType), FloatType), - millis.toFloat / 1000) - checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType), - millis.toDouble / 1000) - checkEvaluation(Cast(Literal(Decimal(1)) cast TimestampType, DecimalType.Unlimited), Decimal(1)) - - // A test for higher precision than millis - checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) - - checkEvaluation(Cast(Literal(Double.NaN), TimestampType), null) - checkEvaluation(Cast(Literal(1.0 / 0.0), TimestampType), null) - checkEvaluation(Cast(Literal(Float.NaN), TimestampType), null) - checkEvaluation(Cast(Literal(1.0f / 0.0f), TimestampType), null) - } - - test("array casting") { - val array = Literal.create(Seq("123", "abc", "", null), - ArrayType(StringType, containsNull = true)) - val array_notNull = Literal.create(Seq("123", "abc", ""), - ArrayType(StringType, containsNull = false)) - - { - val cast = Cast(array, ArrayType(IntegerType, containsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Seq(123, null, null, null)) - } - { - val cast = Cast(array, ArrayType(IntegerType, containsNull = false)) - assert(cast.resolved === false) - } - { - val cast = Cast(array, ArrayType(BooleanType, containsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Seq(true, true, false, null)) - } - { - val cast = Cast(array, ArrayType(BooleanType, containsNull = false)) - assert(cast.resolved === false) - } - - { - val cast = Cast(array_notNull, ArrayType(IntegerType, containsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Seq(123, null, null)) - } - { - val cast = Cast(array_notNull, ArrayType(IntegerType, containsNull = false)) - assert(cast.resolved === false) - } - { - val cast = Cast(array_notNull, ArrayType(BooleanType, containsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Seq(true, true, false)) - } - { - val cast = Cast(array_notNull, ArrayType(BooleanType, containsNull = false)) - assert(cast.resolved === true) - checkEvaluation(cast, Seq(true, true, false)) - } - - { - val cast = Cast(array, IntegerType) - assert(cast.resolved === false) - } - } - - test("map casting") { - val map = Literal.create( - Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null), - MapType(StringType, StringType, valueContainsNull = true)) - val map_notNull = Literal.create( - Map("a" -> "123", "b" -> "abc", "c" -> ""), - MapType(StringType, StringType, valueContainsNull = false)) - - { - val cast = Cast(map, MapType(StringType, IntegerType, valueContainsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null)) - } - { - val cast = Cast(map, MapType(StringType, IntegerType, valueContainsNull = false)) - assert(cast.resolved === false) - } - { - val cast = Cast(map, MapType(StringType, BooleanType, valueContainsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false, "d" -> null)) - } - { - val cast = Cast(map, MapType(StringType, BooleanType, valueContainsNull = false)) - assert(cast.resolved === false) - } - { - val cast = Cast(map, MapType(IntegerType, StringType, valueContainsNull = true)) - assert(cast.resolved === false) - } - - { - val cast = Cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Map("a" -> 123, "b" -> null, "c" -> null)) - } - { - val cast = Cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false)) - assert(cast.resolved === false) - } - { - val cast = Cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true)) - assert(cast.resolved === true) - checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false)) - } - { - val cast = Cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false)) - assert(cast.resolved === true) - checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false)) - } - { - val cast = Cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true)) - assert(cast.resolved === false) - } - - { - val cast = Cast(map, IntegerType) - assert(cast.resolved === false) - } - } - - test("struct casting") { - val struct = Literal.create( - Row("123", "abc", "", null), - StructType(Seq( - StructField("a", StringType, nullable = true), - StructField("b", StringType, nullable = true), - StructField("c", StringType, nullable = true), - StructField("d", StringType, nullable = true)))) - val struct_notNull = Literal.create( - Row("123", "abc", ""), - StructType(Seq( - StructField("a", StringType, nullable = false), - StructField("b", StringType, nullable = false), - StructField("c", StringType, nullable = false)))) - - { - val cast = Cast(struct, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = true), - StructField("d", IntegerType, nullable = true)))) - assert(cast.resolved === true) - checkEvaluation(cast, Row(123, null, null, null)) - } - { - val cast = Cast(struct, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = false), - StructField("d", IntegerType, nullable = true)))) - assert(cast.resolved === false) - } - { - val cast = Cast(struct, StructType(Seq( - StructField("a", BooleanType, nullable = true), - StructField("b", BooleanType, nullable = true), - StructField("c", BooleanType, nullable = true), - StructField("d", BooleanType, nullable = true)))) - assert(cast.resolved === true) - checkEvaluation(cast, Row(true, true, false, null)) - } - { - val cast = Cast(struct, StructType(Seq( - StructField("a", BooleanType, nullable = true), - StructField("b", BooleanType, nullable = true), - StructField("c", BooleanType, nullable = false), - StructField("d", BooleanType, nullable = true)))) - assert(cast.resolved === false) - } - - { - val cast = Cast(struct_notNull, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = true)))) - assert(cast.resolved === true) - checkEvaluation(cast, Row(123, null, null)) - } - { - val cast = Cast(struct_notNull, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = false)))) - assert(cast.resolved === false) - } - { - val cast = Cast(struct_notNull, StructType(Seq( - StructField("a", BooleanType, nullable = true), - StructField("b", BooleanType, nullable = true), - StructField("c", BooleanType, nullable = true)))) - assert(cast.resolved === true) - checkEvaluation(cast, Row(true, true, false)) - } - { - val cast = Cast(struct_notNull, StructType(Seq( - StructField("a", BooleanType, nullable = true), - StructField("b", BooleanType, nullable = true), - StructField("c", BooleanType, nullable = false)))) - assert(cast.resolved === true) - checkEvaluation(cast, Row(true, true, false)) - } - - { - val cast = Cast(struct, StructType(Seq( - StructField("a", StringType, nullable = true), - StructField("b", StringType, nullable = true), - StructField("c", StringType, nullable = true)))) - assert(cast.resolved === false) - } - { - val cast = Cast(struct, IntegerType) - assert(cast.resolved === false) - } - } - - test("complex casting") { - val complex = Literal.create( - Row( - Seq("123", "abc", ""), - Map("a" -> "123", "b" -> "abc", "c" -> ""), - Row(0)), - StructType(Seq( - StructField("a", - ArrayType(StringType, containsNull = false), nullable = true), - StructField("m", - MapType(StringType, StringType, valueContainsNull = false), nullable = true), - StructField("s", - StructType(Seq( - StructField("i", IntegerType, nullable = true))))))) - - val cast = Cast(complex, StructType(Seq( - StructField("a", - ArrayType(IntegerType, containsNull = true), nullable = true), - StructField("m", - MapType(StringType, BooleanType, valueContainsNull = false), nullable = true), - StructField("s", - StructType(Seq( - StructField("l", LongType, nullable = true))))))) - - assert(cast.resolved === true) - checkEvaluation(cast, Row( - Seq(123, null, null), - Map("a" -> true, "b" -> true, "c" -> false), - Row(0L))) - } - - test("null checking") { - val row = create_row("^Ba*n", null, true, null) - val c1 = 'a.string.at(0) - val c2 = 'a.string.at(1) - val c3 = 'a.boolean.at(2) - val c4 = 'a.boolean.at(3) - - checkEvaluation(c1.isNull, false, row) - checkEvaluation(c1.isNotNull, true, row) - - checkEvaluation(c2.isNull, true, row) - checkEvaluation(c2.isNotNull, false, row) - - checkEvaluation(Literal.create(1, ShortType).isNull, false) - checkEvaluation(Literal.create(1, ShortType).isNotNull, true) - - checkEvaluation(Literal.create(null, ShortType).isNull, true) - checkEvaluation(Literal.create(null, ShortType).isNotNull, false) - - checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row) - checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) - - checkEvaluation( - If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) - checkEvaluation(If(c3, c1, c2), "^Ba*n", row) - checkEvaluation(If(c4, c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal.create(true, BooleanType), c1, c2), "^Ba*n", row) - checkEvaluation(If(Literal.create(false, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal.create(false, BooleanType), - Literal.create("a", StringType), Literal.create("b", StringType)), "b", row) - - checkEvaluation(c1 in (c1, c2), true, row) - checkEvaluation( - Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType)), true, row) - checkEvaluation( - Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType), c2), true, row) - } - - test("case when") { - val row = create_row(null, false, true, "a", "b", "c") - val c1 = 'a.boolean.at(0) - val c2 = 'a.boolean.at(1) - val c3 = 'a.boolean.at(2) - val c4 = 'a.string.at(3) - val c5 = 'a.string.at(4) - val c6 = 'a.string.at(5) - - checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row) - - checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row) - checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row) - - assert(CaseWhen(Seq(c2, c4, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true) - - val c4_notNull = 'a.boolean.notNull.at(3) - val c5_notNull = 'a.boolean.notNull.at(4) - val c6_notNull = 'a.boolean.notNull.at(5) - - assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true) - - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true) - - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) - } - - test("case key when") { - val row = create_row(null, 1, 2, "a", "b", "c") - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.int.at(2) - val c4 = 'a.string.at(3) - val c5 = 'a.string.at(4) - val c6 = 'a.string.at(5) - - val literalNull = Literal.create(null, IntegerType) - val literalInt = Literal(1) - val literalString = Literal("a") - - checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row) - checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row) - checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row) - checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row) - checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row) - checkEvaluation(CaseKeyWhen(c4, Seq(c6, c3, c5, c2, Literal(3))), 3, row) - - checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row) - checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row) - checkEvaluation(CaseKeyWhen(c6, Seq(c5, c2, c4, c3)), null, row) - checkEvaluation(CaseKeyWhen(literalNull, Seq(c2, c5, c1, c6)), "c", row) - } - - test("complex type") { - val row = create_row( - "^Ba*n", // 0 - null.asInstanceOf[UTF8String], // 1 - create_row("aa", "bb"), // 2 - Map("aa"->"bb"), // 3 - Seq("aa", "bb") // 4 - ) - - val typeS = StructType( - StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil - ) - val typeMap = MapType(StringType, StringType) - val typeArray = ArrayType(StringType) - - checkEvaluation(GetMapValue(BoundReference(3, typeMap, true), - Literal("aa")), "bb", row) - checkEvaluation(GetMapValue(Literal.create(null, typeMap), Literal("aa")), null, row) - checkEvaluation( - GetMapValue(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) - checkEvaluation(GetMapValue(BoundReference(3, typeMap, true), - Literal.create(null, StringType)), null, row) - - checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true), - Literal(1)), "bb", row) - checkEvaluation(GetArrayItem(Literal.create(null, typeArray), Literal(1)), null, row) - checkEvaluation( - GetArrayItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) - checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true), - Literal.create(null, IntegerType)), null, row) - - def getStructField(expr: Expression, fieldName: String): ExtractValue = { - expr.dataType match { - case StructType(fields) => - val field = fields.find(_.name == fieldName).get - GetStructField(expr, field, fields.indexOf(field)) - } - } - - def quickResolve(u: UnresolvedExtractValue): ExtractValue = { - ExtractValue(u.child, u.extraction, _ == _) - } - - checkEvaluation(getStructField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) - checkEvaluation(getStructField(Literal.create(null, typeS), "a"), null, row) - - val typeS_notNullable = StructType( - StructField("a", StringType, nullable = false) - :: StructField("b", StringType, nullable = false) :: Nil - ) - - assert(getStructField(BoundReference(2, typeS, nullable = true), "a").nullable === true) - assert(getStructField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable - === false) - - assert(getStructField(Literal.create(null, typeS), "a").nullable === true) - assert(getStructField(Literal.create(null, typeS_notNullable), "a").nullable === true) - - checkEvaluation(quickResolve('c.map(typeMap).at(3).getItem("aa")), "bb", row) - checkEvaluation(quickResolve('c.array(typeArray.elementType).at(4).getItem(1)), "bb", row) - checkEvaluation(quickResolve('c.struct(typeS).at(2).getField("a")), "aa", row) - } - - test("error message of ExtractValue") { - val structType = StructType(StructField("a", StringType, true) :: Nil) - val arrayStructType = ArrayType(structType) - val arrayType = ArrayType(StringType) - val otherType = StringType - - def checkErrorMessage( - childDataType: DataType, - fieldDataType: DataType, - errorMesage: String): Unit = { - val e = intercept[org.apache.spark.sql.AnalysisException] { - ExtractValue( - Literal.create(null, childDataType), - Literal.create(null, fieldDataType), - _ == _) - } - assert(e.getMessage().contains(errorMesage)) - } - - checkErrorMessage(structType, IntegerType, "Field name should be String Literal") - checkErrorMessage(arrayStructType, BooleanType, "Field name should be String Literal") - checkErrorMessage(arrayType, StringType, "Array index should be integral type") - checkErrorMessage(otherType, StringType, "Can't extract value from") - } - - test("arithmetic") { - val row = create_row(1, 2, 3, null) - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.int.at(2) - val c4 = 'a.int.at(3) - - checkEvaluation(UnaryMinus(c1), -1, row) - checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100) - - checkEvaluation(Add(c1, c4), null, row) - checkEvaluation(Add(c1, c2), 3, row) - checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation( - Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) - - checkEvaluation(-c1, -1, row) - checkEvaluation(c1 + c2, 3, row) - checkEvaluation(c1 - c2, -1, row) - checkEvaluation(c1 * c2, 2, row) - checkEvaluation(c1 / c2, 0, row) - checkEvaluation(c1 % c2, 1, row) - } - - test("fractional arithmetic") { - val row = create_row(1.1, 2.0, 3.1, null) - val c1 = 'a.double.at(0) - val c2 = 'a.double.at(1) - val c3 = 'a.double.at(2) - val c4 = 'a.double.at(3) - - checkEvaluation(UnaryMinus(c1), -1.1, row) - checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0) - checkEvaluation(Add(c1, c4), null, row) - checkEvaluation(Add(c1, c2), 3.1, row) - checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) - checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) - checkEvaluation( - Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) - - checkEvaluation(-c1, -1.1, row) - checkEvaluation(c1 + c2, 3.1, row) - checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row) - checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row) - checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row) - checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row) - } - - test("BinaryComparison") { - val row = create_row(1, 2, 3, null, 3, null) - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.int.at(2) - val c4 = 'a.int.at(3) - val c5 = 'a.int.at(4) - val c6 = 'a.int.at(5) - - checkEvaluation(LessThan(c1, c4), null, row) - checkEvaluation(LessThan(c1, c2), true, row) - checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation( - LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) - - checkEvaluation(c1 < c2, true, row) - checkEvaluation(c1 <= c2, true, row) - checkEvaluation(c1 > c2, false, row) - checkEvaluation(c1 >= c2, false, row) - checkEvaluation(c1 === c2, false, row) - checkEvaluation(c1 !== c2, true, row) - checkEvaluation(c4 <=> c1, false, row) - checkEvaluation(c1 <=> c4, false, row) - checkEvaluation(c4 <=> c6, true, row) - checkEvaluation(c3 <=> c5, true, row) - checkEvaluation(Literal(true) <=> Literal.create(null, BooleanType), false, row) - checkEvaluation(Literal.create(null, BooleanType) <=> Literal(true), false, row) - } - - test("StringComparison") { - val row = create_row("abc", null) - val c1 = 'a.string.at(0) - val c2 = 'a.string.at(1) - - checkEvaluation(c1 contains "b", true, row) - checkEvaluation(c1 contains "x", false, row) - checkEvaluation(c2 contains "b", null, row) - checkEvaluation(c1 contains Literal.create(null, StringType), null, row) - - checkEvaluation(c1 startsWith "a", true, row) - checkEvaluation(c1 startsWith "b", false, row) - checkEvaluation(c2 startsWith "a", null, row) - checkEvaluation(c1 startsWith Literal.create(null, StringType), null, row) - - checkEvaluation(c1 endsWith "c", true, row) - checkEvaluation(c1 endsWith "b", false, row) - checkEvaluation(c2 endsWith "b", null, row) - checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row) - } - - test("Substring") { - val row = create_row("example", "example".toArray.map(_.toByte)) - - val s = 'a.string.at(0) - - // substring from zero position with less-than-full length - checkEvaluation( - Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) - checkEvaluation( - Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) - - // substring from zero position with full length - checkEvaluation( - Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) - checkEvaluation( - Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) - - // substring from zero position with greater-than-full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), - "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), - "example", row) - - // substring from nonzero position with less-than-full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), - "xa", row) - - // substring from nonzero position with full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), - "xample", row) - - // substring from nonzero position with greater-than-full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), - "xample", row) - - // zero-length substring (within string bounds) - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), - "", row) - - // zero-length substring (beyond string bounds) - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), - "", row) - - // substring(null, _, _) -> null - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), - null, create_row(null)) - - // substring(_, null, _) -> null - checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), - null, row) - - // substring(_, _, null) -> null - checkEvaluation( - Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), - null, - row) - - // 2-arg substring from zero position - checkEvaluation( - Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), - "example", - row) - checkEvaluation( - Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), - "example", - row) - - // 2-arg substring from nonzero position - checkEvaluation( - Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), - "xample", - row) - - val s_notNull = 'a.string.notNull.at(0) - - assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable - === true) - assert( - Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable - === false) - assert(Substring(s_notNull, - Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, - Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) - - checkEvaluation(s.substr(0, 2), "ex", row) - checkEvaluation(s.substr(0), "example", row) - checkEvaluation(s.substring(0, 2), "ex", row) - checkEvaluation(s.substring(0), "example", row) - } - - test("SQRT") { - val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24)) - val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) - val rowSequence = inputSequence.map(l => create_row(l.toDouble)) - val d = 'a.double.at(0) - - for ((row, expected) <- rowSequence zip expectedResults) { - checkEvaluation(Sqrt(d), expected, row) - } - - checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null)) - checkEvaluation(Sqrt(-1), null, EmptyRow) - checkEvaluation(Sqrt(-1.5), null, EmptyRow) - } - - test("Bitwise operations") { - val row = create_row(1, 2, 3, null) - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.int.at(2) - val c4 = 'a.int.at(3) - - checkEvaluation(BitwiseAnd(c1, c4), null, row) - checkEvaluation(BitwiseAnd(c1, c2), 0, row) - checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation( - BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) - - checkEvaluation(BitwiseOr(c1, c4), null, row) - checkEvaluation(BitwiseOr(c1, c2), 3, row) - checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation( - BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) - - checkEvaluation(BitwiseXor(c1, c4), null, row) - checkEvaluation(BitwiseXor(c1, c2), 3, row) - checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation( - BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) - - checkEvaluation(BitwiseNot(c4), null, row) - checkEvaluation(BitwiseNot(c1), -2, row) - checkEvaluation(BitwiseNot(Literal.create(null, IntegerType)), null, row) - - checkEvaluation(c1 & c2, 0, row) - checkEvaluation(c1 | c2, 3, row) - checkEvaluation(c1 ^ c2, 3, row) - checkEvaluation(~c1, -2, row) - } - - /** - * Used for testing math functions for DataFrames. - * @param c The DataFrame function - * @param f The functions in scala.math - * @param domain The set of values to run the function with - * @param expectNull Whether the given values should return null or not - * @tparam T Generic type for primitives - */ - def unaryMathFunctionEvaluation[@specialized(Int, Double, Float, Long) T]( - c: Expression => Expression, - f: T => T, - domain: Iterable[T] = (-20 to 20).map(_ * 0.1), - expectNull: Boolean = false): Unit = { - if (expectNull) { - domain.foreach { value => - checkEvaluation(c(Literal(value)), null, EmptyRow) - } - } else { - domain.foreach { value => - checkEvaluation(c(Literal(value)), f(value), EmptyRow) - } - } - checkEvaluation(c(Literal.create(null, DoubleType)), null, create_row(null)) - } - - test("sin") { - unaryMathFunctionEvaluation(Sin, math.sin) - } - - test("asin") { - unaryMathFunctionEvaluation(Asin, math.asin, (-10 to 10).map(_ * 0.1)) - unaryMathFunctionEvaluation(Asin, math.asin, (11 to 20).map(_ * 0.1), true) - } - - test("sinh") { - unaryMathFunctionEvaluation(Sinh, math.sinh) - } - - test("cos") { - unaryMathFunctionEvaluation(Cos, math.cos) - } - - test("acos") { - unaryMathFunctionEvaluation(Acos, math.acos, (-10 to 10).map(_ * 0.1)) - unaryMathFunctionEvaluation(Acos, math.acos, (11 to 20).map(_ * 0.1), true) - } - - test("cosh") { - unaryMathFunctionEvaluation(Cosh, math.cosh) - } - - test("tan") { - unaryMathFunctionEvaluation(Tan, math.tan) - } - - test("atan") { - unaryMathFunctionEvaluation(Atan, math.atan) - } - - test("tanh") { - unaryMathFunctionEvaluation(Tanh, math.tanh) - } - - test("toDegrees") { - unaryMathFunctionEvaluation(ToDegrees, math.toDegrees) - } - - test("toRadians") { - unaryMathFunctionEvaluation(ToRadians, math.toRadians) - } - - test("cbrt") { - unaryMathFunctionEvaluation(Cbrt, math.cbrt) - } - - test("ceil") { - unaryMathFunctionEvaluation(Ceil, math.ceil) - } - - test("floor") { - unaryMathFunctionEvaluation(Floor, math.floor) - } - - test("rint") { - unaryMathFunctionEvaluation(Rint, math.rint) - } - - test("exp") { - unaryMathFunctionEvaluation(Exp, math.exp) - } - - test("expm1") { - unaryMathFunctionEvaluation(Expm1, math.expm1) - } - - test("signum") { - unaryMathFunctionEvaluation[Double](Signum, math.signum) - } - - test("log") { - unaryMathFunctionEvaluation(Log, math.log, (0 to 20).map(_ * 0.1)) - unaryMathFunctionEvaluation(Log, math.log, (-5 to -1).map(_ * 0.1), true) - } - - test("log10") { - unaryMathFunctionEvaluation(Log10, math.log10, (0 to 20).map(_ * 0.1)) - unaryMathFunctionEvaluation(Log10, math.log10, (-5 to -1).map(_ * 0.1), true) - } - - test("log1p") { - unaryMathFunctionEvaluation(Log1p, math.log1p, (-1 to 20).map(_ * 0.1)) - unaryMathFunctionEvaluation(Log1p, math.log1p, (-10 to -2).map(_ * 1.0), true) - } - - /** - * Used for testing math functions for DataFrames. - * @param c The DataFrame function - * @param f The functions in scala.math - * @param domain The set of values to run the function with - */ - def binaryMathFunctionEvaluation( - c: (Expression, Expression) => Expression, - f: (Double, Double) => Double, - domain: Iterable[(Double, Double)] = (-20 to 20).map(v => (v * 0.1, v * -0.1)), - expectNull: Boolean = false): Unit = { - if (expectNull) { - domain.foreach { case (v1, v2) => - checkEvaluation(c(v1, v2), null, create_row(null)) - } - } else { - domain.foreach { case (v1, v2) => - checkEvaluation(c(v1, v2), f(v1 + 0.0, v2 + 0.0), EmptyRow) - checkEvaluation(c(v2, v1), f(v2 + 0.0, v1 + 0.0), EmptyRow) - } - } - checkEvaluation(c(Literal.create(null, DoubleType), 1.0), null, create_row(null)) - checkEvaluation(c(1.0, Literal.create(null, DoubleType)), null, create_row(null)) - } - - test("pow") { - binaryMathFunctionEvaluation(Pow, math.pow, (-5 to 5).map(v => (v * 1.0, v * 1.0))) - binaryMathFunctionEvaluation(Pow, math.pow, Seq((-1.0, 0.9), (-2.2, 1.7), (-2.2, -1.7)), true) - } - - test("hypot") { - binaryMathFunctionEvaluation(Hypot, math.hypot) - } - - test("atan2") { - binaryMathFunctionEvaluation(Atan2, math.atan2) - } -} - -// TODO: Make the tests work with codegen. -class ExpressionEvaluationWithoutCodeGenSuite extends ExpressionEvaluationBaseSuite { - - override def checkEvaluation( - expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { - checkEvaluationWithoutCodegen(expression, expected, inputRow) - } - - test("CreateStruct") { - val row = Row(1, 2, 3) - val c1 = 'a.int.at(0).as("a") - val c3 = 'c.int.at(2).as("c") - checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row) - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala new file mode 100644 index 0000000000000..f44f55dfb92d1 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -0,0 +1,55 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.StringType + + +class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + + // TODO: Add tests for all data types. + + test("boolean literals") { + checkEvaluation(Literal(true), true) + checkEvaluation(Literal(false), false) + } + + test("int literals") { + checkEvaluation(Literal(1), 1) + checkEvaluation(Literal(0L), 0L) + } + + test("double literals") { + List(0.0, -0.0, Double.NegativeInfinity, Double.PositiveInfinity).foreach { + d => { + checkEvaluation(Literal(d), d) + checkEvaluation(Literal(d.toFloat), d.toFloat) + } + } + } + + test("string literals") { + checkEvaluation(Literal("test"), "test") + checkEvaluation(Literal.create(null, StringType), null) + } + + test("sum two literals") { + checkEvaluation(Add(Literal(1), Literal(1)), 2) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala new file mode 100644 index 0000000000000..25ebc70d095d8 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -0,0 +1,179 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.DoubleType + +class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + /** + * Used for testing unary math expressions. + * + * @param c expression + * @param f The functions in scala.math + * @param domain The set of values to run the function with + * @param expectNull Whether the given values should return null or not + * @tparam T Generic type for primitives + */ + private def testUnary[T]( + c: Expression => Expression, + f: T => T, + domain: Iterable[T] = (-20 to 20).map(_ * 0.1), + expectNull: Boolean = false): Unit = { + if (expectNull) { + domain.foreach { value => + checkEvaluation(c(Literal(value)), null, EmptyRow) + } + } else { + domain.foreach { value => + checkEvaluation(c(Literal(value)), f(value), EmptyRow) + } + } + checkEvaluation(c(Literal.create(null, DoubleType)), null, create_row(null)) + } + + /** + * Used for testing binary math expressions. + * + * @param c The DataFrame function + * @param f The functions in scala.math + * @param domain The set of values to run the function with + */ + private def testBinary( + c: (Expression, Expression) => Expression, + f: (Double, Double) => Double, + domain: Iterable[(Double, Double)] = (-20 to 20).map(v => (v * 0.1, v * -0.1)), + expectNull: Boolean = false): Unit = { + if (expectNull) { + domain.foreach { case (v1, v2) => + checkEvaluation(c(Literal(v1), Literal(v2)), null, create_row(null)) + } + } else { + domain.foreach { case (v1, v2) => + checkEvaluation(c(Literal(v1), Literal(v2)), f(v1 + 0.0, v2 + 0.0), EmptyRow) + checkEvaluation(c(Literal(v2), Literal(v1)), f(v2 + 0.0, v1 + 0.0), EmptyRow) + } + } + checkEvaluation(c(Literal.create(null, DoubleType), Literal(1.0)), null, create_row(null)) + checkEvaluation(c(Literal(1.0), Literal.create(null, DoubleType)), null, create_row(null)) + } + + test("sin") { + testUnary(Sin, math.sin) + } + + test("asin") { + testUnary(Asin, math.asin, (-10 to 10).map(_ * 0.1)) + testUnary(Asin, math.asin, (11 to 20).map(_ * 0.1), expectNull = true) + } + + test("sinh") { + testUnary(Sinh, math.sinh) + } + + test("cos") { + testUnary(Cos, math.cos) + } + + test("acos") { + testUnary(Acos, math.acos, (-10 to 10).map(_ * 0.1)) + testUnary(Acos, math.acos, (11 to 20).map(_ * 0.1), expectNull = true) + } + + test("cosh") { + testUnary(Cosh, math.cosh) + } + + test("tan") { + testUnary(Tan, math.tan) + } + + test("atan") { + testUnary(Atan, math.atan) + } + + test("tanh") { + testUnary(Tanh, math.tanh) + } + + test("toDegrees") { + testUnary(ToDegrees, math.toDegrees) + } + + test("toRadians") { + testUnary(ToRadians, math.toRadians) + } + + test("cbrt") { + testUnary(Cbrt, math.cbrt) + } + + test("ceil") { + testUnary(Ceil, math.ceil) + } + + test("floor") { + testUnary(Floor, math.floor) + } + + test("rint") { + testUnary(Rint, math.rint) + } + + test("exp") { + testUnary(Exp, math.exp) + } + + test("expm1") { + testUnary(Expm1, math.expm1) + } + + test("signum") { + testUnary[Double](Signum, math.signum) + } + + test("log") { + testUnary(Log, math.log, (0 to 20).map(_ * 0.1)) + testUnary(Log, math.log, (-5 to -1).map(_ * 0.1), expectNull = true) + } + + test("log10") { + testUnary(Log10, math.log10, (0 to 20).map(_ * 0.1)) + testUnary(Log10, math.log10, (-5 to -1).map(_ * 0.1), expectNull = true) + } + + test("log1p") { + testUnary(Log1p, math.log1p, (-1 to 20).map(_ * 0.1)) + testUnary(Log1p, math.log1p, (-10 to -2).map(_ * 1.0), expectNull = true) + } + + test("pow") { + testBinary(Pow, math.pow, (-5 to 5).map(v => (v * 1.0, v * 1.0))) + testBinary(Pow, math.pow, Seq((-1.0, 0.9), (-2.2, 1.7), (-2.2, -1.7)), expectNull = true) + } + + test("hypot") { + testBinary(Hypot, math.hypot) + } + + test("atan2") { + testBinary(Atan2, math.atan2) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala new file mode 100644 index 0000000000000..ccdada8b56f83 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala @@ -0,0 +1,65 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types.{BooleanType, StringType, ShortType} + +class NullFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("null checking") { + val row = create_row("^Ba*n", null, true, null) + val c1 = 'a.string.at(0) + val c2 = 'a.string.at(1) + val c3 = 'a.boolean.at(2) + val c4 = 'a.boolean.at(3) + + checkEvaluation(c1.isNull, false, row) + checkEvaluation(c1.isNotNull, true, row) + + checkEvaluation(c2.isNull, true, row) + checkEvaluation(c2.isNotNull, false, row) + + checkEvaluation(Literal.create(1, ShortType).isNull, false) + checkEvaluation(Literal.create(1, ShortType).isNotNull, true) + + checkEvaluation(Literal.create(null, ShortType).isNull, true) + checkEvaluation(Literal.create(null, ShortType).isNotNull, false) + + checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) + checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row) + checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) + + checkEvaluation( + If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) + checkEvaluation(If(c3, c1, c2), "^Ba*n", row) + checkEvaluation(If(c4, c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(true, BooleanType), c1, c2), "^Ba*n", row) + checkEvaluation(If(Literal.create(false, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(false, BooleanType), + Literal.create("a", StringType), Literal.create("b", StringType)), "b", row) + + checkEvaluation(c1 in (c1, c2), true, row) + checkEvaluation( + Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType)), true, row) + checkEvaluation( + Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType), c2), true, row) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala new file mode 100644 index 0000000000000..b6261bfba0786 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -0,0 +1,179 @@ +/* + * 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.catalyst.expressions + +import java.sql.{Date, Timestamp} + +import scala.collection.immutable.HashSet + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.util.DateUtils +import org.apache.spark.sql.types.{IntegerType, BooleanType} + + +class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { + + private def booleanLogicTest( + name: String, + op: (Expression, Expression) => Expression, + truthTable: Seq[(Any, Any, Any)]) { + test(s"3VL $name") { + truthTable.foreach { + case (l, r, answer) => + val expr = op(Literal.create(l, BooleanType), Literal.create(r, BooleanType)) + checkEvaluation(expr, answer) + } + } + } + + // scalastyle:off + /** + * Checks for three-valued-logic. Based on: + * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 + * I.e. in flat cpo "False -> Unknown -> True", + * OR is lowest upper bound, + * AND is greatest lower bound. + * p q p OR q p AND q p = q + * True True True True True + * True False True False False + * True Unknown True Unknown Unknown + * False True True False False + * False False False False True + * False Unknown Unknown False Unknown + * Unknown True True Unknown Unknown + * Unknown False Unknown False Unknown + * Unknown Unknown Unknown Unknown Unknown + * + * p NOT p + * True False + * False True + * Unknown Unknown + */ + // scalastyle:on + val notTrueTable = + (true, false) :: + (false, true) :: + (null, null) :: Nil + + test("3VL Not") { + notTrueTable.foreach { case (v, answer) => + checkEvaluation(Not(Literal.create(v, BooleanType)), answer) + } + } + + booleanLogicTest("AND", And, + (true, true, true) :: + (true, false, false) :: + (true, null, null) :: + (false, true, false) :: + (false, false, false) :: + (false, null, false) :: + (null, true, null) :: + (null, false, false) :: + (null, null, null) :: Nil) + + booleanLogicTest("OR", Or, + (true, true, true) :: + (true, false, true) :: + (true, null, true) :: + (false, true, true) :: + (false, false, false) :: + (false, null, null) :: + (null, true, true) :: + (null, false, null) :: + (null, null, null) :: Nil) + + booleanLogicTest("=", EqualTo, + (true, true, true) :: + (true, false, false) :: + (true, null, null) :: + (false, true, false) :: + (false, false, true) :: + (false, null, null) :: + (null, true, null) :: + (null, false, null) :: + (null, null, null) :: Nil) + + test("IN") { + checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) + checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkEvaluation( + And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), Literal(2)))), + true) + } + + test("INSET") { + val hS = HashSet[Any]() + 1 + 2 + val nS = HashSet[Any]() + 1 + 2 + null + val one = Literal(1) + val two = Literal(2) + val three = Literal(3) + val nl = Literal(null) + val s = Seq(one, two) + val nullS = Seq(one, two, null) + checkEvaluation(InSet(one, hS), true) + checkEvaluation(InSet(two, hS), true) + checkEvaluation(InSet(two, nS), true) + checkEvaluation(InSet(nl, nS), true) + checkEvaluation(InSet(three, hS), false) + checkEvaluation(InSet(three, nS), false) + checkEvaluation(And(InSet(one, hS), InSet(two, hS)), true) + } + + + test("BinaryComparison") { + val row = create_row(1, 2, 3, null, 3, null) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + val c5 = 'a.int.at(4) + val c6 = 'a.int.at(5) + + checkEvaluation(LessThan(c1, c4), null, row) + checkEvaluation(LessThan(c1, c2), true, row) + checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row) + checkEvaluation( + LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + + checkEvaluation(c1 < c2, true, row) + checkEvaluation(c1 <= c2, true, row) + checkEvaluation(c1 > c2, false, row) + checkEvaluation(c1 >= c2, false, row) + checkEvaluation(c1 === c2, false, row) + checkEvaluation(c1 !== c2, true, row) + checkEvaluation(c4 <=> c1, false, row) + checkEvaluation(c1 <=> c4, false, row) + checkEvaluation(c4 <=> c6, true, row) + checkEvaluation(c3 <=> c5, true, row) + checkEvaluation(Literal(true) <=> Literal.create(null, BooleanType), false, row) + checkEvaluation(Literal.create(null, BooleanType) <=> Literal(true), false, row) + + val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01")) + val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02")) + checkEvaluation(Literal(d1) < Literal(d2), true) + + val ts1 = new Timestamp(12) + val ts2 = new Timestamp(123) + checkEvaluation(Literal("ab") < Literal("abc"), true) + checkEvaluation(Literal(ts1) < Literal(ts2), true) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala new file mode 100644 index 0000000000000..2e81296c4e623 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala @@ -0,0 +1,218 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types.{IntegerType, StringType} + + +class StringFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("StringComparison") { + val row = create_row("abc", null) + val c1 = 'a.string.at(0) + val c2 = 'a.string.at(1) + + checkEvaluation(c1 contains "b", true, row) + checkEvaluation(c1 contains "x", false, row) + checkEvaluation(c2 contains "b", null, row) + checkEvaluation(c1 contains Literal.create(null, StringType), null, row) + + checkEvaluation(c1 startsWith "a", true, row) + checkEvaluation(c1 startsWith "b", false, row) + checkEvaluation(c2 startsWith "a", null, row) + checkEvaluation(c1 startsWith Literal.create(null, StringType), null, row) + + checkEvaluation(c1 endsWith "c", true, row) + checkEvaluation(c1 endsWith "b", false, row) + checkEvaluation(c2 endsWith "b", null, row) + checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row) + } + + test("Substring") { + val row = create_row("example", "example".toArray.map(_.toByte)) + + val s = 'a.string.at(0) + + // substring from zero position with less-than-full length + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) + + // substring from zero position with full length + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) + + // substring from zero position with greater-than-full length + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), + "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), + "example", row) + + // substring from nonzero position with less-than-full length + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), + "xa", row) + + // substring from nonzero position with full length + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), + "xample", row) + + // substring from nonzero position with greater-than-full length + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), + "xample", row) + + // zero-length substring (within string bounds) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), + "", row) + + // zero-length substring (beyond string bounds) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + "", row) + + // substring(null, _, _) -> null + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + null, create_row(null)) + + // substring(_, null, _) -> null + checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), + null, row) + + // substring(_, _, null) -> null + checkEvaluation( + Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), + null, + row) + + // 2-arg substring from zero position + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) + + // 2-arg substring from nonzero position + checkEvaluation( + Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "xample", + row) + + val s_notNull = 'a.string.notNull.at(0) + + assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === true) + assert( + Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === false) + assert(Substring(s_notNull, + Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, + Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) + + checkEvaluation(s.substr(0, 2), "ex", row) + checkEvaluation(s.substr(0), "example", row) + checkEvaluation(s.substring(0, 2), "ex", row) + checkEvaluation(s.substring(0), "example", row) + } + + test("LIKE literal Regular Expression") { + checkEvaluation(Literal.create(null, StringType).like("a"), null) + checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) + checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null) + checkEvaluation("abdef" like "abdef", true) + checkEvaluation("a_%b" like "a\\__b", true) + checkEvaluation("addb" like "a_%b", true) + checkEvaluation("addb" like "a\\__b", false) + checkEvaluation("addb" like "a%\\%b", false) + checkEvaluation("a_%b" like "a%\\%b", true) + checkEvaluation("addb" like "a%", true) + checkEvaluation("addb" like "**", false) + checkEvaluation("abc" like "a%", true) + checkEvaluation("abc" like "b%", false) + checkEvaluation("abc" like "bc%", false) + checkEvaluation("a\nb" like "a_b", true) + checkEvaluation("ab" like "a%b", true) + checkEvaluation("a\nb" like "a%b", true) + } + + test("LIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + checkEvaluation("abcd" like regEx, null, create_row(null)) + checkEvaluation("abdef" like regEx, true, create_row("abdef")) + checkEvaluation("a_%b" like regEx, true, create_row("a\\__b")) + checkEvaluation("addb" like regEx, true, create_row("a_%b")) + checkEvaluation("addb" like regEx, false, create_row("a\\__b")) + checkEvaluation("addb" like regEx, false, create_row("a%\\%b")) + checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b")) + checkEvaluation("addb" like regEx, true, create_row("a%")) + checkEvaluation("addb" like regEx, false, create_row("**")) + checkEvaluation("abc" like regEx, true, create_row("a%")) + checkEvaluation("abc" like regEx, false, create_row("b%")) + checkEvaluation("abc" like regEx, false, create_row("bc%")) + checkEvaluation("a\nb" like regEx, true, create_row("a_b")) + checkEvaluation("ab" like regEx, true, create_row("a%b")) + checkEvaluation("a\nb" like regEx, true, create_row("a%b")) + + checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%")) + } + + test("RLIKE literal Regular Expression") { + checkEvaluation(Literal.create(null, StringType) rlike "abdef", null) + checkEvaluation("abdef" rlike Literal.create(null, StringType), null) + checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null) + checkEvaluation("abdef" rlike "abdef", true) + checkEvaluation("abbbbc" rlike "a.*c", true) + + checkEvaluation("fofo" rlike "^fo", true) + checkEvaluation("fo\no" rlike "^fo\no$", true) + checkEvaluation("Bn" rlike "^Ba*n", true) + checkEvaluation("afofo" rlike "fo", true) + checkEvaluation("afofo" rlike "^fo", false) + checkEvaluation("Baan" rlike "^Ba?n", false) + checkEvaluation("axe" rlike "pi|apa", false) + checkEvaluation("pip" rlike "^(pi)*$", false) + + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike "**") + } + } + + test("RLIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + checkEvaluation("abdef" rlike regEx, true, create_row("abdef")) + checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c")) + checkEvaluation("fofo" rlike regEx, true, create_row("^fo")) + checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$")) + checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n")) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike regEx, create_row("**")) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index a4a3a66b8b229..f33a18d53b1a9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -24,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ * Overrides our expression evaluation tests and reruns them after optimization has occured. This * is to ensure that constant folding and other optimizations do not break anything. */ -class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { +class ExpressionOptimizationSuite extends SparkFunSuite with ExpressionEvalHelper { override def checkEvaluation( expression: Expression, expected: Any, From c0eb74b84a9bc54f23ee785e32f65921773cfa39 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Jun 2015 14:39:58 -0700 Subject: [PATCH 4/5] Fixed compilation. --- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 77327f2b84eaa..454af47913bf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -24,7 +24,6 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils From 857599f2545f7194b76a93489999a8860856a090 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Jun 2015 15:00:51 -0700 Subject: [PATCH 5/5] Fixed style violation. --- .../org/apache/spark/sql/catalyst/expressions/CastSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index eededbfeef4f5..5bc7c30eee1b6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -74,7 +74,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(123L, DecimalType(3, 1)), Decimal(123.0)) // TODO: Fix the following bug and re-enable it. - //checkEvaluation(cast(123L, DecimalType(2, 0)), null) + // checkEvaluation(cast(123L, DecimalType(2, 0)), null) } test("cast from boolean") { @@ -153,7 +153,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { DecimalType.Unlimited), LongType), StringType), ShortType), 0.toShort) checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType),ByteType), + cast(cast(cast(cast(cast(cast("5", TimestampType), ByteType), DecimalType.Unlimited), LongType), StringType), ShortType), null) checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.Unlimited),