Skip to content
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.

Commit 3cbdb03

Browse files
committedApr 9, 2025
remaining renames
1 parent eefe895 commit 3cbdb03

File tree

3 files changed

+112
-109
lines changed

3 files changed

+112
-109
lines changed
 

‎flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java

+30-30
Original file line numberDiff line numberDiff line change
@@ -560,12 +560,12 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
560560
BuiltInFunctionDefinition.newBuilder()
561561
.name("ifThenElse")
562562
.callSyntax(
563-
(sqlName, operands, context) ->
563+
(sqlName, operands, sqlFactory) ->
564564
String.format(
565565
"CASE WHEN %s THEN %s ELSE %s END",
566-
operands.get(0).asSerializableString(context),
567-
operands.get(1).asSerializableString(context),
568-
operands.get(2).asSerializableString(context)))
566+
operands.get(0).asSerializableString(sqlFactory),
567+
operands.get(1).asSerializableString(sqlFactory),
568+
operands.get(2).asSerializableString(sqlFactory)))
569569
.kind(SCALAR)
570570
.inputTypeStrategy(
571571
compositeSequence()
@@ -692,15 +692,15 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
692692
.name("between")
693693
.kind(SCALAR)
694694
.callSyntax(
695-
(sqlName, operands, context) ->
695+
(sqlName, operands, sqlFactory) ->
696696
String.format(
697697
"%s BETWEEN %s AND %s",
698698
CallSyntaxUtils.asSerializableOperand(
699-
operands.get(0), context),
699+
operands.get(0), sqlFactory),
700700
CallSyntaxUtils.asSerializableOperand(
701-
operands.get(1), context),
701+
operands.get(1), sqlFactory),
702702
CallSyntaxUtils.asSerializableOperand(
703-
operands.get(2), context)))
703+
operands.get(2), sqlFactory)))
704704
.inputTypeStrategy(
705705
comparable(ConstantArgumentCount.of(3), StructuredComparison.FULL))
706706
.outputTypeStrategy(nullableIfArgs(explicit(DataTypes.BOOLEAN())))
@@ -710,15 +710,15 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
710710
BuiltInFunctionDefinition.newBuilder()
711711
.name("notBetween")
712712
.callSyntax(
713-
(sqlName, operands, context) ->
713+
(sqlName, operands, sqlFactory) ->
714714
String.format(
715715
"%s NOT BETWEEN %s AND %s",
716716
CallSyntaxUtils.asSerializableOperand(
717-
operands.get(0), context),
717+
operands.get(0), sqlFactory),
718718
CallSyntaxUtils.asSerializableOperand(
719-
operands.get(1), context),
719+
operands.get(1), sqlFactory),
720720
CallSyntaxUtils.asSerializableOperand(
721-
operands.get(2), context)))
721+
operands.get(2), sqlFactory)))
722722
.kind(SCALAR)
723723
.inputTypeStrategy(
724724
comparable(ConstantArgumentCount.of(3), StructuredComparison.FULL))
@@ -1139,11 +1139,11 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
11391139
BuiltInFunctionDefinition.newBuilder()
11401140
.name("position")
11411141
.callSyntax(
1142-
(sqlName, operands, context) ->
1142+
(sqlName, operands, sqlFactory) ->
11431143
String.format(
11441144
"POSITION(%s IN %s)",
1145-
operands.get(0).asSerializableString(context),
1146-
operands.get(1).asSerializableString(context)))
1145+
operands.get(0).asSerializableString(sqlFactory),
1146+
operands.get(1).asSerializableString(sqlFactory)))
11471147
.kind(SCALAR)
11481148
.inputTypeStrategy(
11491149
sequence(
@@ -2153,15 +2153,15 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
21532153
.name("extract")
21542154
.callSyntax(
21552155
"EXTRACT",
2156-
(sqlName, operands, context) ->
2156+
(sqlName, operands, sqlFactory) ->
21572157
String.format(
21582158
"%s(%s %s %s)",
21592159
sqlName,
21602160
((ValueLiteralExpression) operands.get(0))
21612161
.getValueAs(TimeIntervalUnit.class)
21622162
.get(),
21632163
"FROM",
2164-
operands.get(1).asSerializableString(context)))
2164+
operands.get(1).asSerializableString(sqlFactory)))
21652165
.kind(SCALAR)
21662166
.inputTypeStrategy(SpecificInputTypeStrategies.EXTRACT)
21672167
.outputTypeStrategy(nullableIfArgs(explicit(BIGINT())))
@@ -2230,13 +2230,13 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
22302230
BuiltInFunctionDefinition.newBuilder()
22312231
.name("temporalOverlaps")
22322232
.callSyntax(
2233-
(sqlName, operands, context) ->
2233+
(sqlName, operands, sqlFactory) ->
22342234
String.format(
22352235
"(%s, %s) OVERLAPS (%s, %s)",
2236-
operands.get(0).asSerializableString(context),
2237-
operands.get(1).asSerializableString(context),
2238-
operands.get(2).asSerializableString(context),
2239-
operands.get(3).asSerializableString(context)))
2236+
operands.get(0).asSerializableString(sqlFactory),
2237+
operands.get(1).asSerializableString(sqlFactory),
2238+
operands.get(2).asSerializableString(sqlFactory),
2239+
operands.get(3).asSerializableString(sqlFactory)))
22402240
.kind(SCALAR)
22412241
.inputTypeStrategy(SpecificInputTypeStrategies.TEMPORAL_OVERLAPS)
22422242
.outputTypeStrategy(nullableIfArgs(explicit(BOOLEAN())))
@@ -2264,7 +2264,7 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
22642264
.kind(SCALAR)
22652265
.callSyntax(
22662266
"TIMESTAMPDIFF",
2267-
(sqlName, operands, context) ->
2267+
(sqlName, operands, sqlFactory) ->
22682268
String.format(
22692269
"%s(%s, %s)",
22702270
sqlName,
@@ -2276,7 +2276,7 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
22762276
resolvedExpression ->
22772277
resolvedExpression
22782278
.asSerializableString(
2279-
context))
2279+
sqlFactory))
22802280
.collect(Collectors.joining(", "))))
22812281
.inputTypeStrategy(
22822282
sequence(
@@ -2395,11 +2395,11 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
23952395
BuiltInFunctionDefinition.newBuilder()
23962396
.name("at")
23972397
.callSyntax(
2398-
(sqlName, operands, context) ->
2398+
(sqlName, operands, sqlFactory) ->
23992399
String.format(
24002400
"%s[%s]",
2401-
operands.get(0).asSerializableString(context),
2402-
operands.get(1).asSerializableString(context)))
2401+
operands.get(0).asSerializableString(sqlFactory),
2402+
operands.get(1).asSerializableString(sqlFactory)))
24032403
.kind(SCALAR)
24042404
.inputTypeStrategy(
24052405
sequence(
@@ -2477,7 +2477,7 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
24772477
BuiltInFunctionDefinition.newBuilder()
24782478
.name("get")
24792479
.callSyntax(
2480-
(sqlName, operands, context) -> {
2480+
(sqlName, operands, sqlFactory) -> {
24812481
final Optional<String> fieldName =
24822482
((ValueLiteralExpression) operands.get(1))
24832483
.getValueAs(String.class);
@@ -2489,12 +2489,12 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
24892489
"%s.%s",
24902490
operands.get(0)
24912491
.asSerializableString(
2492-
context),
2492+
sqlFactory),
24932493
EncodingUtils.escapeIdentifier(n)))
24942494
.orElseGet(
24952495
() ->
24962496
SqlCallSyntax.FUNCTION.unparse(
2497-
sqlName, operands, context));
2497+
sqlName, operands, sqlFactory));
24982498
})
24992499
.kind(OTHER)
25002500
.inputTypeStrategy(

‎flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/JsonFunctionsCallSyntax.java

+28-26
Original file line numberDiff line numberDiff line change
@@ -36,11 +36,11 @@
3636
class JsonFunctionsCallSyntax {
3737

3838
static final SqlCallSyntax IS_JSON =
39-
(sqlName, operands, context) -> {
39+
(sqlName, operands, sqlFactory) -> {
4040
final String s =
4141
String.format(
4242
"%s IS JSON",
43-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context));
43+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory));
4444
if (operands.size() > 1) {
4545
return s + " " + getSymbolLiteral(operands.get(1), JsonType.class);
4646
}
@@ -49,22 +49,23 @@ class JsonFunctionsCallSyntax {
4949
};
5050

5151
static final SqlCallSyntax JSON_VALUE =
52-
(sqlName, operands, context) -> {
52+
(sqlName, operands, sqlFactory) -> {
5353
StringBuilder s =
5454
new StringBuilder(
5555
String.format(
5656
"JSON_VALUE(%s, %s RETURNING %s ",
57-
operands.get(0).asSerializableString(context),
58-
operands.get(1).asSerializableString(context),
59-
operands.get(2).asSerializableString(context)));
57+
operands.get(0).asSerializableString(sqlFactory),
58+
operands.get(1).asSerializableString(sqlFactory),
59+
operands.get(2).asSerializableString(sqlFactory)));
6060

6161
final JsonValueOnEmptyOrError onEmpty =
6262
getSymbolLiteral(operands.get(3), JsonValueOnEmptyOrError.class);
6363

6464
if (onEmpty == JsonValueOnEmptyOrError.DEFAULT) {
6565
s.append(
6666
String.format(
67-
"DEFAULT %s", operands.get(4).asSerializableString(context)));
67+
"DEFAULT %s",
68+
operands.get(4).asSerializableString(sqlFactory)));
6869
} else {
6970
s.append(onEmpty);
7071
}
@@ -76,7 +77,8 @@ class JsonFunctionsCallSyntax {
7677
if (onError == JsonValueOnEmptyOrError.DEFAULT) {
7778
s.append(
7879
String.format(
79-
"DEFAULT %s", operands.get(6).asSerializableString(context)));
80+
"DEFAULT %s",
81+
operands.get(6).asSerializableString(sqlFactory)));
8082
} else {
8183
s.append(onError);
8284
}
@@ -86,21 +88,21 @@ class JsonFunctionsCallSyntax {
8688
};
8789

