Skip to content

Commit

Permalink
[SPARK-25417][SQL] ArrayContains function may return incorrect result…
Browse files Browse the repository at this point in the history
… when right expression is implicitly down casted

## What changes were proposed in this pull request?
In ArrayContains, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result.

Example :
```SQL
spark-sql> select array_contains(array(1), 1.34);
true
```
```SQL
spark-sql> select array_contains(array(1), 'foo');
null
```

We should safely coerce both left and right hand side expressions.
## How was this patch tested?
Added tests in DataFrameFunctionsSuite

Closes #22408 from dilipbiswal/SPARK-25417.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
dilipbiswal authored and cloud-fan committed Sep 20, 2018
1 parent edf5cc6 commit 67f2c6a
Show file tree
Hide file tree
Showing 4 changed files with 128 additions and 15 deletions.
61 changes: 60 additions & 1 deletion docs/sql-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -1879,6 +1879,66 @@ working with timestamps in `pandas_udf`s to get the best performance, see

## Upgrading From Spark SQL 2.3 to 2.4

- In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below.
<table class="table">
<tr>
<th>
<b>Query</b>
</th>
<th>
<b>Result Spark 2.3 or Prior</b>
</th>
<th>
<b>Result Spark 2.4</b>
</th>
<th>
<b>Remarks</b>
</th>
</tr>
<tr>
<th>
<b>SELECT <br> array_contains(array(1), 1.34D);</b>
</th>
<th>
<b>true</b>
</th>
<th>
<b>false</b>
</th>
<th>
<b>In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively.</b>
</th>
</tr>
<tr>
<th>
<b>SELECT <br> array_contains(array(1), '1');</b>
</th>
<th>
<b>true</b>
</th>
<th>
<b>AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner.</b>
</th>
<th>
<b>Users can use explict cast</b>
</th>
</tr>
<tr>
<th>
<b>SELECT <br> array_contains(array(1), 'anystring');</b>
</th>
<th>
<b>null</b>
</th>
<th>
<b>AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner.</b>
</th>
<th>
<b>Users can use explict cast</b>
</th>
</tr>
</table>

- Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite.
- In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive.
- Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis.
Expand Down Expand Up @@ -1912,7 +1972,6 @@ working with timestamps in `pandas_udf`s to get the best performance, see
- The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles.
- Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown.
- Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below:

<table class="table">
<tr>
<th>
Expand Down
3 changes: 1 addition & 2 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -1498,8 +1498,7 @@ def test_array_contains_function(self):
from pyspark.sql.functions import array_contains

df = self.spark.createDataFrame([(["1", "2", "3"],), ([],)], ['data'])
actual = df.select(array_contains(df.data, 1).alias('b')).collect()
# The value argument can be implicitly castable to the element's type of the array.
actual = df.select(array_contains(df.data, "1").alias('b')).collect()
self.assertEqual([Row(b=True), Row(b=False)], actual)

def test_between_function(self):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1331,23 +1331,27 @@ case class ArrayContains(left: Expression, right: Expression)
@transient private lazy val ordering: Ordering[Any] =
TypeUtils.getInterpretedOrdering(right.dataType)

override def inputTypes: Seq[AbstractDataType] = right.dataType match {
case NullType => Seq.empty
case _ => left.dataType match {
case n @ ArrayType(element, _) => Seq(n, element)
override def inputTypes: Seq[AbstractDataType] = {
(left.dataType, right.dataType) match {
case (_, NullType) => Seq.empty
case (ArrayType(e1, hasNull), e2) =>
TypeCoercion.findTightestCommonType(e1, e2) match {
case Some(dt) => Seq(ArrayType(dt, hasNull), dt)
case _ => Seq.empty
}
case _ => Seq.empty
}
}

override def checkInputDataTypes(): TypeCheckResult = {
if (right.dataType == NullType) {
TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as arguments")
} else if (!left.dataType.isInstanceOf[ArrayType]
|| !left.dataType.asInstanceOf[ArrayType].elementType.sameType(right.dataType)) {
TypeCheckResult.TypeCheckFailure(
"Arguments must be an array followed by a value of same type as the array members")
} else {
TypeUtils.checkForOrderingExpr(right.dataType, s"function $prettyName")
(left.dataType, right.dataType) match {
case (_, NullType) =>
TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as arguments")
case (ArrayType(e1, _), e2) if e1.sameType(e2) =>
TypeUtils.checkForOrderingExpr(e2, s"function $prettyName")
case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
s"been ${ArrayType.simpleString} followed by a value with same element type, but it's " +
s"[${left.dataType.catalogString}, ${right.dataType.catalogString}].")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import scala.util.Random
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -735,6 +736,56 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
df.selectExpr("array_contains(array(1, null), array(1, null)[0])"),
Seq(Row(true), Row(true))
)

checkAnswer(
OneRowRelation().selectExpr("array_contains(array(1), 1.23D)"),
Seq(Row(false))
)

checkAnswer(
OneRowRelation().selectExpr("array_contains(array(1), 1.0D)"),
Seq(Row(true))
)

checkAnswer(
OneRowRelation().selectExpr("array_contains(array(1.0D), 1)"),
Seq(Row(true))
)

checkAnswer(
OneRowRelation().selectExpr("array_contains(array(1.23D), 1)"),
Seq(Row(false))
)

checkAnswer(
OneRowRelation().selectExpr("array_contains(array(array(1)), array(1.0D))"),
Seq(Row(true))
)

checkAnswer(
OneRowRelation().selectExpr("array_contains(array(array(1)), array(1.23D))"),
Seq(Row(false))
)

val e1 = intercept[AnalysisException] {
OneRowRelation().selectExpr("array_contains(array(1), .01234567890123456790123456780)")
}
val errorMsg1 =
s"""
|Input to function array_contains should have been array followed by a
|value with same element type, but it's [array<int>, decimal(29,29)].
""".stripMargin.replace("\n", " ").trim()
assert(e1.message.contains(errorMsg1))

val e2 = intercept[AnalysisException] {
OneRowRelation().selectExpr("array_contains(array(1), 'foo')")
}
val errorMsg2 =
s"""
|Input to function array_contains should have been array followed by a
|value with same element type, but it's [array<int>, string].
""".stripMargin.replace("\n", " ").trim()
assert(e2.message.contains(errorMsg2))
}

test("arrays_overlap function") {
Expand Down

0 comments on commit 67f2c6a

Please sign in to comment.