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 unnest multiple arrays #10044

Merged
merged 13 commits into from
Apr 15, 2024
Merged

Conversation

jonahgao
Copy link
Member

@jonahgao jonahgao commented Apr 11, 2024

Which issue does this PR close?

Closes #212.
Closes #1608.
Closes #6555.
Closes #6796.
Closes #7087.

Rationale for this change

From the PostgreSQL's document.

Expands multiple arrays (possibly of different data types) into a set of rows. If the arrays are not all the same length then the shorter ones are padded with NULLs. This form is only allowed in a query's FROM clause

An unnest function with N parameters is equivalent to N single-parameter unnest functions, that is,
select * from unnest(arg1, arg2, arg3) is equivalent to select unnest(arg1), unnest(arg2), unnest(arg3).
This PR will unify them all into the latter to be handled by UnnestExec.

In order to support UnnestExec with multiple unnest columns:

  • For each row in a RecordBatch, some list columns need to be unnested. We will expand the values in each list into multiple rows, taking the longest length among these lists, and shorter lists are padded with NULLs.
  • For columns that don't need to be unnested, repeat their values until reaching the longest length.

What changes are included in this PR?

Expand UnnestExec to support unnesting multiple arrays.
Also, enable the following two types of queries:

  1. There are multiple unnest functions in the select list.
  2. In the from clause, unnest can support multiple arguments.
DataFusion CLI v37.0.0
❯ select unnest([1,2]), unnest([3]);
+---------------------------------------+------------------------------+
| unnest(make_array(Int64(1),Int64(2))) | unnest(make_array(Int64(3))) |
+---------------------------------------+------------------------------+
| 1                                     | 3                            |
| 2                                     |                              |
+---------------------------------------+------------------------------+
2 row(s) fetched.
Elapsed 0.072 seconds.

❯ select * from unnest([1,2], [3]);
+---------------------------------------+------------------------------+
| unnest(make_array(Int64(1),Int64(2))) | unnest(make_array(Int64(3))) |
+---------------------------------------+------------------------------+
| 1                                     | 3                            |
| 2                                     |                              |
+---------------------------------------+------------------------------+

Are these changes tested?

Yes

Are there any user-facing changes?

Yes.
Several unnest related structs have changed.