8890
static final SqlCallSyntax JSON_EXISTS =
89-
(sqlName, operands, context) -> {
91+
(sqlName, operands, sqlFactory) -> {
9092
if (operands.size() == 3) {
9193
return String.format(
9294
"%s(%s, %s %s ON ERROR)",
9395
sqlName,
94-
operands.get(0).asSerializableString(context),
95-
operands.get(1).asSerializableString(context),
96+
operands.get(0).asSerializableString(sqlFactory),
97+
operands.get(1).asSerializableString(sqlFactory),
9698
getSymbolLiteral(operands.get(2), JsonExistsOnError.class));
9799
} else {
98-
return SqlCallSyntax.FUNCTION.unparse(sqlName, operands, context);
100+
return SqlCallSyntax.FUNCTION.unparse(sqlName, operands, sqlFactory);
99101
}
100102
};
101103

102104
static final SqlCallSyntax JSON_QUERY =
103-
(sqlName, operands, context) -> {
105+
(sqlName, operands, sqlFactory) -> {
104106
final JsonQueryWrapper wrapper =
105107
getSymbolLiteral(operands.get(3), JsonQueryWrapper.class);
106108
final JsonQueryOnEmptyOrError onEmpty =
@@ -110,62 +112,62 @@ class JsonFunctionsCallSyntax {
110112

111113
return String.format(
112114
"JSON_QUERY(%s, %s RETURNING %s %s WRAPPER %s ON EMPTY %s ON ERROR)",
113-
operands.get(0).asSerializableString(context),
114-
operands.get(1).asSerializableString(context),
115-
operands.get(2).asSerializableString(context),
115+
operands.get(0).asSerializableString(sqlFactory),
116+
operands.get(1).asSerializableString(sqlFactory),
117+
operands.get(2).asSerializableString(sqlFactory),
116118
toString(wrapper),
117119
onEmpty.toString().replaceAll("_", " "),
118120
onError.toString().replaceAll("_", " "));
119121
};
120122

121123
static final SqlCallSyntax JSON_OBJECT =
122-
(sqlName, operands, context) -> {
124+
(sqlName, operands, sqlFactory) -> {
123125
final String entries =
124126
IntStream.range(0, operands.size() / 2)
125127
.mapToObj(
126128
i ->
127129
String.format(
128130
"KEY %s VALUE %s",
129131
operands.get(2 * i + 1)
130-
.asSerializableString(context),
132+
.asSerializableString(sqlFactory),
131133
operands.get(2 * i + 2)
132-
.asSerializableString(context)))
134+
.asSerializableString(sqlFactory)))
133135
.collect(Collectors.joining(", "));
134136

135137
final JsonOnNull onNull = getSymbolLiteral(operands.get(0), JsonOnNull.class);
136138
return String.format("JSON_OBJECT(%s %s ON NULL)", entries, onNull);
137139
};
138140

139141
static final SqlCallSyntax JSON_ARRAY =
140-
(sqlName, operands, context) -> {
142+
(sqlName, operands, sqlFactory) -> {
141143
if (operands.size() == 1) {
142144
return "JSON_ARRAY()";
143145
}
144146
final String entries =
145147
operands.subList(1, operands.size()).stream()
146148
.map(
147149
resolvedExpression ->
148-
resolvedExpression.asSerializableString(context))
150+
resolvedExpression.asSerializableString(sqlFactory))
149151
.collect(Collectors.joining(", "));
150152

151153
final JsonOnNull onNull = getSymbolLiteral(operands.get(0), JsonOnNull.class);
152154
return String.format("JSON_ARRAY(%s %s ON NULL)", entries, onNull);
153155
};
154156

155157
static SqlCallSyntax jsonArrayAgg(JsonOnNull onNull) {
156-
return (sqlName, operands, context) ->
158+
return (sqlName, operands, sqlFactory) ->
157159
String.format(
158160
"%s(%s %s ON NULL)",
159-
sqlName, operands.get(0).asSerializableString(context), onNull);
161+
sqlName, operands.get(0).asSerializableString(sqlFactory), onNull);
160162
}
161163

162164
static SqlCallSyntax jsonObjectAgg(JsonOnNull onNull) {
163-
return (sqlName, operands, context) ->
165+
return (sqlName, operands, sqlFactory) ->
164166
String.format(
165167
"%s(KEY %s VALUE %s %s ON NULL)",
166168
sqlName,
167-
operands.get(0).asSerializableString(context),
168-
operands.get(1).asSerializableString(context),
169+
operands.get(0).asSerializableString(sqlFactory),
170+
operands.get(1).asSerializableString(sqlFactory),
169171
onNull);
170172
}
171173

‎flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/SqlCallSyntax.java

+54-53
Original file line numberDiff line numberDiff line change
@@ -85,14 +85,14 @@ private String doUnParse(
8585
* Function syntax for functions without parenthesis (e.g., CURRENT_DATE, LOCALTIMESTAMP,
8686
* LOCALTIME, CURRENT_TIMESTAMP, CURRENT_TIME).
8787
*/
88-
SqlCallSyntax NO_PARENTHESIS = (sqlName, operands, context) -> sqlName;
88+
SqlCallSyntax NO_PARENTHESIS = (sqlName, operands, sqlFactory) -> sqlName;
8989

9090
/**
9191
* Function syntax for handling DISTINCT aggregates. Special case. It does not have a syntax
9292
* itself, but modifies the syntax of the nested call.
9393
*/
9494
SqlCallSyntax DISTINCT =
95-
(sqlName, operands, context) -> {
95+
(sqlName, operands, sqlFactory) -> {
9696
final CallExpression callExpression = (CallExpression) operands.get(0);
9797
if (callExpression.getFunctionDefinition() instanceof BuiltInFunctionDefinition) {
9898
final BuiltInFunctionDefinition builtinDefinition =
@@ -102,40 +102,40 @@ private String doUnParse(
102102
.unparseDistinct(
103103
builtinDefinition.getSqlName(),
104104
callExpression.getResolvedChildren(),
105-
context);
105+
sqlFactory);
106106
} else {
107107
return SqlCallSyntax.FUNCTION.unparseDistinct(
108108
callExpression.getFunctionName(),
109109
callExpression.getResolvedChildren(),
110-
context);
110+
sqlFactory);
111111
}
112112
};
113113

114114
/** Function syntax for collection ctors, such as ARRAY[1, 2, 3] or MAP['a', 1, 'b', 2]. */
115115
SqlCallSyntax COLLECTION_CTOR =
116-
(sqlName, operands, context) ->
116+
(sqlName, operands, sqlFactory) ->
117117
String.format(
118118
"%s[%s]",
119119
sqlName,
120120
operands.stream()
121121
.map(
122122
resolvedExpression ->
123123
resolvedExpression.asSerializableString(
124-
context))
124+
sqlFactory))
125125
.collect(Collectors.joining(", ")));
126126

