Skip to content

Commit 4971601

Browse files
committed
use openContext name
1 parent 6c3275e commit 4971601

File tree

8 files changed

+60
-13
lines changed

8 files changed

+60
-13
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,36 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.api.common.functions;
20+
21+
import org.apache.flink.annotation.PublicEvolving;
22+
import org.apache.flink.configuration.Configuration;
23+
24+
/** A special {@link OpenContext} for passing configuration to udf. */
25+
@PublicEvolving
26+
public class WithConfigurationOpenContext implements OpenContext {
27+
private final Configuration configuration;
28+
29+
public WithConfigurationOpenContext(Configuration configuration) {
30+
this.configuration = configuration;
31+
}
32+
33+
public Configuration getConfiguration() {
34+
return configuration;
35+
}
36+
}

Diff for: flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionContext.java

+10-2
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,9 @@
2020

2121
import org.apache.flink.annotation.PublicEvolving;
2222
import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
23+
import org.apache.flink.api.common.functions.OpenContext;
2324
import org.apache.flink.api.common.functions.RuntimeContext;
25+
import org.apache.flink.api.common.functions.WithConfigurationOpenContext;
2426
import org.apache.flink.configuration.Configuration;
2527
import org.apache.flink.metrics.MetricGroup;
2628
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
@@ -62,10 +64,16 @@ public class FunctionContext {
6264
public FunctionContext(
6365
@Nullable RuntimeContext context,
6466
@Nullable ClassLoader userClassLoader,
65-
@Nullable Configuration jobParameters) {
67+
@Nullable OpenContext openContext) {
6668
this.context = context;
6769
this.userClassLoader = userClassLoader;
68-
this.jobParameters = jobParameters != null ? jobParameters.toMap() : null;
70+
if (openContext instanceof WithConfigurationOpenContext) {
71+
Configuration configuration =
72+
((WithConfigurationOpenContext) openContext).getConfiguration();
73+
this.jobParameters = configuration.toMap();
74+
} else {
75+
this.jobParameters = null;
76+
}
6977
}
7078

7179
public FunctionContext(RuntimeContext context) {

Diff for: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CollectorCodeGenerator.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ object CollectorCodeGenerator {
7575
}
7676

7777
@Override
78-
public void open(${className[OpenContext]} context) throws Exception {
78+
public void open(${className[OpenContext]} openContext) throws Exception {
7979
${ctx.reuseOpenCode()}
8080
}
8181

@@ -143,7 +143,7 @@ object CollectorCodeGenerator {
143143
}
144144

145145
@Override
146-
public void open(${className[OpenContext]} context) throws Exception {
146+
public void open(${className[OpenContext]} openContext) throws Exception {
147147
${ctx.reuseOpenCode()}
148148
}
149149

Diff for: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala

+6-3
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
*/
1818
package org.apache.flink.table.planner.codegen
1919

20-
import org.apache.flink.api.common.functions.{DefaultOpenContext, MapFunction, OpenContext, RichMapFunction}
20+
import org.apache.flink.api.common.functions.{DefaultOpenContext, MapFunction, OpenContext, RichMapFunction, WithConfigurationOpenContext}
2121
import org.apache.flink.configuration.{Configuration, PipelineOptions, ReadableConfig}
2222
import org.apache.flink.table.api.{TableConfig, TableException}
2323
import org.apache.flink.table.data.{DecimalData, GenericRowData, TimestampData}
@@ -102,9 +102,12 @@ class ExpressionReducer(
102102
throw new TableException("RichMapFunction[GenericRowData, GenericRowData] required here")
103103
}
104104

105+
val parameters = toScala(tableConfig.getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS))
106+
.map(Configuration.fromMap)
107+
.getOrElse(new Configuration)
105108
val reduced =
106109
try {
107-
richMapFunction.open(DefaultOpenContext.INSTANCE)
110+
richMapFunction.open(new WithConfigurationOpenContext(parameters))
108111
// execute
109112
richMapFunction.map(EMPTY_ROW)
110113
} catch {
@@ -312,7 +315,7 @@ class ConstantCodeGeneratorContext(tableConfig: ReadableConfig, classLoader: Cla
312315
super.addReusableFunction(
313316
function,
314317
classOf[FunctionContext],
315-
Seq("null", "this.getClass().getClassLoader()", "context"))
318+
Seq("null", "this.getClass().getClassLoader()", "openContext"))
316319
}
317320

318321
override def addReusableConverter(dataType: DataType, classLoaderTerm: String = null): String = {

Diff for: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCodeGenerator.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -154,7 +154,7 @@ object FunctionCodeGenerator {
154154
${ctx.reuseConstructorCode(funcName)}
155155

156156
@Override
157-
public void open(${classOf[OpenContext].getCanonicalName} context) throws Exception {
157+
public void open(${classOf[OpenContext].getCanonicalName} openContext) throws Exception {
158158
${ctx.reuseOpenCode()}
159159
}
160160

@@ -230,7 +230,7 @@ object FunctionCodeGenerator {
230230
${ctx.reuseConstructorCode(funcName)}
231231

232232
@Override
233-
public void open(${className[OpenContext]} context) throws Exception {
233+
public void open(${className[OpenContext]} openContext) throws Exception {
234234
${ctx.reuseOpenCode()}
235235
}
236236

Diff for: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -381,7 +381,7 @@ object LookupJoinCodeGenerator {
381381
}
382382

383383
@Override
384-
public void open(${className[OpenContext]} context) throws Exception {
384+
public void open(${className[OpenContext]} openContext) throws Exception {
385385
${ctx.reuseOpenCode()}
386386
}
387387

@@ -492,7 +492,7 @@ object LookupJoinCodeGenerator {
492492
}
493493

494494
@Override
495-
public void open(${className[OpenContext]} context) throws Exception {
495+
public void open(${className[OpenContext]} openContext) throws Exception {
496496
${ctx.reuseOpenCode()}
497497
}
498498

Diff for: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -88,7 +88,7 @@ object WatermarkGeneratorCodeGenerator {
8888
}
8989

9090
@Override
91-
public void open(${classOf[OpenContext].getCanonicalName} context) throws Exception {
91+
public void open(${classOf[OpenContext].getCanonicalName} openContext) throws Exception {
9292
${ctx.reuseOpenCode()}
9393
}
9494

Diff for: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingFunctionGenUtil.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -630,7 +630,7 @@ object BridgingFunctionGenUtil {
630630
| ${ctx.reuseInitCode()}
631631
| }
632632
|
633-
| public void open(${className[OpenContext]} context) throws Exception {
633+
| public void open(${className[OpenContext]} openContext) throws Exception {
634634
| ${ctx.reuseOpenCode()}
635635
| }
636636
|

0 commit comments

Comments
 (0)