-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-21949][table] Support ARRAY_AGG aggregate function #23411
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
Conversation
ffb85dd
to
8cee2f8
Compare
Hi @wuchong, could you help review this when you have time? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for the contribution and sorry for the delay
one thing that I noticed: during aggregation there is java's hashmap is used
unfortunately java comparison rules are not fully in sync with SQL comparison rules
for that reason we can not use hash mpas here for generic cases.
We need to use custom expression evaluator, example could be seen in ArrayDistinct implementation
Thanks @snuyanzin for the review. |
Do you mean using equalityEvaluator to compare when merging and retracting lists? |
yes, sorry was not clear enough
yes |
72323f4
to
7fbdcf3
Compare
@flinkbot run azure |
7991087
to
a860e51
Compare
Hi @snuyanzin, could you help take a look again when you have time? |
a860e51
to
8ce0f58
Compare
It took me a while to understand that myself, but I think it's actually ok to depend on the
In I haven't seen the previous version @Jiabao-Sun , but I believe we can remove the |
private final List<RelDataType> paramTypes; | ||
|
||
private BridgingSqlAggFunction( | ||
FlinkContext context, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is the reason for those changes? Is it only to be able to create the equalityHandler
? If so, let's keep it simple and remove those.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I'll revert that changes.
Thanks.
thanks a lot for the explanation @dawidwys you're right |
Arrays.asList( | ||
Row.of("A", new Integer[] {1, 2}), | ||
Row.of("B", new Integer[] {2, 2, 3}), | ||
Row.of("C", new Integer[] {3}), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder why here is just
Row.of("C", new Integer[] {3}),
based on input I would expect
Row.of("C", new Integer[] {3, null}),
or did I miss anything?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also similar query for postgres
with input(a, c) as (
select 'a', 1
union all
select 'a', 2
union all
select 'c', 3
union all
select 'c', null
)
select a, array_agg(distinct c) from input group by a
gives
a|array_agg|
-+---------+
a|{1,2} |
c|{3,NULL} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
-- calcite
ARRAY_AGG([ ALL | DISTINCT ] value [ RESPECT NULLS | IGNORE NULLS ] [ ORDER BY orderItem [, orderItem ]* ] )
-- flink
ARRAY_AGG([ ALL | DISTINCT ] expression)
This function simplifies some aspects compared to calcite.
- Currently, the Flink parser does not support parsing RESPECT NULLS | IGNORE NULLS, but it is still possible to make changes to support it.
- AggregateUtil#extractDistinctInformation 848~858 ignores the ignoreNulls fields.
Lines 848 to 858 in 6bdb4f7
AggregateCall.create( call.getAggregation, false, false, false, call.getArgList, -1, // remove filterArg null, RelCollations.EMPTY, call.getType, call.getName) - ListView does not support null values.
flink/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/dataview/ListView.java
Lines 45 to 47 in 40cbf23
* <p>Note: Elements of a {@link ListView} must not be null. For heap-based state backends, {@code * hashCode/equals} of the original (i.e. external) class are used. However, the serialization * format will use internal data structures.
For that reasons I made some simplification. If we need it, I can make some attempts.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's ok to support it only partially however the main concern here:
I would expect more consistent result
since there is also input Row.ofKind(INSERT, "D", null),
and expected value Row.of("D", null),
So I would expect for ths either containing nulls or both not containing null however not mixed
UPD: to be more clear:
after playing with Postgres and BigQuery I noticed that both respect nulls by default and for BigQuery to make it ignoring nulls it should be specified explicitely. Is there a reason why for Flink it is done differently? Probably it's better to have similar behaviour
I don't tell that we need to support RESPECT NULLS | IGNORE NULLS
syntax, however need to make RESPECT NULLS
default behaviour to be on same page with vendors
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @snuyanzin, I'm trying to support RESPECT NULLS | IGNORE NULLS
syntax.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@snuyanzin
ListView<T>
cannot hold null values, and since it is a generic List,
it cannot be represented by a specific object for null values. Is there any good solution for this?
Line 65 in d92ab39
private static final StringData NULL_STR = StringData.fromString("null"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it's a very good point. It would be nice to be compatible with SQL and other vendors.
One idea is we could keep the null indices and set those while retrieving the result. Happy to hear better solutions though.
Another is to wrap all values in a GenericRowData
, but we'd need to iterate over the data to unwrap it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I was also thinking about storing null indexes
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @dawidwys, @snuyanzin,
I referenced LagAggFunction
to use LinkedList
to hold null values and it works. Now RESPECT NULLS
and IGNORE NULLS
are supported.
Lines 147 to 150 in 1ffb481
public static class LagAcc<T> { | |
public int offset = 1; | |
public T defaultValue = null; | |
public LinkedList<T> buffer = new LinkedList<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The downside of the solution is that the array must fit into memory at all times. The difference with the LagAggFunction
is that LAG
keeps at most n
elements where n
is controlled by the user.
Still I am reasonably good with the LinkedList
approach because it anyhow needs to fit into memory when we emit it at the end as a single record. Writing this down for awareness.
Arrays.asList( | ||
Row.of("A", new Integer[] {1, 2}), | ||
Row.of("B", new Integer[] {2, 3}), | ||
Row.of("C", new Integer[] {3}), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
same here
based on input I would expect
Row.of("C", new Integer[] {3, null}),
8ce0f58
to
14222b4
Compare
Thanks @snuyanzin @dawidwys for the review. |
TestSpec.forFunction(BuiltInFunctionDefinitions.ARRAY_AGG) | ||
.withDescription("ARRAY changelog stream aggregation") | ||
.withSource( | ||
ROW(STRING(), INT()), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One more thing
currently it checks only ability to work with INT
input for ARRAY_AGG
it would be great to have tests for other types
especially ROW
, ARRAY
, MAP
where expected output should be ARRAY<ROW>
, ARRAY<ARRAY>
, ARRAY<MAP>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Full types test is in ArrayAggFunctionTest
.
Thanks for addressing comments |
207c452
to
68799f2
Compare
Yes, ORDER BY allows sorting of any field in the input rows, but currently it is difficult to obtain the complete input rows for sorting in the function implementation. Therefore, the @snuyanzin, please help take a look again when you have time. |
@flinkbot run azure |
68799f2
to
7f6dd08
Compare
docs/data/sql_functions.yml
Outdated
description: | | ||
By default or with keyword `ALL` and, return an array that concatenates the input rows | ||
and returns `NULL` if there are no input rows. Use `DISTINCT` for one unique instance of each value. | ||
By default null values are respected, use `IGNORE NULLS` to skip null values. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
By default null values are respected, use `IGNORE NULLS` to skip null values. | |
By default `NULL` values are respected, use `IGNORE NULLS` to skip `NULL` values. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks ok from my side
@dawidwys do you have anything to add (asking since you're also participating here)
@flinkbot run azure |
Hi @dawidwys, please help review it again when you have time. |
Sorry, it takes such a long time from my side. I had a vacation in the meantime. I'll try to check it Monday. Nevertheless if you're comfortable with the PR @snuyanzin feel free to merge it without waiting for my review. |
I think we could wait until Monday or even more since right now there is a feature freeze and need to wait for cutting release branch |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I put some comments to the implementation
|
||
/** | ||
* Use the definitions in Flink instead of {@link SqlLibraryOperators#ARRAY_AGG}, because we | ||
* ignore nulls and returns nullable ARRAY type. Order by clause like <code> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
is the comment still correct after the last changes?
Arrays.asList( | ||
Row.of("A", new Integer[] {1, 2}), | ||
Row.of("B", new Integer[] {2, 2, 3}), | ||
Row.of("C", new Integer[] {3}), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The downside of the solution is that the array must fit into memory at all times. The difference with the LagAggFunction
is that LAG
keeps at most n
elements where n
is controlled by the user.
Still I am reasonably good with the LinkedList
approach because it anyhow needs to fit into memory when we emit it at the end as a single record. Writing this down for awareness.
List<T> retractBuffer = new ArrayList<>(); | ||
for (T element : acc.retractList) { | ||
retractBuffer.add(element); | ||
} | ||
for (T element : otherAcc.retractList) { | ||
retractBuffer.add(element); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do we need the retractBuffer
? Can't we just iterate over both the retractList
and create only the final newRetractBuffer
? It seems we create quite some unnecessary objects and potentially list resizing here.
|
||
// update to acc | ||
acc.list.clear(); | ||
acc.list.addAll(buffer); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can't we populate acc.list
in a single go? The current approach does make sense with a ListView
but it does not with a LinkedList
kept in memory.
I believe we don't need the intermediate buffer
and newRetractBuffer
} | ||
|
||
public void retract(ArrayAggAccumulator<T> acc, T value) throws Exception { | ||
if (value != null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what about retracting nulls?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
would be super nice to have a test case for that
6e2dc00
to
d898519
Compare
Thanks for the update @Jiabao-Sun The implementation looks good now. I want to go through the tests again, but I need a bit more time. I hope this is fine, cause anyway we need to wait for a branch cut. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM, thank you for the contribution @Jiabao-Sun !
What is the purpose of the change
[FLINK-21949][table] Support ARRAY_AGG aggregate function
Some nosql databases like mongodb and elasticsearch support nested data types.
Aggregating multiple rows into ARRAY is a common requirement.
Brief change log
Introduce built in function
ARRAY_AGG([ ALL | DISTINCT ] expression)
to return an array that concatenates the input rowsand returns NULL if there are no input rows. NULL values will be ignored. Use DISTINCT for one unique instance of each value.
Note that we have made some simplifications based on Calcite's
SqlLibraryOperators.ARRAY_AGG
.The differences from Calcite are as follows:
Verifying this change
ITCase and UnitCase are added.
Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: (yes)Documentation