127127
/** Binary operator syntax, as in "x - y". */
128128
SqlCallSyntax BINARY_OP =
129-
(sqlName, operands, context) ->
129+
(sqlName, operands, sqlFactory) ->
130130
String.format(
131131
"%s %s %s",
132-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context),
132+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory),
133133
sqlName,
134-
CallSyntaxUtils.asSerializableOperand(operands.get(1), context));
134+
CallSyntaxUtils.asSerializableOperand(operands.get(1), sqlFactory));
135135

136136
/** Syntax for unparsing '+', Special handling for a plus on string arguments. */
137137
SqlCallSyntax PLUS_OP =
138-
(sqlName, operands, context) -> {
138+
(sqlName, operands, sqlFactory) -> {
139139
boolean isString =
140140
operands.stream()
141141
.anyMatch(
@@ -145,9 +145,9 @@ private String doUnParse(
145145
.is(LogicalTypeFamily.CHARACTER_STRING));
146146
if (isString) {
147147
return FUNCTION.unparse(
148-
BuiltInFunctionDefinitions.CONCAT.getSqlName(), operands, context);
148+
BuiltInFunctionDefinitions.CONCAT.getSqlName(), operands, sqlFactory);
149149
} else {
150-
return BINARY_OP.unparse(sqlName, operands, context);
150+
return BINARY_OP.unparse(sqlName, operands, sqlFactory);
151151
}
152152
};
153153

@@ -156,60 +156,60 @@ private String doUnParse(
156156
* AND w".
157157
*/
158158
SqlCallSyntax MULTIPLE_BINARY_OP =
159-
(sqlName, operands, context) ->
159+
(sqlName, operands, sqlFactory) ->
160160
operands.stream()
161161
.map(
162162
expression ->
163163
CallSyntaxUtils.asSerializableOperand(
164-
expression, context))
164+
expression, sqlFactory))
165165
.collect(Collectors.joining(String.format(" %s ", sqlName)));
166166

