You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2021/09/29 18:19:29 UTC

[GitHub] [calcite] jcamachor commented on a change in pull request #2538: [CALCITE-4546]  Changing Rel Metadata Dispatch(James Starr)

jcamachor commented on a change in pull request #2538:
URL: https://github.com/apache/calcite/pull/2538#discussion_r718554108



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/DelegatingMetadataRel.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.calcite.rel.metadata;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * Interface for {@link RelNode} where the metadata is derived from another node.
+ */
+public interface DelegatingMetadataRel {

Review comment:
       We introduce `DelegatingMetadataRel` but we only use it for `HepPlanner` nodes (`HepRelVertex`). Why is this not used for Volcano nodes too, e.g., `RelSubset`? What is the intuition to use it only for some of the planners?
   
   In addition, is this change strictly necessary for this patch? I am wondering whether it would make sense to include it in a follow-up if it needs to be extended to work with Volcano too.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/DelegatingMetadataRel.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.calcite.rel.metadata;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ * Interface for {@link RelNode} where the metadata is derived from another node.
+ */
+public interface DelegatingMetadataRel {
+  RelNode getCurrentRel();

Review comment:
       I assume the reason to use this method name is that it is already present in `HepRelVertex`. Is `getDeletegateRel` a better choice for the method name though (since this will be a public API and other nodes may implement this interface)?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
##########
@@ -107,45 +73,10 @@
           CalciteSystemProperty.METADATA_HANDLER_CACHE_MAXIMUM_SIZE.value())
           .build(
               CacheLoader.from(key ->
-                  load3(key.def, key.provider.handlers(key.def),
-                      key.relClasses)));
+                  load3(key.def, key.provider.handlers(key.def))));
 
   // Pre-register the most common relational operators, to reduce the number of

Review comment:
       Probably this comment should go away too.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