@jonahgao jonahgao marked this pull request as draft April 11, 2024 05:44
@jonahgao jonahgao changed the title feat: support unnest with multiple arguments feat: support unnest multiple columns Apr 11, 2024
@github-actions github-actions bot added sql SQL Planner logical-expr Logical plan and expressions core Core DataFusion crate sqllogictest SQL Logic Tests (.slt) labels Apr 11, 2024
@jonahgao jonahgao changed the title feat: support unnest multiple columns feat: support unnest multiple arrays Apr 11, 2024
@jonahgao jonahgao marked this pull request as ready for review April 12, 2024 14:43
@@ -186,7 +186,16 @@ pub enum Expr {

#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Unnest {
pub exprs: Vec<Expr>,
pub expr: Box<Expr>,
Copy link
Member Author

Choose a reason for hiding this comment

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

In the expression position, only one argument can be accepted.

This comment was marked as outdated.

@alamb
Copy link
Contributor

alamb commented Apr 13, 2024

Screenshot 2024-04-12 at 8 24 06 PM

That is a pretty sweet list of things this closes. Thank you @jonahgao -- I will try and review this PR over the next day or two if no one beats me to it

query error DataFusion error: This feature is not implemented: Only support single unnest expression for now
select unnest(column1), unnest(column2) from unnest_table;
query ?I
select unnest([]), unnest(NULL::int[]);

This comment was marked as resolved.

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we should support null as well 🤔

query error DataFusion error: This feature is not implemented: unnest\(\) does not support null yet
select unnest([]), unnest(NULL);

Copy link
Member Author

@jonahgao jonahgao Apr 13, 2024

Choose a reason for hiding this comment

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

I'm not sure how to support it yet. PostgreSQL, BigQuery, and Clickhouse don't support unnesting untyped NULLs.

postgres=# select unnest(NULL);
ERROR:  function unnest(unknown) is not unique
LINE 1: select unnest(NULL);
               ^
HINT:  Could not choose a best candidate function. You might need to add explicit type casts.

Although DuckDB supports it, its behavior seems a bit strange to me.

D select unnest(NULL), unnest([1,2]);
┌──────────────┬───────────────────────────────┐
│ unnest(NULL) │ unnest(main.list_value(1, 2)) │
│    int32     │             int32             │
├──────────────────────────────────────────────┤
│                    0 rows                    │
└──────────────────────────────────────────────┘
D select unnest(NULL::int[]), unnest([1,2]);
┌─────────────────────────────────┬───────────────────────────────┐
│ unnest(CAST(NULL AS INTEGER[])) │ unnest(main.list_value(1, 2)) │
│              int32              │             int32             │
├─────────────────────────────────┼───────────────────────────────┤
│                                 │                             1 │
│                                 │                             2 │
└─────────────────────────────────┴───────────────────────────────┘

Unnesting an untyped NULL and a typed NULL produced different results.

I think there may be two appropriate methods:

  1. Use type coercion to convert an untype NULL into a certain List type like cast(NULL, NULL::int[]).
  2. Explicitly reject unnesting untyped NULLs.

Copy link
Contributor

Choose a reason for hiding this comment

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

I prefer 2, since PostgresSQL, and Clickhouse return errors in this case, we follow the majority.

Copy link
Member Author

Choose a reason for hiding this comment

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

I prefer 2, since PostgresSQL, and Clickhouse return errors in this case, we follow the majority.

Make sense to me!

Copy link
Contributor

Choose a reason for hiding this comment

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

I double checked and this PR does error as expected. I'll add a small test for this case here and merge this PR.

❯ select unnest(null) from t;
This feature is not implemented: unnest() does not support null yet

unnest_field.name(),
field.data_type().clone(),
// Unnesting may produce NULLs even if the list is not null.
// For example: unnset([1], []) -> 1, null
Copy link
Contributor

@jayzhan211 jayzhan211 Apr 13, 2024

Choose a reason for hiding this comment

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

Although the test does not cover this case, sadly I don't think there is currently a way to create a list with nullable as false 🤔 Maybe we can write a simple rust test for this?

Copy link
Member Author

Choose a reason for hiding this comment

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

I think so too. true should be equivalent to the previous unnest_field.is_nullable().
I planned to write some tests related to DataFrame, which could potentially implement it. But the changes in this PR are larger than I expected, so those tests have been delayed.

Copy link
Contributor

Choose a reason for hiding this comment

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

I planned to write some tests related to DataFrame, which could potentially implement it. But the changes in this PR are larger than I expected, so those tests have been delayed.

We can do it in another PR!

Copy link
Member Author

@jonahgao jonahgao Apr 15, 2024

Choose a reason for hiding this comment

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

We can do it in another PR!

👌

Copy link
Contributor

@jayzhan211 jayzhan211 left a comment

Choose a reason for hiding this comment

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

it looks pretty nice!

return Ok(input);
}
};
let mut unnested_fields: HashMap<usize, _> = HashMap::with_capacity(columns.len());
Copy link
Contributor

Choose a reason for hiding this comment

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

💯

@alamb
Copy link
Contributor

alamb commented Apr 15, 2024

Closes #212.
Closes #1608.
Closes #6555.
Closes #6796.
Closes #7087.

😍

Thank you @jonahgao and thank you @jayzhan211 for the review. I merged this branch up from main and added a test for untyped nulls and plan to merge it once CI passes

@alamb alamb merged commit 34eda15 into apache:main Apr 15, 2024
25 checks passed
@alamb
Copy link
Contributor

alamb commented Apr 15, 2024

🚀

@jonahgao
Copy link
Member Author

Thank you @alamb for the new test and the review. Thank you @jayzhan211 for the review.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
core Core DataFusion crate logical-expr Logical plan and expressions sql SQL Planner sqllogictest SQL Logic Tests (.slt)
Projects
None yet
3 participants