fsk119 commented on code in PR #26641:
URL: https://github.com/apache/flink/pull/26641#discussion_r2135260989


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ContextResolvedModelJsonDeserializer.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.serde;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.config.TableConfigOptions;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ContextResolvedModel;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ResolvedCatalogModel;
+import org.apache.flink.table.catalog.ResolvedSchema;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
+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 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS;
+import static 
org.apache.flink.table.api.config.TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.CompiledPlanSerdeUtil.deserializeOptionalField;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.CompiledPlanSerdeUtil.traverse;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.ContextResolvedModelJsonSerializer.FIELD_NAME_CATALOG_MODEL;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.ContextResolvedModelJsonSerializer.FIELD_NAME_IDENTIFIER;
+import static 
org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogModelJsonSerializer.OPTIONS;
+
+/**
+ * JSON deserializer for {@link ContextResolvedModel}.
+ *
+ * @see ContextResolvedModelJsonSerializer for the reverse operation
+ */
+@Internal
+public class ContextResolvedModelJsonDeserializer extends 
StdDeserializer<ContextResolvedModel> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final JsonPointer optionsPointer =
+            JsonPointer.compile("/" + FIELD_NAME_CATALOG_MODEL + "/" + 
OPTIONS);
+
+    public ContextResolvedModelJsonDeserializer() {
+        super(ContextResolvedModel.class);
+    }
+
+    @Override
+    public ContextResolvedModel deserialize(JsonParser jsonParser, 
DeserializationContext ctx)
+            throws IOException {
+        final TableConfigOptions.CatalogPlanRestore planRestoreOption =
+                
SerdeContext.get(ctx).getConfiguration().get(PLAN_RESTORE_CATALOG_OBJECTS);
+        final CatalogManager catalogManager =
+                SerdeContext.get(ctx).getFlinkContext().getCatalogManager();
+        final ObjectNode objectNode = jsonParser.readValueAsTree();
+
+        final ObjectIdentifier identifier =
+                ctx.readValue(
+                        traverse(objectNode.required(FIELD_NAME_IDENTIFIER), 
jsonParser.getCodec()),
+                        ObjectIdentifier.class);
+        final @Nullable ResolvedCatalogModel restoredModel =
+                deserializeOptionalField(
+                                objectNode,
+                                FIELD_NAME_CATALOG_MODEL,
+                                ResolvedCatalogModel.class,
+                                jsonParser.getCodec(),
+                                ctx)
+                        .orElse(null);
+
+        final Optional<ContextResolvedModel> contextResolvedModelFromCatalog =
+                catalogManager.getModel(identifier);
+
+        // If plan has no catalog model field or no options field,
+        // the model is permanent in the catalog and the option is plan all 
enforced, then fail
+        if ((objectNode.at(optionsPointer).isMissingNode()
+                && isPlanEnforced(planRestoreOption)
+                && contextResolvedModelFromCatalog
+                        .map(ContextResolvedModel::isPermanent)
+                        .orElse(false))) {
+            throw lookupDisabled(identifier);
+        }
+
+        // If we have a schema from the plan and from the catalog, we need to 
check they match.
+        if (restoredModel != null && 
contextResolvedModelFromCatalog.isPresent()) {
+            ContextResolvedModel modelFromCatalog = 
contextResolvedModelFromCatalog.get();
+            if (!areResolvedSchemasEqual(
+                    restoredModel.getResolvedInputSchema(),
+                    
modelFromCatalog.getResolvedModel().getResolvedInputSchema())) {
+                throw schemaNotMatching(
+                        identifier,
+                        "input schema",
+                        restoredModel.getResolvedInputSchema(),
+                        
modelFromCatalog.getResolvedModel().getResolvedInputSchema());
+            }
+            if (!areResolvedSchemasEqual(
+                    restoredModel.getResolvedOutputSchema(),
+                    
modelFromCatalog.getResolvedModel().getResolvedOutputSchema())) {
+                throw schemaNotMatching(
+                        identifier,
+                        "output schema",
+                        restoredModel.getResolvedInputSchema(),
+                        
modelFromCatalog.getResolvedModel().getResolvedInputSchema());
+            }
+        }
+
+        if (restoredModel == null || isLookupForced(planRestoreOption)) {
+            return contextResolvedModelFromCatalog.orElseThrow(
+                    () -> missingModelFromCatalog(identifier, 
isLookupForced(planRestoreOption)));
+        }
+
+        if (contextResolvedModelFromCatalog.isPresent()) {
+            // If no config map is present, then the ContextResolvedTable was 
serialized with
+            // SCHEMA, so we just need to return the catalog query result
+            if (objectNode.at(optionsPointer).isMissingNode()) {
+                return contextResolvedModelFromCatalog.get();
+            }
+
+            return contextResolvedModelFromCatalog
+                    .flatMap(ContextResolvedModel::getCatalog)
+                    .map(c -> ContextResolvedModel.permanent(identifier, c, 
restoredModel))
+                    .orElseGet(() -> 
ContextResolvedModel.temporary(identifier, restoredModel));
+        }
+
+        return ContextResolvedModel.temporary(identifier, restoredModel);
+    }
+
+    private boolean isPlanEnforced(TableConfigOptions.CatalogPlanRestore 
planRestoreOption) {
+        return planRestoreOption == 
TableConfigOptions.CatalogPlanRestore.ALL_ENFORCED;
+    }
+
+    private boolean isLookupForced(TableConfigOptions.CatalogPlanRestore 
planRestoreOption) {
+        return planRestoreOption == 
TableConfigOptions.CatalogPlanRestore.IDENTIFIER;
+    }
+
+    private boolean areResolvedSchemasEqual(
+            ResolvedSchema schemaFromPlan, ResolvedSchema schemaFromCatalog) {
+        // For schema equality we check:
+        //  * Columns size and order
+        //  * For each column: name, kind (class) and type
+        final List<Column> columnsFromPlan = schemaFromPlan.getColumns();
+        final List<Column> columnsFromCatalog = schemaFromCatalog.getColumns();
+
+        if (columnsFromPlan.size() != columnsFromCatalog.size()) {
+            return false;
+        }
+
+        for (int i = 0; i < columnsFromPlan.size(); i++) {
+            final Column columnFromPlan = columnsFromPlan.get(i);
+            final Column columnFromCatalog = columnsFromCatalog.get(i);
+            if (!Objects.equals(columnFromPlan.getName(), 
columnFromCatalog.getName())
+                    || !Objects.equals(columnFromPlan.getClass(), 
columnFromCatalog.getClass())
+                    || !Objects.equals(
+                            columnFromPlan.getDataType(), 
columnFromCatalog.getDataType())) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    static TableException schemaNotMatching(

Review Comment:
   The exception messages are different. One is for table, the other is for 
model.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to