##########
@@ -238,139 +167,15 @@ public static JaninoRelMetadataProvider of(RelMetadataProvider provider) {
         .append(def.metadataClass.getName())
         .append(".DEF;\n")
         .append("  }\n");
-    for (Ord<Method> method : Ord.zip(def.methods)) {
-      buff.append("  public ")
-          .append(method.e.getReturnType().getName())
-          .append(" ")
-          .append(method.e.getName())
-          .append("(\n")
-          .append("      ")
-          .append(RelNode.class.getName())
-          .append(" r,\n")
-          .append("      ")
-          .append(RelMetadataQuery.class.getName())
-          .append(" mq");
-      paramList(buff, method.e)
-          .append(") {\n");
-      buff.append("    final java.util.List key = ")
-          .append(
-              (method.e.getParameterTypes().length < 4
-              ? org.apache.calcite.runtime.FlatLists.class
-              : ImmutableList.class).getName())
-          .append(".of(")
-          .append(def.metadataClass.getName());
-      if (method.i == 0) {
-        buff.append(".DEF");
-      } else {
-        buff.append(".DEF.methods.get(")
-            .append(method.i)
-            .append(")");
-      }
-      safeArgList(buff, method.e)
-          .append(");\n")
-          .append("    final Object v = mq.map.get(r, key);\n")
-          .append("    if (v != null) {\n")
-          .append("      if (v == ")
-          .append(NullSentinel.class.getName())
-          .append(".ACTIVE) {\n")
-          .append("        throw new ")
-          .append(CyclicMetadataException.class.getName())
-          .append("();\n")
-          .append("      }\n")
-          .append("      if (v == ")
-          .append(NullSentinel.class.getName())
-          .append(".INSTANCE) {\n")
-          .append("        return null;\n")
-          .append("      }\n")
-          .append("      return (")
-          .append(method.e.getReturnType().getName())
-          .append(") v;\n")
-          .append("    }\n")
-          .append("    mq.map.put(r, key,")
-          .append(NullSentinel.class.getName())
-          .append(".ACTIVE);\n")
-          .append("    try {\n")
-          .append("      final ")
-          .append(method.e.getReturnType().getName())
-          .append(" x = ")
-          .append(method.e.getName())
-          .append("_(r, mq");
-      argList(buff, method.e)
-          .append(");\n")
-          .append("      mq.map.put(r, key, ")
-          .append(NullSentinel.class.getName())
-          .append(".mask(x));\n")
-          .append("      return x;\n")
-          .append("    } catch (")
-          .append(Exception.class.getName())
-          .append(" e) {\n")
-          .append("      mq.map.row(r).clear();\n")
-          .append("      throw e;\n")
-          .append("    }\n")
-          .append("  }\n")
-          .append("\n")
-          .append("  private ")
-          .append(method.e.getReturnType().getName())
-          .append(" ")
-          .append(method.e.getName())
-          .append("_(\n")
-          .append("      ")
-          .append(RelNode.class.getName())
-          .append(" r,\n")
-          .append("      ")
-          .append(RelMetadataQuery.class.getName())
-          .append(" mq");
-      paramList(buff, method.e)
-          .append(") {\n");
-      buff.append("    switch (relClasses.indexOf(r.getClass())) {\n");
-
-      // Build a list of clauses, grouping clauses that have the same action.
-      final Multimap<String, Integer> clauses = LinkedHashMultimap.create();
-      final StringBuilder buf2 = new StringBuilder();
-      for (Ord<Class<? extends RelNode>> relClass : Ord.zip(relClasses)) {
-        if (relClass.e == HepRelVertex.class) {
-          buf2.append("      return ")
-              .append(method.e.getName())
-              .append("(((")
-              .append(relClass.e.getName())
-              .append(") r).getCurrentRel(), mq");
-          argList(buf2, method.e)
-              .append(");\n");
-        } else {
-          final Method handler = space.find(relClass.e, method.e);
-          final String v = findProvider(providerList, handler.getDeclaringClass());
-          buf2.append("      return ")
-              .append(v)
-              .append(".")
-              .append(method.e.getName())
-              .append("((")
-              .append(handler.getParameterTypes()[0].getName())
-              .append(") r, mq");
-          argList(buf2, method.e)
-              .append(");\n");
-        }
-        clauses.put(buf2.toString(), relClass.i);
-        buf2.setLength(0);
-      }
-      buf2.append("      throw new ")
-          .append(NoHandler.class.getName())
-          .append("(r.getClass());\n")
-          .append("    }\n")
-          .append("  }\n");
-      clauses.put(buf2.toString(), -1);
-      for (Map.Entry<String, Collection<Integer>> pair : clauses.asMap().entrySet()) {
-        if (pair.getValue().contains(relClasses.indexOf(RelNode.class))) {
-          buff.append("    default:\n");
-        } else {
-          for (Integer integer : pair.getValue()) {
-            buff.append("    case ").append(integer).append(":\n");
-          }
-        }
-        buff.append(pair.getKey());
-      }
+    DispatchGenerator dispatchGenerator = new DispatchGenerator(handlerToName);
+    for (Ord<Method> method : Ord.zip(map.keySet())) {
+      generateCachedMethod(buff, method.e, method.i);
+      dispatchGenerator.dispatchMethod(buff, method.e, map.get(method.e));
     }
-    final List<Object> argList = new ArrayList<>(Pair.right(providerList));
-    argList.add(0, ImmutableList.copyOf(relClasses));
+    //buff.append("}");

Review comment:
       Commented out line can be removed.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
##########
@@ -193,41 +124,39 @@ public static JaninoRelMetadataProvider of(RelMetadataProvider provider) {
   }
 
   private static <M extends Metadata> MetadataHandler<M> load3(
-      MetadataDef<M> def, Multimap<Method, MetadataHandler<M>> map,
-      ImmutableList<Class<? extends RelNode>> relClasses) {
+      MetadataDef<M> def, Multimap<Method, ? extends MetadataHandler<?>> map) {
     final StringBuilder buff = new StringBuilder();
     final String name =
-        "GeneratedMetadataHandler_" + def.metadataClass.getSimpleName();
-    final Set<MetadataHandler> providerSet = new HashSet<>();
-    final List<Pair<String, MetadataHandler>> providerList = new ArrayList<>();
-    //noinspection unchecked
-    final ReflectiveRelMetadataProvider.Space space =
-        new ReflectiveRelMetadataProvider.Space((Multimap) map);
-    for (MetadataHandler provider : space.providerMap.values()) {
+        "GeneratedMetadata_" + simpleNameForHandler(def.handlerClass);
+    final Set<MetadataHandler<?>> providerSet = new HashSet<>();
+
+    final Map<MetadataHandler<?>, String> handlerToName = new LinkedHashMap<>();
+    for (MetadataHandler<?> provider : map.values()) {
       if (providerSet.add(provider)) {

Review comment:
       Do you need both the `providerSet` and `handlerToName`? Couldn't we get rid of the `providerSet` and do a `contains` check against the map instead?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
##########
@@ -475,33 +356,15 @@ private static StringBuilder paramList(StringBuilder buff, Method method) {
 
   synchronized <M extends Metadata, H extends MetadataHandler<M>> H revise(
       Class<? extends RelNode> rClass, MetadataDef<M> def) {
-    if (ALL_RELS.add(rClass)) {
-      HANDLERS.invalidateAll();
-    }
     //noinspection unchecked
     return (H) create(def);
   }
 
   /** Registers some classes. Does not flush the providers, but next time we
    * need to generate a provider, it will handle all of these classes. So,
    * calling this method reduces the number of times we need to re-generate. */
+  @Deprecated
   public void register(Iterable<Class<? extends RelNode>> classes) {
-    // Register the classes and their base classes up to RelNode. Don't bother

Review comment:
       Is there any loss of functionality difference with this patch (difficult to infer from the dynamically generated code)? Are RelNode classes that are defined by engines relying on Calcite, e.g., Hive, handled as before? Will this require code changes in those systems?




-- 
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: commits-unsubscribe@calcite.apache.org

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