-
Notifications
You must be signed in to change notification settings - Fork 13k
/
OperationExpressionsUtils.java
248 lines (212 loc) · 8.54 KB
/
OperationExpressionsUtils.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
/*
* 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.operations;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.expressions.ApiExpressionDefaultVisitor;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.FieldReferenceExpression;
import org.apache.flink.table.expressions.LocalReferenceExpression;
import org.apache.flink.table.expressions.LookupCallExpression;
import org.apache.flink.table.expressions.TableReferenceExpression;
import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
import org.apache.flink.table.functions.FunctionDefinition;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import static org.apache.flink.table.expressions.ApiExpressionUtils.call;
import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral;
import static org.apache.flink.table.expressions.ExpressionUtils.extractValue;
import static org.apache.flink.table.expressions.ExpressionUtils.isFunctionOfKind;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.WINDOW_PROPERTIES;
import static org.apache.flink.table.functions.FunctionKind.AGGREGATE;
/**
* Utility methods for transforming {@link Expression} to use them in {@link QueryOperation}s.
*/
@Internal
public class OperationExpressionsUtils {
private static final ExtractNameVisitor extractNameVisitor = new ExtractNameVisitor();
/**
* Container for extracted expressions of the same family.
*/
public static class CategorizedExpressions {
private final List<Expression> projections;
private final List<Expression> aggregations;
private final List<Expression> windowProperties;
CategorizedExpressions(
List<Expression> projections,
List<Expression> aggregations,
List<Expression> windowProperties) {
this.projections = projections;
this.aggregations = aggregations;
this.windowProperties = windowProperties;
}
public List<Expression> getProjections() {
return projections;
}
public List<Expression> getAggregations() {
return aggregations;
}
public List<Expression> getWindowProperties() {
return windowProperties;
}
}
/**
* Extracts and deduplicates all aggregation and window property expressions (zero, one, or more)
* from the given expressions.
*
* @param expressions a list of expressions to extract
* @return a Tuple2, the first field contains the extracted and deduplicated aggregations,
* and the second field contains the extracted and deduplicated window properties.
*/
public static CategorizedExpressions extractAggregationsAndProperties(List<Expression> expressions) {
AggregationAndPropertiesSplitter splitter = new AggregationAndPropertiesSplitter();
expressions.forEach(expr -> expr.accept(splitter));
List<Expression> projections = expressions.stream()
.map(expr -> expr.accept(new AggregationAndPropertiesReplacer(splitter.aggregates,
splitter.properties)))
.collect(Collectors.toList());
List<Expression> aggregates = nameExpressions(splitter.aggregates);
List<Expression> properties = nameExpressions(splitter.properties);
return new CategorizedExpressions(projections, aggregates, properties);
}
/**
* Extracts names from given expressions if they have one. Expressions that have names are:
* <ul>
* <li>{@link FieldReferenceExpression}</li>
* <li>{@link TableReferenceExpression}</li>
* <li>{@link LocalReferenceExpression}</li>
* <li>{@link BuiltInFunctionDefinitions#AS}</li>
* </ul>
*
* @param expressions list of expressions to extract names from
* @return corresponding list of optional names
*/
public static List<Optional<String>> extractNames(List<Expression> expressions) {
return expressions.stream().map(OperationExpressionsUtils::extractName).collect(Collectors.toList());
}
/**
* Extracts name from given expression if it has one. Expressions that have names are:
* <ul>
* <li>{@link FieldReferenceExpression}</li>
* <li>{@link TableReferenceExpression}</li>
* <li>{@link LocalReferenceExpression}</li>
* <li>{@link BuiltInFunctionDefinitions#AS}</li>
* </ul>
*
* @param expression expression to extract name from
* @return optional name of given expression
*/
public static Optional<String> extractName(Expression expression) {
return expression.accept(extractNameVisitor);
}
private static List<Expression> nameExpressions(Map<Expression, String> expressions) {
return expressions.entrySet()
.stream()
.map(entry -> call(AS, entry.getKey(), valueLiteral(entry.getValue())))
.collect(Collectors.toList());
}
private static class AggregationAndPropertiesSplitter extends ApiExpressionDefaultVisitor<Void> {
private int uniqueId = 0;
private final Map<Expression, String> aggregates = new LinkedHashMap<>();
private final Map<Expression, String> properties = new LinkedHashMap<>();
@Override
public Void visit(LookupCallExpression unresolvedCall) {
throw new IllegalStateException("All calls should be resolved by now. Got: " + unresolvedCall);
}
@Override
public Void visit(CallExpression call) {
FunctionDefinition functionDefinition = call.getFunctionDefinition();
if (isFunctionOfKind(call, AGGREGATE)) {
aggregates.computeIfAbsent(call, expr -> "EXPR$" + uniqueId++);
} else if (WINDOW_PROPERTIES.contains(functionDefinition)) {
properties.computeIfAbsent(call, expr -> "EXPR$" + uniqueId++);
} else {
call.getChildren().forEach(c -> c.accept(this));
}
return null;
}
@Override
protected Void defaultMethod(Expression expression) {
return null;
}
}
private static class AggregationAndPropertiesReplacer extends ApiExpressionDefaultVisitor<Expression> {
private final Map<Expression, String> aggregates;
private final Map<Expression, String> properties;
private AggregationAndPropertiesReplacer(
Map<Expression, String> aggregates,
Map<Expression, String> properties) {
this.aggregates = aggregates;
this.properties = properties;
}
@Override
public Expression visit(LookupCallExpression unresolvedCall) {
throw new IllegalStateException("All calls should be resolved by now. Got: " + unresolvedCall);
}
@Override
public Expression visit(CallExpression call) {
if (aggregates.get(call) != null) {
return unresolvedRef(aggregates.get(call));
} else if (properties.get(call) != null) {
return unresolvedRef(properties.get(call));
}
final Expression[] args = call.getChildren()
.stream()
.map(c -> c.accept(this))
.toArray(Expression[]::new);
return call(call.getFunctionDefinition(), args);
}
@Override
protected Expression defaultMethod(Expression expression) {
return expression;
}
}
private static class ExtractNameVisitor extends ApiExpressionDefaultVisitor<Optional<String>> {
@Override
public Optional<String> visit(CallExpression call) {
if (call.getFunctionDefinition().equals(AS)) {
return extractValue(call.getChildren().get(1), String.class);
} else {
return Optional.empty();
}
}
@Override
public Optional<String> visit(LocalReferenceExpression localReference) {
return Optional.of(localReference.getName());
}
@Override
public Optional<String> visit(TableReferenceExpression tableReference) {
return Optional.of(tableReference.getName());
}
@Override
public Optional<String> visit(FieldReferenceExpression fieldReference) {
return Optional.of(fieldReference.getName());
}
@Override
protected Optional<String> defaultMethod(Expression expression) {
return Optional.empty();
}
}
private OperationExpressionsUtils() {
}
}