167167
/** Postfix unary operator syntax, as in "x ++". */
168168
SqlCallSyntax UNARY_SUFFIX_OP =
169-
(sqlName, operands, context) ->
169+
(sqlName, operands, sqlFactory) ->
170170
String.format(
171171
"%s %s",
172-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context),
172+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory),
173173
sqlName);
174174

175175
/** Prefix unary operator syntax, as in "- x". */
176176
SqlCallSyntax UNARY_PREFIX_OP =
177-
(sqlName, operands, context) ->
177+
(sqlName, operands, sqlFactory) ->
178178
String.format(
179179
"%s %s",
180180
sqlName,
181-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context));
181+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory));
182182

183183
/**
184184
* Special sql syntax for CAST operators (CAST, TRY_CAST, REINTERPRET_CAST).
185185
*
186186
* <p>Example: CAST(123 AS STRING)
187187
*/
188188
SqlCallSyntax CAST =
189-
(sqlName, operands, context) ->
189+
(sqlName, operands, sqlFactory) ->
190190
String.format(
191191
"%s(%s AS %s)",
192192
sqlName,
193-
operands.get(0).asSerializableString(context),
194-
operands.get(1).asSerializableString(context));
193+
operands.get(0).asSerializableString(sqlFactory),
194+
operands.get(1).asSerializableString(sqlFactory));
195195

