-
Notifications
You must be signed in to change notification settings - Fork 903
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
[BUG] [Java] CudfException on conversion of data between Arrow and Cudf #16794
Comments
Here are my findings so far:
I can narrow this repro down further with a Java test shortly. |
I should revise this: It wouldn't be productive to convert this into a standalone Java test. The problem is specifically caused because the result of the UDF in the repro is returning an Arrow I have done a fair bit of digging into the problem, and bisecting the changes. The crux of my findings concerns the effect of this part of the changes in #16590: - auto result = cudf::to_arrow(*tview, state->get_column_metadata(*tview));
+ auto got_arrow_schema = cudf::to_arrow_schema(*tview, state->get_column_metadata(*tview));
+ cudf::jni::set_nullable(got_arrow_schema.get());
+ auto got_arrow_array = cudf::to_arrow_host(*tview);
+ auto batch =
+ arrow::ImportRecordBatch(&got_arrow_array->array, got_arrow_schema.get()).ValueOrDie();
+ auto result = arrow::Table::FromRecordBatches({batch}).ValueOrDie(); I can confirm that rolling back to using It seems odd, but something in the way On the face of it, the schemas of the tables constructed in both methods (i.e. I'm still hopeful that we should be able to remedy this with a change in either |
I think I have a way forward for this bug. It has to do with the nullability vector. I'll post a PR, once I've gotten to the bottom of how nullability was transmitted in the old implementation (i.e. with |
I'll raise a PR for what follows. I'm not sure I've grokked the context for the following change from #16590: void set_nullable(ArrowSchema* schema)
{
schema->flags |= ARROW_FLAG_NULLABLE;
for (int i = 0; i < schema->n_children; ++i) {
set_nullable(schema->children[i]);
}
} @vyasr: This code sets all the columns to "nullable", unconditionally. Given that I'm considering leaving the nullability as it is set from |
The breakdownThere are a couple of factors at play here. Here is the breakdown of what's happening: The failing test uses the One eccentricity of Why didn't it fail before?Before #16590, the test above was passing because the Why fail now?As part of #16590, all table columns (and child columns) had their nullability set to true. I've yet to fully understand the context of that change. With the The remedyPer the test, the
|
#16590 (comment) explains the issue. The problem is that the schema is generated once and only once using the first batch of data. We probably need to regenerate the schema each time that we send a batch. But the fact that awkward-array changes to a LARGE_LIST for a nullable array is a bit odd. Why would it do that when there is not enough data to warrant it. |
Baffling. It does so even with a single row, if it's a null row. I have verified that this spark.conf.set("spark.rapids.sql.enabled","false")
df = spark.sql("SELECT intI FROM values (ARRAY(3, 4)), (CAST(NULL AS ARRAY<INT>)) AS intI")
df.show(truncate=False)
newdf = df.mapInArrow(udf_array_square, df.schema)
newdf.show(truncate=False) With this two-row table, with the second row being null, and CUDF/JNI nowhere in sight, here's Apache Spark choking on the same thing:
So If removing the nullable reset isn't viable, then maybe we should consider |
I think we have to fix the UDF test at least. xfail is okay in the short term, but long term I don't think that awkward is compatible with spark. |
#16621 seems relevant here, although probably not in the short term. We need to figure out how best to match Arrow's choice of nullability in libcudf, particularly when there are no nulls in a column (and in the case of Spark, when there are no nulls in one batch but future batches might have them). It's tricky because of some implementation details like the fact that nullability in Arrow is a property of the type (more precisely, the field) while nullability in libcudf is defined by the (non)existence of the null mask in the column. It seems that the change to the C Data Interface is uncovering a lot more of those subtle edge cases. Rather than changing the behavior of the arrow conversion APIs, perhaps we need to add more explicit |
Description
The following exception is seen when CUDF JNI bindings are used to convert CUDF data to Arrow format, and then back to CUDF:
This was found during integration tests with https://github.com/NVIDIA/spark-rapids and https://github.com/NVIDIA/spark-rapids-jni.
I have narrowed it down to when #16590 was merged. Prior versions of CUDF that don't have this commit seem to work fine.
Repro
We don't yet have a narrow repro that uses on CUDF/JNI. I will include the
pyspark
repro here, and replace it with something smaller, once we have it:Running this Pyspark script causes the CudfException to occur, and the query to fail.
Expected behaviour
One would expect that type-conversions not fail between CUDF and Arrow.
The text was updated successfully, but these errors were encountered: