slinkydeveloper commented on a change in pull request #18756: URL: https://github.com/apache/flink/pull/18756#discussion_r807919608
########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ConfigurationJsonDeserializer.java ########## @@ -0,0 +1,45 @@ +/* + * 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.planner.plan.nodes.exec; + +import org.apache.flink.configuration.Configuration; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import java.io.IOException; +import java.util.Map; + +/** Custom deserializer for {@link Configuration} used for {@link ExecNodeBase#persistedConfig}. */ +public class ConfigurationJsonDeserializer extends StdDeserializer<Configuration> { + + public ConfigurationJsonDeserializer() { + super(Configuration.class); + } + + @SuppressWarnings("unchecked") + @Override + public Configuration deserialize( + JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + return Configuration.fromMap( + (Map<String, String>) deserializationContext.readValue(jsonParser, Map.class)); Review comment: I think you should build the type manually here, otherwise this is gonna bypass checks like making sure the map contains only string values. Check options reading in `ResolvedCatalogTableJsonDeserializer` ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java ########## @@ -137,7 +161,8 @@ public void replaceInputEdge(int index, ExecEdge newInputEdge) { } /** Internal method, translates this node into a Flink operator. */ - protected abstract Transformation<T> translateToPlanInternal(PlannerBase planner); + protected abstract Transformation<T> translateToPlanInternal( Review comment: Please add a comment expliciting that now who implements this needs to use this config rather than the planner config ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ConfigurationJsonSerializer.java ########## @@ -0,0 +1,67 @@ +/* + * 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.planner.plan.nodes.exec; + +import org.apache.flink.configuration.Configuration; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; +import java.util.TreeMap; + +/** + * Custom serializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig()}. + */ +public class ConfigurationJsonSerializer extends StdSerializer<Configuration> { Review comment: Here and in other places, check you have the `@Internal` annotation ########## File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfiguration.java ########## @@ -0,0 +1,121 @@ +/* + * 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.planner.plan.nodes.exec; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.planner.delegation.PlannerConfiguration; + +import java.time.ZoneId; +import java.util.Optional; + +import static java.time.ZoneId.SHORT_IDS; + +/** + * Configuration view that combines the {@link PlannerConfiguration} with the {@link + * ExecNodeBase#getPersistedConfig()} configuration. The persisted configuration of the {@link + * ExecNode} which is deserialized from the JSON plan has precedence over the {@link + * PlannerConfiguration}. + */ +@Internal +public class ExecNodeConfiguration implements ReadableConfig { Review comment: Perhaps can you extend from `PlannerConfiguration` directly to reuse the merging logic? TBH I would move `PlannerConfiguration` in `org.apache.flink.configuration` and rename it to something more generic, as it's essentially doing a merge of the configs, choosing one as precedence -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