196196
/**
197197
* Special sql syntax for SUBSTRING operators (SUBSTRING, SUBSTR).
198198
*
199199
* <p>Example: SUBSTR('abc' FROM 'abcdef' FOR 3)
200200
*/
201201
SqlCallSyntax SUBSTRING =
202-
(sqlName, operands, context) -> {
202+
(sqlName, operands, sqlFactory) -> {
203203
final String s =
204204
String.format(
205205
"%s(%s FROM %s",
206206
sqlName,
207-
operands.get(0).asSerializableString(context),
208-
operands.get(1).asSerializableString(context));
207+
operands.get(0).asSerializableString(sqlFactory),
208+
operands.get(1).asSerializableString(sqlFactory));
209209
if (operands.size() == 3) {
210210
return s
211211
+ String.format(
212-
" FOR %s)", operands.get(2).asSerializableString(context));
212+
" FOR %s)", operands.get(2).asSerializableString(sqlFactory));
213213
}
214214

215215
return s + ")";
@@ -226,16 +226,16 @@ private String doUnParse(
226226
* </ul>
227227
*/
228228
SqlCallSyntax FLOOR_OR_CEIL =
229-
(sqlName, operands, context) -> {
229+
(sqlName, operands, sqlFactory) -> {
230230
if (operands.size() == 1) {
231231
// case for numeric floor & ceil
232-
return SqlCallSyntax.FUNCTION.unparse(sqlName, operands, context);
232+
return SqlCallSyntax.FUNCTION.unparse(sqlName, operands, sqlFactory);
233233
} else {
234234
// case for flooring/ceiling to temporal units
235235
return String.format(
236236
"%s(%s TO %s)",
237237
sqlName,
238-
operands.get(0).asSerializableString(context),
238+
operands.get(0).asSerializableString(sqlFactory),
239239
((ValueLiteralExpression) operands.get(1))
240240
.getValueAs(TimeIntervalUnit.class)
241241
.get());
@@ -248,7 +248,7 @@ private String doUnParse(
248248
* <p>Example: TRIM(BOTH ' ' FROM ' 0 ');
249249
*/
250250
SqlCallSyntax TRIM =
251-
(sqlName, operands, context) -> {
251+
(sqlName, operands, sqlFactory) -> {
252252
final boolean trimLeading =
253253
((ValueLiteralExpression) operands.get(0)).getValueAs(Boolean.class).get();
254254
final boolean trimTrailing =
@@ -268,8 +268,8 @@ private String doUnParse(
268268

269269
return String.format(
270270
format,
271-
operands.get(2).asSerializableString(context),
272-
operands.get(3).asSerializableString(context));
271+
operands.get(2).asSerializableString(sqlFactory),
272+
operands.get(3).asSerializableString(sqlFactory));
273273
};
274274

275275
/**
@@ -278,27 +278,27 @@ private String doUnParse(
278278
* <p>Example: OVERLAY('abcd' PLACING 'def' FROM 3 FOR 2)
279279
*/
280280
SqlCallSyntax OVERLAY =
281-
(sqlName, operands, context) -> {
281+
(sqlName, operands, sqlFactory) -> {
282282
final String s =
283283
String.format(
284284
"OVERLAY(%s PLACING %s FROM %s",
285-
operands.get(0).asSerializableString(context),
286-
operands.get(1).asSerializableString(context),
287-
operands.get(2).asSerializableString(context));
285+
operands.get(0).asSerializableString(sqlFactory),
286+
operands.get(1).asSerializableString(sqlFactory),
287+
operands.get(2).asSerializableString(sqlFactory));
288288

289289
// optional length
290290
if (operands.size() == 4) {
291291
return s
292292
+ String.format(
293-
" FOR %s)", operands.get(3).asSerializableString(context));
293+
" FOR %s)", operands.get(3).asSerializableString(sqlFactory));
294294
}
295295

296296
return s + ")";
297297
};
298298

299299
/** Special sql syntax for AS. The string literal is formatted as an identifier. */
300300
SqlCallSyntax AS =
301-
(sqlName, operands, context) -> {
301+
(sqlName, operands, sqlFactory) -> {
302302
if (operands.size() != 2) {
303303
throw new TableException(
304304
"The AS function with multiple aliases is not SQL"
@@ -308,63 +308,64 @@ private String doUnParse(
308308
final String identifier = ExpressionUtils.stringValue(operands.get(1));
309309
return String.format(
310310
"%s %s %s",
311-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context),
311+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory),
312312
sqlName,
313313
EncodingUtils.escapeIdentifier(identifier));
314314
};
315315

316316
/** Call syntax for {@link BuiltInFunctionDefinitions#IN}. */
317317
SqlCallSyntax IN =
318-
(sqlName, operands, context) ->
318+
(sqlName, operands, sqlFactory) ->
319319
String.format(
320320
"%s IN (%s)",
321-
operands.get(0).asSerializableString(context),
321+
operands.get(0).asSerializableString(sqlFactory),
322322
operands.subList(1, operands.size()).stream()
323323
.map(
324324
resolvedExpression ->
325325
resolvedExpression.asSerializableString(
326-
context))
326+
sqlFactory))
327327
.collect(Collectors.joining(", ")));
328328

329-
SqlCallSyntax WINDOW_START_END = (sqlName, operands, context) -> String.format("%s", sqlName);
329+
SqlCallSyntax WINDOW_START_END =
330+
(sqlName, operands, sqlFactory) -> String.format("%s", sqlName);
330331

331332
/**
332333
* Special sql syntax for LIKE.
333334
*
334335
* <p>Example: 'TE_ST' LIKE '%E&_S%' ESCAPE '&';
335336
*/
336337
SqlCallSyntax LIKE =
337-
(sqlName, operands, context) -> {
338+
(sqlName, operands, sqlFactory) -> {
338339
if (operands.size() == 2) {
339340
return String.format(
340341
"%s %s %s",
341-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context),
342+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory),
342343
sqlName,
343-
CallSyntaxUtils.asSerializableOperand(operands.get(1), context));
344+
CallSyntaxUtils.asSerializableOperand(operands.get(1), sqlFactory));
344345
} else {
345346
return String.format(
346347
"%s %s %s ESCAPE %s",
347-
CallSyntaxUtils.asSerializableOperand(operands.get(0), context),
348+
CallSyntaxUtils.asSerializableOperand(operands.get(0), sqlFactory),
348349
sqlName,
349-
CallSyntaxUtils.asSerializableOperand(operands.get(1), context),
350-
CallSyntaxUtils.asSerializableOperand(operands.get(2), context));
350+
CallSyntaxUtils.asSerializableOperand(operands.get(1), sqlFactory),
351+
CallSyntaxUtils.asSerializableOperand(operands.get(2), sqlFactory));
351352
}
352353
};
353354

354355
SqlCallSyntax OVER =
355-
((sqlName, operands, context) -> {
356-
String projection = operands.get(0).asSerializableString(context);
357-
String order = operands.get(1).asSerializableString(context);
356+
((sqlName, operands, sqlFactory) -> {
357+
String projection = operands.get(0).asSerializableString(sqlFactory);
358+
String order = operands.get(1).asSerializableString(sqlFactory);
358359
String rangeBounds =
359360
CallSyntaxUtils.overRangeToSerializableString(
360-
operands.get(2), operands.get(3), context);
361+
operands.get(2), operands.get(3), sqlFactory);
361362
if (operands.size() == 4) {
362363
return String.format("%s OVER(ORDER BY %s%s)", projection, order, rangeBounds);
363364
} else {
364365
return String.format(
365366
"%s OVER(PARTITION BY %s ORDER BY %s%s)",
366367
projection,
367-
CallSyntaxUtils.asSerializableOperand(operands.get(4), context),
368+
CallSyntaxUtils.asSerializableOperand(operands.get(4), sqlFactory),
368369
order,
369370
rangeBounds);
370371
}

0 commit comments

Comments
 (0)
Please sign in to comment.