|
17 | 17 | */
|
18 | 18 | package org.apache.flink.table.planner.codegen
|
19 | 19 |
|
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} |
21 | 21 | import org.apache.flink.configuration.{Configuration, PipelineOptions, ReadableConfig}
|
22 | 22 | import org.apache.flink.table.api.{TableConfig, TableException}
|
23 | 23 | import org.apache.flink.table.data.{DecimalData, GenericRowData, TimestampData}
|
@@ -102,9 +102,12 @@ class ExpressionReducer(
|
102 | 102 | throw new TableException("RichMapFunction[GenericRowData, GenericRowData] required here")
|
103 | 103 | }
|
104 | 104 |
|
| 105 | + val parameters = toScala(tableConfig.getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS)) |
| 106 | + .map(Configuration.fromMap) |
| 107 | + .getOrElse(new Configuration) |
105 | 108 | val reduced =
|
106 | 109 | try {
|
107 |
| - richMapFunction.open(DefaultOpenContext.INSTANCE) |
| 110 | + richMapFunction.open(new WithConfigurationOpenContext(parameters)) |
108 | 111 | // execute
|
109 | 112 | richMapFunction.map(EMPTY_ROW)
|
110 | 113 | } catch {
|
@@ -312,7 +315,7 @@ class ConstantCodeGeneratorContext(tableConfig: ReadableConfig, classLoader: Cla
|
312 | 315 | super.addReusableFunction(
|
313 | 316 | function,
|
314 | 317 | classOf[FunctionContext],
|
315 |
| - Seq("null", "this.getClass().getClassLoader()", "context")) |
| 318 | + Seq("null", "this.getClass().getClassLoader()", "openContext")) |
316 | 319 | }
|
317 | 320 |
|
318 | 321 | override def addReusableConverter(dataType: DataType, classLoaderTerm: String = null): String = {
|
|
0 commit comments