Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: support 'col IN (a, b, c)' type expressions #652

Open
wants to merge 10 commits into
base: main
Choose a base branch
from

Conversation

roeap
Copy link
Collaborator

@roeap roeap commented Jan 18, 2025

What changes are proposed in this pull request?

Currently, evaluation expressions of type col IN (a, b, c) is missing an implementation. While this might be the exact case @scovich cautioned us about, where the rhs might get significant in size and should really be handled as EngineData, I hope that we at least do not make things worse here. Unfortunately delta-rs already has support for these types of expressions, so the main intend right now is to retain feature parity over there while migrating.

How was this change tested?

Additional tests for specific expression flavor.

Copy link

codecov bot commented Jan 18, 2025

Codecov Report

Attention: Patch coverage is 92.48555% with 26 lines in your changes missing coverage. Please review.

Project coverage is 84.31%. Comparing base (06d8dbb) to head (6c813e9).

Files with missing lines Patch % Lines
kernel/src/engine/arrow_expression.rs 91.80% 21 Missing and 4 partials ⚠️
kernel/src/engine/arrow_conversion.rs 0.00% 1 Missing ⚠️
Additional details and impacted files
@@            Coverage Diff             @@
##             main     #652      +/-   ##
==========================================
+ Coverage   84.14%   84.31%   +0.16%     
==========================================
  Files          77       77              
  Lines       17710    18032     +322     
  Branches    17710    18032     +322     
