matriv commented on a change in pull request #18479:
URL: https://github.com/apache/flink/pull/18479#discussion_r795451478



##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.utils;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadatas;
+import org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeduplicate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDropUpdateBefore;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExpand;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIntervalJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMatch;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCalc;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCorrelate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonOverAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecRank;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSortLimit;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalJoin;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecValues;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowJoin;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank;
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/** Utility class for ExecNodeMetadata related functionality. */
+public final class ExecNodeMetadataUtil {
+
+    private ExecNodeMetadataUtil() {
+        // no instantiation
+    }
+
+    private static final Map<ExecNodeNameVersion, Class<? extends 
ExecNode<?>>> lookupMap =
+            new HashMap<>();
+
+    private static final Set<Class<? extends ExecNode<?>>> execNodes = new 
HashSet<>();
+
+    static {
+        execNodes.add(StreamExecCalc.class);
+        execNodes.add(StreamExecChangelogNormalize.class);
+        execNodes.add(StreamExecCorrelate.class);
+        execNodes.add(StreamExecDeduplicate.class);
+        execNodes.add(StreamExecDropUpdateBefore.class);
+        execNodes.add(StreamExecExchange.class);
+        execNodes.add(StreamExecExpand.class);
+        execNodes.add(StreamExecGlobalGroupAggregate.class);
+        execNodes.add(StreamExecGlobalWindowAggregate.class);
+        execNodes.add(StreamExecGroupAggregate.class);
+        execNodes.add(StreamExecGroupWindowAggregate.class);
+        execNodes.add(StreamExecIncrementalGroupAggregate.class);
+        execNodes.add(StreamExecIntervalJoin.class);
+        execNodes.add(StreamExecJoin.class);
+        execNodes.add(StreamExecLimit.class);
+        execNodes.add(StreamExecLocalGroupAggregate.class);
+        execNodes.add(StreamExecLocalWindowAggregate.class);
+        execNodes.add(StreamExecLookupJoin.class);
+        execNodes.add(StreamExecMatch.class);
+        execNodes.add(StreamExecMiniBatchAssigner.class);
+        execNodes.add(StreamExecOverAggregate.class);
+        execNodes.add(StreamExecPythonCalc.class);
+        execNodes.add(StreamExecPythonCorrelate.class);
+        execNodes.add(StreamExecPythonGroupAggregate.class);
+        execNodes.add(StreamExecPythonGroupWindowAggregate.class);
+        execNodes.add(StreamExecPythonOverAggregate.class);
+        execNodes.add(StreamExecRank.class);
+        execNodes.add(StreamExecSink.class);
+        execNodes.add(StreamExecSortLimit.class);
+        execNodes.add(StreamExecTableSourceScan.class);
+        execNodes.add(StreamExecTemporalJoin.class);
+        execNodes.add(StreamExecTemporalSort.class);
+        execNodes.add(StreamExecUnion.class);
+        execNodes.add(StreamExecValues.class);
+        execNodes.add(StreamExecWatermarkAssigner.class);
+        execNodes.add(StreamExecWindowAggregate.class);
+        execNodes.add(StreamExecWindowDeduplicate.class);
+        execNodes.add(StreamExecWindowJoin.class);
+        execNodes.add(StreamExecWindowRank.class);
+        execNodes.add(StreamExecWindowTableFunction.class);
+    }
+
+    static {
+        for (Class<? extends ExecNode<?>> execNodeClass : execNodes) {
+            addToLookupMap(execNodeClass);
+        }
+    }
+
+    public static Set<Class<? extends ExecNode<?>>> execNodes() {
+        return execNodes;
+    }
+
+    public static Class<? extends ExecNode<?>> retrieveExecNode(String name, 
int version) {
+        return lookupMap.get(new ExecNodeNameVersion(name, version));
+    }
+
+    @VisibleForTesting
+    static void addTestNode(Class<? extends ExecNode<?>> execNodeClass) {
+        addToLookupMap(execNodeClass);
+    }
+
+    @SuppressWarnings("rawtypes")
+    private static List<ExecNodeMetadata> extractMetadataFromAnnotation(
+            Class<? extends ExecNode> execNodeClass) {
+        List<ExecNodeMetadata> metadata = new ArrayList<>();
+        ExecNodeMetadata annotation = 
execNodeClass.getDeclaredAnnotation(ExecNodeMetadata.class);
+        if (annotation != null) {
+            metadata.add(annotation);
+        }
+
+        ExecNodeMetadatas annotations =

Review comment:
       With the "wrapper" `MultipleExecNodeMetadata` we can allow multiple 
individual `ExecNodeMetadata` annotations or a single 
`MultipleExecNodeMetadata`, where the `value` is an array of 
`ExecNodeMetadata`, but apparently java allows to have 2 annotations as well, 
one `MultipleExecNodeMetadata` and one `ExecNodeMetadata`, so we're checking 
this as not allowed.




-- 
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]


Reply via email to