-
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
Changes from all commits
c3d2839
d46b2f3
58bb064
ccabc46
96c5c2c
a7e114f
01051a2
d898519
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -201,6 +201,7 @@ | |
"AFTER" | ||
"ALWAYS" | ||
"APPLY" | ||
"ARRAY_AGG" | ||
"ASC" | ||
"ASSERTION" | ||
"ASSIGNMENT" | ||
|
Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
@@ -0,0 +1,91 @@ | ||||||||||||||||||||||||||||||||||||||||
/* | ||||||||||||||||||||||||||||||||||||||||
* Licensed to the Apache Software Foundation (ASF) under one | ||||||||||||||||||||||||||||||||||||||||
* or more contributor license agreements. See the NOTICE file | ||||||||||||||||||||||||||||||||||||||||
* distributed with this work for additional information | ||||||||||||||||||||||||||||||||||||||||
* regarding copyright ownership. The ASF licenses this file | ||||||||||||||||||||||||||||||||||||||||
* to you under the Apache License, Version 2.0 (the | ||||||||||||||||||||||||||||||||||||||||
* "License"); you may not use this file except in compliance | ||||||||||||||||||||||||||||||||||||||||
* with the License. You may obtain a copy of the License at | ||||||||||||||||||||||||||||||||||||||||
* | ||||||||||||||||||||||||||||||||||||||||
* http://www.apache.org/licenses/LICENSE-2.0 | ||||||||||||||||||||||||||||||||||||||||
* | ||||||||||||||||||||||||||||||||||||||||
* Unless required by applicable law or agreed to in writing, software | ||||||||||||||||||||||||||||||||||||||||
* distributed under the License is distributed on an "AS IS" BASIS, | ||||||||||||||||||||||||||||||||||||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||||||||||||||||||||||||||||||||||||||
* See the License for the specific language governing permissions and | ||||||||||||||||||||||||||||||||||||||||
* limitations under the License. | ||||||||||||||||||||||||||||||||||||||||
*/ | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
package org.apache.flink.table.planner.functions; | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
import org.apache.flink.table.functions.BuiltInFunctionDefinitions; | ||||||||||||||||||||||||||||||||||||||||
import org.apache.flink.types.Row; | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
import java.util.Arrays; | ||||||||||||||||||||||||||||||||||||||||
import java.util.Collections; | ||||||||||||||||||||||||||||||||||||||||
import java.util.stream.Stream; | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.table.api.DataTypes.ARRAY; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.table.api.DataTypes.INT; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.table.api.DataTypes.ROW; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.table.api.DataTypes.STRING; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.table.api.Expressions.$; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.types.RowKind.DELETE; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.types.RowKind.INSERT; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.types.RowKind.UPDATE_AFTER; | ||||||||||||||||||||||||||||||||||||||||
import static org.apache.flink.types.RowKind.UPDATE_BEFORE; | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
/** Tests for built-in ARRAY_AGG aggregation functions. */ | ||||||||||||||||||||||||||||||||||||||||
class ArrayAggFunctionITCase extends BuiltInAggregateFunctionTestBase { | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
@Override | ||||||||||||||||||||||||||||||||||||||||
Stream<TestSpec> getTestCaseSpecs() { | ||||||||||||||||||||||||||||||||||||||||
return Stream.of( | ||||||||||||||||||||||||||||||||||||||||
TestSpec.forFunction(BuiltInFunctionDefinitions.ARRAY_AGG) | ||||||||||||||||||||||||||||||||||||||||
.withDescription("ARRAY changelog stream aggregation") | ||||||||||||||||||||||||||||||||||||||||
.withSource( | ||||||||||||||||||||||||||||||||||||||||
ROW(STRING(), INT()), | ||||||||||||||||||||||||||||||||||||||||
Arrays.asList( | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "A", 1), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "A", 2), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "B", 2), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "B", 2), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "B", 3), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "C", 3), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "C", null), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(DELETE, "C", null), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "D", null), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "E", 4), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(INSERT, "E", 5), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(DELETE, "E", 5), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(UPDATE_BEFORE, "E", 4), | ||||||||||||||||||||||||||||||||||||||||
Row.ofKind(UPDATE_AFTER, "E", 6))) | ||||||||||||||||||||||||||||||||||||||||
.testResult( | ||||||||||||||||||||||||||||||||||||||||
source -> | ||||||||||||||||||||||||||||||||||||||||
"SELECT f0, array_agg(f1) FROM " + source + " GROUP BY f0", | ||||||||||||||||||||||||||||||||||||||||
TableApiAggSpec.groupBySelect( | ||||||||||||||||||||||||||||||||||||||||
Collections.singletonList($("f0")), | ||||||||||||||||||||||||||||||||||||||||
$("f0"), | ||||||||||||||||||||||||||||||||||||||||
$("f1").arrayAgg()), | ||||||||||||||||||||||||||||||||||||||||
ROW(STRING(), ARRAY(INT())), | ||||||||||||||||||||||||||||||||||||||||
ROW(STRING(), ARRAY(INT())), | ||||||||||||||||||||||||||||||||||||||||
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 commentThe 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 commentThe 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
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
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 commentThe 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: UPD: to be more clear: I don't tell that we need to support There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks @snuyanzin, I'm trying to support There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @snuyanzin Line 65 in d92ab39
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. Hi @dawidwys, @snuyanzin, I referenced Lines 147 to 150 in 1ffb481
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 Still I am reasonably good with the |
||||||||||||||||||||||||||||||||||||||||
Row.of("D", new Integer[] {null}), | ||||||||||||||||||||||||||||||||||||||||
Row.of("E", new Integer[] {6}))) | ||||||||||||||||||||||||||||||||||||||||
.testSqlResult( | ||||||||||||||||||||||||||||||||||||||||
source -> | ||||||||||||||||||||||||||||||||||||||||
"SELECT f0, array_agg(DISTINCT f1 IGNORE NULLS) FROM " | ||||||||||||||||||||||||||||||||||||||||
+ source | ||||||||||||||||||||||||||||||||||||||||
+ " GROUP BY f0", | ||||||||||||||||||||||||||||||||||||||||
ROW(STRING(), ARRAY(INT())), | ||||||||||||||||||||||||||||||||||||||||
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 commentThe reason will be displayed to describe this comment to others. Learn more. same here Row.of("C", new Integer[] {3, null}), |
||||||||||||||||||||||||||||||||||||||||
Row.of("D", null), | ||||||||||||||||||||||||||||||||||||||||
Row.of("E", new Integer[] {6})))); | ||||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||||
} |
Uh oh!
There was an error while loading. Please reload this page.
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 forARRAY_AGG
it would be great to have tests for other types
especially
ROW
,ARRAY
,MAP
where expected output should beARRAY<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
.