==========================================
+ Hits        14902    15203     +301     
- Misses       2096     2112      +16     
- Partials      712      717       +5     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@roeap roeap force-pushed the feat/col-in-arr branch 2 times, most recently from 007b4e2 to 6977db9 Compare January 18, 2025 16:19
@@ -208,7 +208,7 @@ impl TryFrom<&ArrowDataType> for DataType {
ArrowDataType::Date64 => Ok(DataType::DATE),
ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => Ok(DataType::TIMESTAMP_NTZ),
ArrowDataType::Timestamp(TimeUnit::Microsecond, Some(tz))
if tz.eq_ignore_ascii_case("utc") =>
if tz.eq_ignore_ascii_case("utc") || tz.eq_ignore_ascii_case("+00:00") =>
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The data in arrow arrays should always represent a timestamp in UTC, so is this check even necessary?

https://github.com/apache/arrow-rs/blob/af777cd53e56f8382382137b6e08af249c475397/arrow-schema/src/datatype.rs#L179-L182

kernel/src/engine/arrow_expression.rs Outdated Show resolved Hide resolved
kernel/src/engine/arrow_expression.rs Outdated Show resolved Hide resolved
kernel/src/engine/arrow_expression.rs Outdated Show resolved Hide resolved
kernel/src/engine/arrow_expression.rs Outdated Show resolved Hide resolved
ad: &ArrayData,
) -> BooleanArray {
#[allow(deprecated)]
let res = col.map(|val| val.map(|v| ad.array_elements().contains(&v.into())));
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this handles NULL values correctly? See e.g. https://spark.apache.org/docs/3.5.1/sql-ref-null-semantics.html#innot-in-subquery-:

  • TRUE is returned when the non-NULL value in question is found in the list
  • FALSE is returned when the non-NULL value is not found in the list and the list does not contain NULL values
  • UNKNOWN is returned when the value is NULL, or the non-NULL value is not found in the list and the list contains at least one NULL value

I think, instead of calling contains, you could borrow the code from PredicateEvaluatorDefaults::finish_eval_variadic, with true as the "dominator" value.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, I think you could just invoke that method directly, with a properly crafted iterator?

// `v IN (k1, ..., kN)` is logically equivalent to `v = k1 OR ... OR v = kN`, so evaluate
// it as such, ensuring correct handling of NULL inputs (including `Scalar::Null`).
col.map(|v| {
    PredicateEvaluatorDefaults::finish_eval_variadic(
        VariadicOperator::Or, 
        inlist.iter().map(Some(Scalar::partial_cmp(v?, k?)? == Ordering::Equal)),
        false,
    )
})

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Was I correct in thinking that None - no dominant value, but found Null - should just be false in this case?

ad: &ArrayData,
) -> BooleanArray {
#[allow(deprecated)]
let res = col.map(|val| val.map(|v| ad.array_elements().contains(&v.into())));
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Aside: We actually have a lurking bug -- Scalar derives PartialEq which will allow two Scalar::Null to compare equal. But SQL semantics dictate that NULL doesn't compare equal to anything -- not even itself.

Our manual impl of PartialOrd for Scalar does this correctly, but it breaks the rules for PartialEq:

If PartialOrd or Ord are also implemented for Self and Rhs, their methods must also be consistent with PartialEq (see the documentation of those traits for the exact requirements). It’s easy to accidentally make them disagree by deriving some of the traits and manually implementing others.

Looks like we'll need to define a manual impl PartialEq for Scalar that follows the same approach.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is indeed not covered. Added an implementation for PartialEq that mirrors PartialOrd.

@roeap roeap requested a review from scovich January 24, 2025 23:55
kernel/src/engine/arrow_expression.rs Outdated Show resolved Hide resolved
Some(
PredicateEvaluatorDefaults::finish_eval_variadic(
VariadicOperator::Or,
inlist.iter().map(|k| v.as_ref().map(|vv| vv == k)),
Copy link
Collaborator

@scovich scovich Jan 25, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't correct -- we need comparisons against Scalar::Null to return None. That's why I had previously recommended using Scalar::partial_cmp instead of ==.

Also, can we not use ? to unwrap the various options here?

Suggested change
inlist.iter().map(|k| v.as_ref().map(|vv| vv == k)),
inlist.iter().map(Some(Scalar::partial_cmp(v?, k?)? == Ordering::Equal)),

Unpacking that -- if the value we search for is NULL, or if the inlist entry is NULL, or if the two values are incomparable, then return None for that pair. Otherwise, return Some boolean indicating whether the values compared equal or not. That automatically covers the various required cases, and also makes us robust to any type mismatches that might creep in.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note: If we wanted to be a tad more efficient, we could also unpack v outside the inner loop:

values.map(|v| {
    let v = v?;
    PredicateEvaluatorDefaults::finish_eval_variadic(...)
})

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm -- empty in-lists pose a corner case with respect to unpacking v:

NULL IN ()

Operator OR with zero inputs normally produces FALSE (which is correct if you stop to think about it) -- but unpacking a NULL v first makes the operator return NULL instead (which is also correct if you squint, because NULL input always produces NULL output).

Unfortunately, the only clear docs I could find -- https://spark.apache.org/docs/3.5.1/sql-ref-null-semantics.html#innot-in-subquery- -- are also ambiguous:

Conceptually a IN expression is semantically equivalent to a set of equality condition separated by a disjunctive operator (OR).

... suggests FALSE while

UNKNOWN is returned when the value is NULL

... suggests NULL

The difference matters for NOT IN, because NULL NOT IN () would either return TRUE (keep rows) or NULL (drop row).

Copy link
Collaborator

@scovich scovich Jan 25, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

NOTE: SQL engines normally forbid statically empty in-list but do not forbid subqueries from producing an empty result.

I tried the following expression on three engines (sqlite, mysql, postgres):

SELECT 1 WHERE NULL NOT IN (SELECT 1 WHERE FALSE)

And all three returned 1. So OR semantics prevail, and we must NOT unpack v outside the loop.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hoping I now considered all your comments, which essentially means going with your original version.

kernel/src/engine/arrow_expression.rs Outdated Show resolved Hide resolved
Comment on lines 336 to 345
(ArrowDataType::Utf8, PrimitiveType::String) => op_in(inlist, str_op(column.as_string::<i32>().iter())),
(ArrowDataType::LargeUtf8, PrimitiveType::String) => op_in(inlist, str_op(column.as_string::<i64>().iter())),
(ArrowDataType::Utf8View, PrimitiveType::String) => op_in(inlist, str_op(column.as_string_view().iter())),
(ArrowDataType::Int8, PrimitiveType::Byte) => op_in(inlist,op::<Int8Type>( column.as_ref(), Scalar::from)),
(ArrowDataType::Int16, PrimitiveType::Short) => op_in(inlist,op::<Int16Type>(column.as_ref(), Scalar::from)),
(ArrowDataType::Int32, PrimitiveType::Integer) => op_in(inlist,op::<Int32Type>(column.as_ref(), Scalar::from)),
(ArrowDataType::Int64, PrimitiveType::Long) => op_in(inlist,op::<Int64Type>(column.as_ref(), Scalar::from)),
(ArrowDataType::Float32, PrimitiveType::Float) => op_in(inlist,op::<Float32Type>(column.as_ref(), Scalar::from)),
(ArrowDataType::Float64, PrimitiveType::Double) => op_in(inlist,op::<Float64Type>(column.as_ref(), Scalar::from)),
(ArrowDataType::Date32, PrimitiveType::Date) => op_in(inlist,op::<Date32Type>(column.as_ref(), Scalar::Date)),
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are all a lot longer than 100 chars... why doesn't the fmt check blow up??

@@ -280,6 +281,84 @@ fn evaluate_expression(
(ArrowDataType::Decimal256(_, _), Decimal256Type)
}
}
(Column(name), Literal(Scalar::Array(ad))) => {
fn op<T: ArrowPrimitiveType>(
values: &dyn Array,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
values: &dyn Array,
values: ArrayRef,

(avoids the need for .as_ref() at the call site)

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i think the main thing was that we need this to be a reference, otherwise the compiler starts complaining about lifetimes. I did shorten the code at the call-site a bit, hope that works as well.

Copy link
Collaborator

@scovich scovich Jan 30, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why would an ArrayRef (= Arc<dyn Array>) give lifetime problems, sorry?
We can always call as_ref() on it to get a reference that lives at least as long as the arc?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not the parameter itself, but the as_primitive cast inside the functions returns a ref, which we then iterate over. This then causes issues with the iterator referencing data owned by the function.

kernel/src/expressions/scalars.rs Outdated Show resolved Hide resolved
kernel/src/expressions/scalars.rs Show resolved Hide resolved
Copy link
Collaborator

@scovich scovich left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we're in good shape now, just need missing tests.

}
impl PartialEq for Scalar {
fn eq(&self, other: &Scalar) -> bool {
self.partial_cmp(other) == Some(Ordering::Equal)
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

aside: Interesting, PartialOrd requires Self: PartialEq but we can still invoke the former from the latter?
(feels somehow like a circular dependency, but I guess if it compiles it works)

kernel/src/engine/arrow_expression.rs Show resolved Hide resolved
@@ -692,6 +771,85 @@ mod tests {
assert_eq!(in_result.as_ref(), &in_expected);
}

#[test]
fn test_column_in_array() {
Copy link
Collaborator

@scovich scovich Jan 30, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Relating to #652 (comment) and #652 (comment) -- we don't seem to have any tests that cover NULL value semantics?

1 IN (1, NULL) -- TRUE
1 IN (2, NULL) -- NULL
NULL IN (1, 2) -- NULL

1 NOT IN (1, NULL) -- FALSE (**)
1 NOT IN (2, NULL) -- NULL
NULL NOT IN (1, 2) -- NULL

(**) NOTE from https://spark.apache.org/docs/3.5.1/sql-ref-null-semantics.html#innot-in-subquery-:

NOT IN always returns UNKNOWN when the list contains NULL, regardless of the input value. This is because IN returns UNKNOWN if the value is not in the list containing NULL, and because NOT UNKNOWN is again UNKNOWN.

IMO, that explanation is confusing and factually incorrect. If we explain it in terms of NOT(OR):

1 NOT IN (1, NULL)
= NOT(1 IN (1, NULL))
= NOT(1 = 1 OR 1 = NULL)
= NOT(1 = 1) AND NOT(1 = NULL)
= 1 != 1 AND 1 != NULL
= FALSE AND NULL
= FALSE

As additional support for my claim: sqlite, postgres, and mysql all return FALSE (not NULL) for that expression.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added some test according to the cases mentioned above, hopefully covering all cases. This uncovered some cases where we were not handling NULLs correctly in the other in-list branches, mainly b/c the arrow kernels don't seem to be adhering to the SQL NULL semantics.

In addition to the engines above, I also tried duckdb and datafusion, which also support @scovich's claim.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also, is this something worth upstreaming to arrow-rs similar to the *_kleene variants for other kernels?

.map(|k| Some(lit.partial_cmp(k)? == Ordering::Equal)),
false,
);
Ok(Arc::new(BooleanArray::from(vec![exists; batch.num_rows()])))
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

aside from the null handling, I think we should return the result for each row in the input batch?

Comment on lines +263 to +267
(Decimal(v1, _, s1), Decimal(v2, _, s2)) => {
let lhs = rust_decimal::Decimal::from_i128_with_scale(*v1, *s1 as u32);
let rhs = rust_decimal::Decimal::from_i128_with_scale(*v2, *s2 as u32);
lhs.partial_cmp(&rhs)
}
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Am I correct in thinking that the "effective" precision is determined by the fact that we are using i128 to represent the data, so we need not handle that separately?

@roeap roeap requested a review from scovich February 2, 2025 17:38
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants