Skip to content

Commit 32e9710

Browse files
committed
fixup! aa
1 parent 305de16 commit 32e9710

File tree

5 files changed

+23
-14
lines changed

5 files changed

+23
-14
lines changed

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

+3-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
20+
import org.apache.flink.api.common.functions.{DefaultOpenContext, OpenContext}
2121
import org.apache.flink.configuration.Configuration
2222
import org.apache.flink.table.planner.codegen.CodeGenUtils._
2323
import org.apache.flink.table.planner.codegen.Indenter.toISC
@@ -75,7 +75,7 @@ object CollectorCodeGenerator {
7575
}
7676

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

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

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

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -230,7 +230,7 @@ object FunctionCodeGenerator {
230230
${ctx.reuseConstructorCode(funcName)}
231231

232232
@Override
233-
public void open(${className[Configuration]} parameters) throws Exception {
233+
public void open(${className[OpenContext]} context) 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

+3-5
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.{FlatMapFunction, Function}
20+
import org.apache.flink.api.common.functions.{FlatMapFunction, Function, OpenContext}
2121
import org.apache.flink.configuration.{Configuration, ReadableConfig}
2222
import org.apache.flink.streaming.api.functions.async.AsyncFunction
2323
import org.apache.flink.table.api.ValidationException
@@ -47,12 +47,10 @@ import org.apache.flink.table.types.logical.{LogicalType, RowType}
4747
import org.apache.flink.table.types.utils.DataTypeUtils.transform
4848
import org.apache.flink.types.Row
4949
import org.apache.flink.util.Collector
50-
5150
import org.apache.calcite.rel.`type`.RelDataType
5251
import org.apache.calcite.rex.RexNode
5352

5453
import java.util
55-
5654
import scala.collection.JavaConverters._
5755

5856
object LookupJoinCodeGenerator {
@@ -381,7 +379,7 @@ object LookupJoinCodeGenerator {
381379
}
382380

383381
@Override
384-
public void open(${className[Configuration]} parameters) throws Exception {
382+
public void open(${className[OpenContext]} context) throws Exception {
385383
${ctx.reuseOpenCode()}
386384
}
387385

@@ -492,7 +490,7 @@ object LookupJoinCodeGenerator {
492490
}
493491

494492
@Override
495-
public void open(${className[Configuration]} parameters) throws Exception {
493+
public void open(${className[OpenContext]} context) throws Exception {
496494
${ctx.reuseOpenCode()}
497495
}
498496

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

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

20-
import org.apache.flink.api.common.functions.{AbstractRichFunction, RichFunction}
20+
import org.apache.flink.api.common.functions.{AbstractRichFunction, OpenContext, RichFunction}
2121
import org.apache.flink.configuration.{Configuration, ReadableConfig}
2222
import org.apache.flink.table.api.{DataTypes, TableException}
2323
import org.apache.flink.table.api.Expressions.callSql
@@ -27,7 +27,7 @@ import org.apache.flink.table.expressions.ApiExpressionUtils.{typeLiteral, unres
2727
import org.apache.flink.table.expressions.Expression
2828
import org.apache.flink.table.functions._
2929
import org.apache.flink.table.functions.SpecializedFunction.{ExpressionEvaluator, ExpressionEvaluatorFactory}
30-
import org.apache.flink.table.functions.UserDefinedFunctionHelper.{validateClassForRuntime, ASYNC_SCALAR_EVAL, ASYNC_TABLE_EVAL, SCALAR_EVAL, TABLE_EVAL}
30+
import org.apache.flink.table.functions.UserDefinedFunctionHelper.{ASYNC_SCALAR_EVAL, ASYNC_TABLE_EVAL, SCALAR_EVAL, TABLE_EVAL, validateClassForRuntime}
3131
import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexFactory}
3232
import org.apache.flink.table.planner.codegen._
3333
import org.apache.flink.table.planner.codegen.CodeGenUtils._
@@ -46,11 +46,9 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoid
4646
import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isCompositeType
4747
import org.apache.flink.table.types.utils.DataTypeUtils.{isInternal, validateInputDataType, validateOutputDataType}
4848
import org.apache.flink.util.Preconditions
49-
5049
import AsyncCodeGenerator.DEFAULT_DELEGATING_FUTURE_TERM
5150

5251
import java.util.concurrent.CompletableFuture
53-
5452
import scala.collection.JavaConverters._
5553

5654
/**
@@ -630,7 +628,7 @@ object BridgingFunctionGenUtil {
630628
| ${ctx.reuseInitCode()}
631629
| }
632630
|
633-
| public void open(${className[Configuration]} parameters) throws Exception {
631+
| public void open(${className[OpenContext]} context) throws Exception {
634632
| ${ctx.reuseOpenCode()}
635633
| }
636634
|

Diff for: pom.xml

+13
Original file line numberDiff line numberDiff line change
@@ -2341,6 +2341,19 @@ under the License.
23412341
<!-- Mark these 2 methods to @Internal. Tracked under FLINK-34130, should be removed in 2.0 -->
23422342
<exclude>org.apache.flink.configuration.Configuration#getBytes(java.lang.String,byte[])</exclude>
23432343
<exclude>org.apache.flink.configuration.Configuration#setBytes(java.lang.String,byte[])</exclude>
2344+
<!-- FLIP-344: Remove parameter in RichFunction#open in 2.0 -->
2345+
<exclude>org.apache.flink.api.common.functions.AbstractRichFunction#open(org.apache.flink.configuration.Configuration)</exclude>
2346+
<exclude>org.apache.flink.api.common.functions.RichCoGroupFunction</exclude>
2347+
<exclude>org.apache.flink.api.common.functions.RichCrossFunction</exclude>
2348+
<exclude>org.apache.flink.api.common.functions.RichFilterFunction</exclude>
2349+
<exclude>org.apache.flink.api.common.functions.RichFlatJoinFunction</exclude>
2350+
<exclude>org.apache.flink.api.common.functions.RichFlatMapFunction</exclude>
2351+
<exclude>org.apache.flink.api.common.functions.RichGroupCombineFunction</exclude>
2352+
<exclude>org.apache.flink.api.common.functions.RichGroupReduceFunction</exclude>
2353+
<exclude>org.apache.flink.api.common.functions.RichJoinFunction</exclude>
2354+
<exclude>org.apache.flink.api.common.functions.RichMapFunction</exclude>
2355+
<exclude>org.apache.flink.api.common.functions.RichMapPartitionFunction</exclude>
2356+
<exclude>org.apache.flink.api.common.functions.RichReduceFunction</exclude>
23442357
<!-- FLINK-34085 Deprecated string config should be removed in 2.0 -->
23452358
<exclude>org.apache.flink.configuration.ConfigConstants</exclude>
23462359
<!-- FLINK-35886: WatermarksWithIdleness constructor was marked as deprecated -->

0 commit comments

Comments
 (0)