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/03/23 03:48:10 UTC

[GitHub] [calcite] jamesstarr opened a new pull request #2378: Metadata Fly weight

jamesstarr opened a new pull request #2378:
URL: https://github.com/apache/calcite/pull/2378


   WIP do not review


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

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



[GitHub] [calcite] jacques-n commented on pull request #2378: [CALCITE-4551] Reusing Immutable metadata cache keys

Posted by GitBox <gi...@apache.org>.
jacques-n commented on pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#issuecomment-925287107


   @jamesstarr, can you rebase this on master?


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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742179082



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the two solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.




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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742185479



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       To put this another way: RelMetadataQuery is the what, RelMetadataProvider is one possible how. Let's not couple the two.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r738810938



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/janino/CacheUtil.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.janino;
+
+import java.util.stream.IntStream;
+
+/**
+ * Functions used by generated code.
+ */
+public class CacheUtil {

Review comment:
       This has been marked internal on the merged code.




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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742179082



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r738812680



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       If you do not want a Reviseable Handler, you can simply return the final handler from the initialHandler.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742494848



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -62,49 +62,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, Object, Object> map = HashBasedTable.create();
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @API(status = API.Status.INTERNAL)
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;
+
+  @Deprecated // to be removed before 2.0
+  public final @Nullable JaninoRelMetadataProvider metadataProvider = THREAD_PROVIDERS.get();
 
   //~ Static fields/initializers ---------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public static final ThreadLocal<@Nullable JaninoRelMetadataProvider> THREAD_PROVIDERS =
       new ThreadLocal<>();
 
-  //~ Constructors -----------------------------------------------------------
-

Review comment:
       I am doubtful any one inherited directly from RelMetadataQueryBase.




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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742179082



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the two solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       To put this another way: RelMetadataQuery is the what, RelMetadataProvider is one possible how. Let's not couple the two.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {
+
+  <MH extends MetadataHandler<?>> MH initialHandler(Class<MH> handlerClass);
+
+  /** Re-generates the handler for a given kind of metadata.. */
+  <MH extends MetadataHandler<?>> MH revise(Class<MH> handlerClass);
+
+  /**
+   * Creates a new cache.
+   *
+   * @return A new cache for {@link RelMetadataQuery}
+   */
+  MetadataCache buildCache();

Review comment:
       I don't really understand the meaning of this method. Why does someone call this? What happens if I call it multiple times? What is this caching? Does it have to be called? Why is it on this interface? Also, if you wanted to customize the cache for things like a double property, wouldn't you potentially want to implement a cache that avoided the constant creation of objects and boxing (which the interface of MetadataCache forces)?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -62,49 +62,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, Object, Object> map = HashBasedTable.create();
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @API(status = API.Status.INTERNAL)
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;
+
+  @Deprecated // to be removed before 2.0
+  public final @Nullable JaninoRelMetadataProvider metadataProvider = THREAD_PROVIDERS.get();
 
   //~ Static fields/initializers ---------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public static final ThreadLocal<@Nullable JaninoRelMetadataProvider> THREAD_PROVIDERS =
       new ThreadLocal<>();
 
-  //~ Constructors -----------------------------------------------------------
-

Review comment:
       Isn't removing this constructor a breaking change?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +109,70 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
+  @Deprecated
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {
+    super(metadataHandlerProvider);
+    this.collationHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Collation.Handler.class);
+    this.columnOriginHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
+    this.expressionLineageHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
+    this.tableReferencesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.TableReferences.Handler.class);
+    this.columnUniquenessHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
+    this.cumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
+    this.distinctRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
+    this.distributionHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Distribution.Handler.class);
+    this.explainVisibilityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
+    this.maxRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
+    this.minRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
+    this.memoryHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Memory.Handler.class);
+    this.nonCumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
+    this.parallelismHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Parallelism.Handler.class);
     this.percentageOriginalRowsHandler =
-        initialHandler(BuiltInMetadata.PercentageOriginalRows.Handler.class);
-    this.populationSizeHandler = initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
-    this.predicatesHandler = initialHandler(BuiltInMetadata.Predicates.Handler.class);
-    this.allPredicatesHandler = initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
-    this.nodeTypesHandler = initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
-    this.rowCountHandler = initialHandler(BuiltInMetadata.RowCount.Handler.class);
-    this.selectivityHandler = initialHandler(BuiltInMetadata.Selectivity.Handler.class);
-    this.sizeHandler = initialHandler(BuiltInMetadata.Size.Handler.class);
-    this.uniqueKeysHandler = initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
-    this.lowerBoundCostHandler = initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
-  }
-
-  private RelMetadataQuery(JaninoRelMetadataProvider metadataProvider,
-      RelMetadataQuery prototype) {
-    super(requireNonNull(metadataProvider, "metadataProvider"));
+        metadataHandlerProvider.initialHandler(
+            BuiltInMetadata.PercentageOriginalRows.Handler.class);
+    this.populationSizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
+    this.predicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Predicates.Handler.class);
+    this.allPredicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
+    this.nodeTypesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
+    this.rowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.RowCount.Handler.class);
+    this.selectivityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Selectivity.Handler.class);
+    this.sizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Size.Handler.class);
+    this.uniqueKeysHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
+    this.lowerBoundCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
+  }
+
+  protected RelMetadataQuery(RelMetadataQuery prototype) {

Review comment:
       please make this private.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       Config feels like it would be weird here. As I have thought more about the name I'm pretty much okay with it as is.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I think I have a simple solution to avoiding the introduction of a new constructor on this class:
   - Make this constructor private
   - Add a new inner static subclass that uses that constructor and exposes a protected class.
   
   Then we tell people that want to use the handler behavior to extend that subclass. That way, we have deprecated subclassing RelMetadataQuery directly while moving towards this new pattern with minimal code changes.
   
   This avoids the introduction of a new coupling api between RelMetadataQuery the handler concept, is minimal changes in your patch and sets us on a path to moving RelMetadataQuery to an interface or dumb abstract class.

##########
File path: core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
##########
@@ -136,7 +134,7 @@ public RexBuilder getRexBuilder() {
     return rexBuilder;
   }
 
-  public @Nullable RelMetadataProvider getMetadataProvider() {
+  public RelMetadataProvider getMetadataProvider() {

Review comment:
       I see this marked resolved but don't see it being deprecated.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
+
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;

Review comment:
       Please make metadataHandlerProvider private. I think this means that for now you need to move it to relmetadataquery and pass a cache supplier into the RelMetadataQueryBase
   
   Please make the cache private. I think using the Api pattern to "hide" things should be used as a last resort. This may mean it should also move to RelMetadataQuery (for now).

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   -This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect `RelMetdataQuery.builder()` to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the two solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       To put this another way: RelMetadataQuery is the what, RelMetadataProvider is one possible how. Let's not couple the two.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {
+
+  <MH extends MetadataHandler<?>> MH initialHandler(Class<MH> handlerClass);
+
+  /** Re-generates the handler for a given kind of metadata.. */
+  <MH extends MetadataHandler<?>> MH revise(Class<MH> handlerClass);
+
+  /**
+   * Creates a new cache.
+   *
+   * @return A new cache for {@link RelMetadataQuery}
+   */
+  MetadataCache buildCache();

Review comment:
       I don't really understand the meaning of this method. Why does someone call this? What happens if I call it multiple times? What is this caching? Does it have to be called? Why is it on this interface? Also, if you wanted to customize the cache for things like a double property, wouldn't you potentially want to implement a cache that avoided the constant creation of objects and boxing (which the interface of MetadataCache forces)?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -62,49 +62,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, Object, Object> map = HashBasedTable.create();
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @API(status = API.Status.INTERNAL)
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;
+
+  @Deprecated // to be removed before 2.0
+  public final @Nullable JaninoRelMetadataProvider metadataProvider = THREAD_PROVIDERS.get();
 
   //~ Static fields/initializers ---------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public static final ThreadLocal<@Nullable JaninoRelMetadataProvider> THREAD_PROVIDERS =
       new ThreadLocal<>();
 
-  //~ Constructors -----------------------------------------------------------
-

Review comment:
       Isn't removing this constructor a breaking change?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +109,70 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
+  @Deprecated
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {
+    super(metadataHandlerProvider);
+    this.collationHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Collation.Handler.class);
+    this.columnOriginHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
+    this.expressionLineageHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
+    this.tableReferencesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.TableReferences.Handler.class);
+    this.columnUniquenessHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
+    this.cumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
+    this.distinctRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
+    this.distributionHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Distribution.Handler.class);
+    this.explainVisibilityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
+    this.maxRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
+    this.minRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
+    this.memoryHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Memory.Handler.class);
+    this.nonCumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
+    this.parallelismHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Parallelism.Handler.class);
     this.percentageOriginalRowsHandler =
-        initialHandler(BuiltInMetadata.PercentageOriginalRows.Handler.class);
-    this.populationSizeHandler = initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
-    this.predicatesHandler = initialHandler(BuiltInMetadata.Predicates.Handler.class);
-    this.allPredicatesHandler = initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
-    this.nodeTypesHandler = initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
-    this.rowCountHandler = initialHandler(BuiltInMetadata.RowCount.Handler.class);
-    this.selectivityHandler = initialHandler(BuiltInMetadata.Selectivity.Handler.class);
-    this.sizeHandler = initialHandler(BuiltInMetadata.Size.Handler.class);
-    this.uniqueKeysHandler = initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
-    this.lowerBoundCostHandler = initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
-  }
-
-  private RelMetadataQuery(JaninoRelMetadataProvider metadataProvider,
-      RelMetadataQuery prototype) {
-    super(requireNonNull(metadataProvider, "metadataProvider"));
+        metadataHandlerProvider.initialHandler(
+            BuiltInMetadata.PercentageOriginalRows.Handler.class);
+    this.populationSizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
+    this.predicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Predicates.Handler.class);
+    this.allPredicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
+    this.nodeTypesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
+    this.rowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.RowCount.Handler.class);
+    this.selectivityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Selectivity.Handler.class);
+    this.sizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Size.Handler.class);
+    this.uniqueKeysHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
+    this.lowerBoundCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
+  }
+
+  protected RelMetadataQuery(RelMetadataQuery prototype) {

Review comment:
       please make this private.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       Config feels like it would be weird here. As I have thought more about the name I'm pretty much okay with it as is.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I think I have a simple solution to avoiding the introduction of a new constructor on this class:
   - Make this constructor private
   - Add a new inner static subclass that uses that constructor and exposes a protected class.
   
   Then we tell people that want to use the handler behavior to extend that subclass. That way, we have deprecated subclassing RelMetadataQuery directly while moving towards this new pattern with minimal code changes.
   
   This avoids the introduction of a new coupling api between RelMetadataQuery the handler concept, is minimal changes in your patch and sets us on a path to moving RelMetadataQuery to an interface or dumb abstract class.

##########
File path: core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
##########
@@ -136,7 +134,7 @@ public RexBuilder getRexBuilder() {
     return rexBuilder;
   }
 
-  public @Nullable RelMetadataProvider getMetadataProvider() {
+  public RelMetadataProvider getMetadataProvider() {

Review comment:
       I see this marked resolved but don't see it being deprecated.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
+
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;

Review comment:
       Please make metadataHandlerProvider private. I think this means that for now you need to move it to relmetadataquery and pass a cache supplier into the RelMetadataQueryBase
   
   Please make the cache private. I think using the Api pattern to "hide" things should be used as a last resort. This may mean it should also move to RelMetadataQuery (for now).

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   -This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect `RelMetdataQuery.builder()` to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r738807901



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/janino/CacheGenerator.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.janino;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.CyclicMetadataException;
+import org.apache.calcite.rel.metadata.DelegatingMetadataRel;
+import org.apache.calcite.rel.metadata.NullSentinel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.FlatLists;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Method;
+
+import static org.apache.calcite.rel.metadata.janino.CodeGeneratorUtil.argList;
+import static org.apache.calcite.rel.metadata.janino.CodeGeneratorUtil.paramList;
+
+/**
+ * Generates caching code for janino backed metadata.
+ */
+class CacheGenerator {

Review comment:
       Caching part has already been split into its own PR and merged.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r739618684



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       At this point the MetadataProvider a list a of tuples (Class<? extends MetadataHandler>, MetadataHandler instance) which represent binding a set methods to a particular metadata.  Without this patch the Metadata class is not used in any meaningful way in code path that is not already deprecated.  You have to have a symbol(global constant) to bind a the metadata implementation too.
   
   Part of the whole point of the interface is to let people customize/change the behavior of the handler generation.  Some people want to eagerly generate the code at runtime, other want to tweaks the caching of generated classes, you and others do not want to use janino altogether.  This would support all of those use cases.
   
   RelMetadataQuery needs to call the implementation of the metadata calls which is currently done through handlers.  RelMetadataQueryBase already has a supported api for doing conceptually similar things with similar amount of exposed api.  Their are just a set of things that you can not do with that existing and supported API that I am attempting to rectify.
   
   Several consumer of apache calcite have custom metadata calls.  Currently the suggested way to create a custom call is to subclass RelMetadataQuery.  What you are purposing would require 2 different subclasses of RelMetadataQuery and also require them to correctly maintain the caching/cyclic dependency boiler plate.  I understand the desire to be able to debug code  and have a separation of concerns but the API you are proposing would require several subclasses to do something conceptually simple and introduce breaking changes.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742284367



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       There are several requirements metadata in calcite:
   Metadata retrieval - the api for getting a metdata value for a relnode and give set of arguments.  This is currently the RelMetadataQuery.
   Metadata implementation - The code that is called to for a relnode to get a metadata value.  This is currently sub-classes of RelMetdataProvider.
   Defaulting and Validation - There is defaulting, canonization, validation, etc after a metadata value is retrieved in RelMetadataQuery.
   Dispatch - How the implementation is called for given arguments.  This is currently done in Janino compiled code.
   Cycle Detection - This is currently a **hard** requirement of volcano planner.  This is currently done in Janino compiled code.
   Caching Invalidation - This is required for volcano planner.  This is currently done through RelMetadataQuery.clearCache.
   Caching Implementation - This is coupled with cycle detection, but is currently accomplished through RelMetadataQuery.map.
   Caching Values - The caches need to be checked and updated.  This is currently done in the generated code.
   
   Metadata retrieval and Metadata implementation are hard requirements for external facing APIs.  They are currently exist as external facing APIs.  Many downstream projects depend on the ability to either customize existing metadata types or add their own metadata types.  When a down stream project adds their own metadata type, they need to be able to expose it to the larger system aka through metadata retrieval.  This is currently done by subclassing RelMetadataQuery and calling methods RelMetadataQueryBase.  An example of this can be seen in RelMetadataTest.MyRelMetadataQuery.
   
   So the metadata retrieval api is the only api need to for the rest of calcite core, however, downstream users of calcite need to be able to configure and extend the metadata implementation.  So some of RelMetadataQuery implementation must be exposed with the current setup.
   
   I support making RelMetadataQuery an interface so the metadata retrieval is clearly defined.  However, this is not the only public API necessary to expose.  Also, it is breaking change which does not separate the defaulting and validation from the RelMetadataQueryImpl.  If RelMetadataQuery was split into RelMetadataQuery(an interface), DefaultingAndValidatingRelMetadataQuery and HandlerBackedRelMetadataQuery, then down stream projects would have to implement 2 different RelMetadataQueries to add a custom metadata.  This api seems a bit convoluted and cumbersome.  Alternatively, breaking it up into class hierarchy of RelMetadataQuery -> DefaultingAndValidatingRelMetadataQuery ->  HandlerBackedRelMetadataQuery also feels a bit convoluted and is a breaking change.
   
   I also support removing the leaky abstraction of the metadata implementation to the rest of calcite core.  But changing this does not actually help things much other than remove a thread local that can cause odd behaviors in uncommon nesting scenarios. 




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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742179082



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the two solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'd like to separate the concern and the solution into two separate discussions. 
   
   First the concern:
   The problem is the coupling between RelMetdataQuery and RelMetdataProvider. In this patch you add additional new public apis that reinforce this coupling. There is no need for the coupling. It is completely reasonable that someone should be able to implement RelMetadataQuery without having to work at all with RelMetdataProvider related items. That is the beauty of the RelMetadataQuery surface area from the planning/rule pov. For example, someone should be able hand write a RelMetadatQuery implementation that has all the handling of supported rel types.
   
   Second, the solution:
   It is important to remove the coupling of RelMetdataQuery and RelMetdataProvider. Ideally, we would convert RelMetadataQuery to an abstract implementation (or interface) that only does a minimal number of data cleansing/canonicalization options (null to sentinels, conveniences interfaces, etc). However, because of the exposure of the protected constructor for RelMetadatQuery, this has to be done in two steps to avoid breaking changes. I proposed one way to get there but I don't think it is the only way. I don't understand any of your comments around complexity, more subclasses, caching cyclic dependency and boiler plate. As part of the first step towards a clean api there has to be some shims to support old users until we can remove old apis but that's also true with all of these metadata patches. We shouldn't dismiss an ultimately cleaner solution because we have to maintain a shim for one release (and really, a shim must exist in either case).
   
   Lastly, it would be good to clarify what you mean by breaking changes. In both the proposal I've put forward and the current patch, I see deprecation/upgrade paths required (new apis that replace old apis exist simultaneously and then old apis are removed post version deprecation). In both cases, a user should have to do no code changes when upgrading but should see deprecation warnings and then the following upgrade would see breakage if they didn't solve deprecation warnings.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       To put this another way: RelMetadataQuery is the what, RelMetadataProvider is one possible how. Let's not couple the two.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {
+
+  <MH extends MetadataHandler<?>> MH initialHandler(Class<MH> handlerClass);
+
+  /** Re-generates the handler for a given kind of metadata.. */
+  <MH extends MetadataHandler<?>> MH revise(Class<MH> handlerClass);
+
+  /**
+   * Creates a new cache.
+   *
+   * @return A new cache for {@link RelMetadataQuery}
+   */
+  MetadataCache buildCache();

Review comment:
       I don't really understand the meaning of this method. Why does someone call this? What happens if I call it multiple times? What is this caching? Does it have to be called? Why is it on this interface? Also, if you wanted to customize the cache for things like a double property, wouldn't you potentially want to implement a cache that avoided the constant creation of objects and boxing (which the interface of MetadataCache forces)?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -62,49 +62,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, Object, Object> map = HashBasedTable.create();
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @API(status = API.Status.INTERNAL)
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;
+
+  @Deprecated // to be removed before 2.0
+  public final @Nullable JaninoRelMetadataProvider metadataProvider = THREAD_PROVIDERS.get();
 
   //~ Static fields/initializers ---------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public static final ThreadLocal<@Nullable JaninoRelMetadataProvider> THREAD_PROVIDERS =
       new ThreadLocal<>();
 
-  //~ Constructors -----------------------------------------------------------
-

Review comment:
       Isn't removing this constructor a breaking change?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +109,70 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
+  @Deprecated
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {
+    super(metadataHandlerProvider);
+    this.collationHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Collation.Handler.class);
+    this.columnOriginHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
+    this.expressionLineageHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
+    this.tableReferencesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.TableReferences.Handler.class);
+    this.columnUniquenessHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
+    this.cumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
+    this.distinctRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
+    this.distributionHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Distribution.Handler.class);
+    this.explainVisibilityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
+    this.maxRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
+    this.minRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
+    this.memoryHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Memory.Handler.class);
+    this.nonCumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
+    this.parallelismHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Parallelism.Handler.class);
     this.percentageOriginalRowsHandler =
-        initialHandler(BuiltInMetadata.PercentageOriginalRows.Handler.class);
-    this.populationSizeHandler = initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
-    this.predicatesHandler = initialHandler(BuiltInMetadata.Predicates.Handler.class);
-    this.allPredicatesHandler = initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
-    this.nodeTypesHandler = initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
-    this.rowCountHandler = initialHandler(BuiltInMetadata.RowCount.Handler.class);
-    this.selectivityHandler = initialHandler(BuiltInMetadata.Selectivity.Handler.class);
-    this.sizeHandler = initialHandler(BuiltInMetadata.Size.Handler.class);
-    this.uniqueKeysHandler = initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
-    this.lowerBoundCostHandler = initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
-  }
-
-  private RelMetadataQuery(JaninoRelMetadataProvider metadataProvider,
-      RelMetadataQuery prototype) {
-    super(requireNonNull(metadataProvider, "metadataProvider"));
+        metadataHandlerProvider.initialHandler(
+            BuiltInMetadata.PercentageOriginalRows.Handler.class);
+    this.populationSizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
+    this.predicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Predicates.Handler.class);
+    this.allPredicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
+    this.nodeTypesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
+    this.rowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.RowCount.Handler.class);
+    this.selectivityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Selectivity.Handler.class);
+    this.sizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Size.Handler.class);
+    this.uniqueKeysHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
+    this.lowerBoundCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
+  }
+
+  protected RelMetadataQuery(RelMetadataQuery prototype) {

Review comment:
       please make this private.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       Config feels like it would be weird here. As I have thought more about the name I'm pretty much okay with it as is.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I think I have a simple solution to avoiding the introduction of a new constructor on this class:
   - Make this constructor private
   - Add a new inner static subclass that uses that constructor and exposes a protected class.
   
   Then we tell people that want to use the handler behavior to extend that subclass. That way, we have deprecated subclassing RelMetadataQuery directly while moving towards this new pattern with minimal code changes.
   
   This avoids the introduction of a new coupling api between RelMetadataQuery the handler concept, is minimal changes in your patch and sets us on a path to moving RelMetadataQuery to an interface or dumb abstract class.

##########
File path: core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
##########
@@ -136,7 +134,7 @@ public RexBuilder getRexBuilder() {
     return rexBuilder;
   }
 
-  public @Nullable RelMetadataProvider getMetadataProvider() {
+  public RelMetadataProvider getMetadataProvider() {

Review comment:
       I see this marked resolved but don't see it being deprecated.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
+
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;

Review comment:
       Please make metadataHandlerProvider private. I think this means that for now you need to move it to relmetadataquery and pass a cache supplier into the RelMetadataQueryBase
   
   Please make the cache private. I think using the Api pattern to "hide" things should be used as a last resort. This may mean it should also move to RelMetadataQuery (for now).

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   -This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect `RelMetdataQuery.builder()` to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.




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



[GitHub] [calcite] jamesstarr commented on pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#issuecomment-958435989


   > @jamesstarr , what is the status of this PR? @jacques-n left some feedback but I think the PR was not synced to the main dev branch, it's not clear which issues are legit and need addressing, and which ones had already been solved.
   
   @jcamachor, @Jacques-n said he would look at it on tomorrow.  Overall, I think I need to add some more javadocs and possibly clean up some naming, but I am waiting for feed back from @jacques-n  before I do the last bit of clean up.


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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742284367



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       There are several requirements metadata in calcite:
   Metadata retrieval - the api for getting a metdata value for a relnode and give set of arguments.  This is currently the RelMetadataQuery.
   Metadata implementation - The code that is called to for a relnode to get a metadata value.  This is currently sub-classes of RelMetdataProvider.
   Defaulting and Validation - There is defaulting, canonization, validation, etc after a metadata value is retrieved in RelMetadataQuery.
   Dispatch - How the implementation is called for given arguments.  This is currently done in Janino compiled code.
   Cycle Detection - This is currently a **hard** requirement of volcano planner.  This is currently done in Janino compiled code.
   Caching Invalidation - This is required for volcano planner.  This is currently done through RelMetadataQuery.clearCache.
   Caching Implementation - This is coupled with cycle detection, but is currently accomplished through RelMetadataQuery.map.
   Caching Values - The caches need to be checked and updated.  This is currently done in the generated code.
   
   Metadata retrieval and Metadata implementation are hard requirements for external facing APIs.  They are currently exist as external facing APIs.  Many downstream projects depend on the ability to either customize existing metadata types or add their own metadata types.  When a down stream project adds their own metadata type, they need to be able to expose it to the larger system aka through metadata retrieval.  This is currently done by subclassing RelMetadataQuery and calling methods RelMetadataQueryBase.  An example of this can be seen in RelMetadataTest.MyRelMetadataQuery.
   
   So the metadata retrieval api is the only api need to for the rest of calcite core, however, downstream users of calcite need to be able to configure and extend the metadata implementation.  So some of RelMetadataQuery implementation must be exposed with the current setup.
   
   I support making RelMetadataQuery an interface so the metadata retrieval is clearly defined.  However, this is not the only public API necessary to expose.  Also, it is breaking change which does not separate the defaulting and validation from the RelMetadataQueryImpl.  If RelMetadataQuery was split into RelMetadataQuery(an interface), DefaultingAndValidatingRelMetadataQuery and HandlerBackedRelMetadataQuery, then down stream projects would have to implement 2 different RelMetadataQueries to add a custom metadata.  This api seems a bit convoluted and cumbersome.  Alternatively, breaking it up into class hierarchy of RelMetadataQuery -> DefaultingAndValidatingRelMetadataQuery ->  HandlerBackedRelMetadataQuery also feels a bit convoluted and is a breaking change.
   
   I also support removing the leaky abstraction of the metadata implementation to the rest of calcite core.  But changing this does not actually help things much other than remove a thread local that can cause odd behaviors in uncommon nesting scenarios. 




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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742424624



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   -This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect a builder on RelMetdataQuery to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r739619273



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       I have deprecated JaninoRelMetadataProvider and moved all the logic to RelMetadataHandlerGeneratorUtil.  I was actually think that the name should be broader so that anything could be injected would be passed through this interface oppose to be more explicit.  Maybe this should be a config object?




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



[GitHub] [calcite] jcamachor commented on pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jcamachor commented on pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#issuecomment-958403231


   @jamesstarr , what is the status of this PR? @jacques-n left some feedback but I think the PR was not synced to the main dev branch, it's not clear which issues are legit and need addressing, and which ones had already been solved.


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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742354266



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {
+
+  <MH extends MetadataHandler<?>> MH initialHandler(Class<MH> handlerClass);
+
+  /** Re-generates the handler for a given kind of metadata.. */
+  <MH extends MetadataHandler<?>> MH revise(Class<MH> handlerClass);
+
+  /**
+   * Creates a new cache.
+   *
+   * @return A new cache for {@link RelMetadataQuery}
+   */
+  MetadataCache buildCache();

Review comment:
       I don't really understand the meaning of this method. Why does someone call this? What happens if I call it multiple times? What is this caching? Does it have to be called? Why is it on this interface? Also, if you wanted to customize the cache for things like a double property, wouldn't you potentially want to implement a cache that avoided the constant creation of objects and boxing (which the interface of MetadataCache forces)?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -62,49 +62,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, Object, Object> map = HashBasedTable.create();
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @API(status = API.Status.INTERNAL)
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;
+
+  @Deprecated // to be removed before 2.0
+  public final @Nullable JaninoRelMetadataProvider metadataProvider = THREAD_PROVIDERS.get();
 
   //~ Static fields/initializers ---------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public static final ThreadLocal<@Nullable JaninoRelMetadataProvider> THREAD_PROVIDERS =
       new ThreadLocal<>();
 
-  //~ Constructors -----------------------------------------------------------
-

Review comment:
       Isn't removing this constructor a breaking change?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +109,70 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
+  @Deprecated
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {
+    super(metadataHandlerProvider);
+    this.collationHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Collation.Handler.class);
+    this.columnOriginHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
+    this.expressionLineageHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
+    this.tableReferencesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.TableReferences.Handler.class);
+    this.columnUniquenessHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
+    this.cumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
+    this.distinctRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
+    this.distributionHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Distribution.Handler.class);
+    this.explainVisibilityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
+    this.maxRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
+    this.minRowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
+    this.memoryHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Memory.Handler.class);
+    this.nonCumulativeCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
+    this.parallelismHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Parallelism.Handler.class);
     this.percentageOriginalRowsHandler =
-        initialHandler(BuiltInMetadata.PercentageOriginalRows.Handler.class);
-    this.populationSizeHandler = initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
-    this.predicatesHandler = initialHandler(BuiltInMetadata.Predicates.Handler.class);
-    this.allPredicatesHandler = initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
-    this.nodeTypesHandler = initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
-    this.rowCountHandler = initialHandler(BuiltInMetadata.RowCount.Handler.class);
-    this.selectivityHandler = initialHandler(BuiltInMetadata.Selectivity.Handler.class);
-    this.sizeHandler = initialHandler(BuiltInMetadata.Size.Handler.class);
-    this.uniqueKeysHandler = initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
-    this.lowerBoundCostHandler = initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
-  }
-
-  private RelMetadataQuery(JaninoRelMetadataProvider metadataProvider,
-      RelMetadataQuery prototype) {
-    super(requireNonNull(metadataProvider, "metadataProvider"));
+        metadataHandlerProvider.initialHandler(
+            BuiltInMetadata.PercentageOriginalRows.Handler.class);
+    this.populationSizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.PopulationSize.Handler.class);
+    this.predicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Predicates.Handler.class);
+    this.allPredicatesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.AllPredicates.Handler.class);
+    this.nodeTypesHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.NodeTypes.Handler.class);
+    this.rowCountHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.RowCount.Handler.class);
+    this.selectivityHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Selectivity.Handler.class);
+    this.sizeHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.Size.Handler.class);
+    this.uniqueKeysHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.UniqueKeys.Handler.class);
+    this.lowerBoundCostHandler =
+        metadataHandlerProvider.initialHandler(BuiltInMetadata.LowerBoundCost.Handler.class);
+  }
+
+  protected RelMetadataQuery(RelMetadataQuery prototype) {

Review comment:
       please make this private.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       Config feels like it would be weird here. As I have thought more about the name I'm pretty much okay with it as is.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I think I have a simple solution to avoiding the introduction of a new constructor on this class:
   - Make this constructor private
   - Add a new inner static subclass that uses that constructor and exposes a protected class.
   
   Then we tell people that want to use the handler behavior to extend that subclass. That way, we have deprecated subclassing RelMetadataQuery directly while moving towards this new pattern with minimal code changes.
   
   This avoids the introduction of a new coupling api between RelMetadataQuery the handler concept, is minimal changes in your patch and sets us on a path to moving RelMetadataQuery to an interface or dumb abstract class.

##########
File path: core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
##########
@@ -136,7 +134,7 @@ public RexBuilder getRexBuilder() {
     return rexBuilder;
   }
 
-  public @Nullable RelMetadataProvider getMetadataProvider() {
+  public RelMetadataProvider getMetadataProvider() {

Review comment:
       I see this marked resolved but don't see it being deprecated.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
+
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;

Review comment:
       Please make metadataHandlerProvider private. I think this means that for now you need to move it to relmetadataquery and pass a cache supplier into the RelMetadataQueryBase
   
   Please make the cache private. I think using the Api pattern to "hide" things should be used as a last resort. This may mean it should also move to RelMetadataQuery (for now).




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



[GitHub] [calcite] jamesstarr commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742284367



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       There are several requirements metadata in calcite:
   Metadata retrieval - the api for getting a metdata value for a relnode and give set of arguments.  This is currently the RelMetadataQuery.
   Metadata implementation - The code that is called to for a relnode to get a metadata value.  This is currently sub-classes of RelMetdataProvider.
   Defaulting and Validation - There is defaulting, canonization, validation, etc after a metadata value is retrieved in RelMetadataQuery.
   Dispatch - How the implementation is called for given arguments.  This is currently done in Janino compiled code.
   Cycle Detection - This is currently a **hard** requirement of volcano planner.  This is currently done in Janino compiled code.
   Caching Invalidation - This is required for volcano planner.  This is currently done through RelMetadataQuery.clearCache.
   Caching Implementation - This is coupled with cycle detection, but is currently accomplished through RelMetadataQuery.map.
   Caching Values - The caches need to be checked and updated.  This is currently done in the generated code.
   
   Metadata retrieval and Metadata implementation are hard requirements for external facing APIs.  They are currently exist as external facing APIs.  Many downstream projects depend on the ability to either customize existing metadata types or add their own metadata types.  When a down stream project adds their own metadata type, they need to be able to expose it to the larger system aka through metadata retrieval.  This is currently done by subclassing RelMetadataQuery and calling methods RelMetadataQueryBase.  An example of this can be seen in RelMetadataTest.MyRelMetadataQuery.
   
   So the metadata retrieval api is the only api need to for the rest of calcite core, however, downstream users of calcite need to be able to configure and extend the metadata implementation.  So some of RelMetadataQuery implementation must be exposed with the current setup.
   
   I support making RelMetadataQuery an interface so the metadata retrieval is clearly defined.  However, this is not the only public API necessary to expose.  Also, it is breaking change which does not separate the defaulting and validation from the RelMetadataQueryImpl.  If RelMetadataQuery was split into RelMetadataQuery(an interface), DefaultingAndValidatingRelMetadataQuery and HandlerBackedRelMetadataQuery, then down stream projects would have to implement 2 different RelMetadataQueries to add a custom metadata.  This api seems a bit convoluted and cumbersome.  Alternatively, breaking it up into class hierarchy of RelMetadataQuery -> DefaultingAndValidatingRelMetadataQuery ->  HandlerBackedRelMetadataQuery also feels a bit convoluted and is a breaking change.
   
   I also support removing the leaky abstraction of the metadata implementation to the rest of calcite core.  But changing this does not actually help things much other than remove a thread local that can cause odd behaviors in uncommon nesting scenarios. 

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       There are several requirements metadata in calcite:
   Metadata retrieval - the api for getting a metdata value for a relnode and give set of arguments.  This is currently the RelMetadataQuery.
   Metadata implementation - The code that is called to for a relnode to get a metadata value.  This is currently sub-classes of RelMetdataProvider.
   Defaulting and Validation - There is defaulting, canonization, validation, etc after a metadata value is retrieved in RelMetadataQuery.
   Dispatch - How the implementation is called for given arguments.  This is currently done in Janino compiled code.
   Cycle Detection - This is currently a **hard** requirement of volcano planner.  This is currently done in Janino compiled code.
   Caching Invalidation - This is required for volcano planner.  This is currently done through RelMetadataQuery.clearCache.
   Caching Implementation - This is coupled with cycle detection, but is currently accomplished through RelMetadataQuery.map.
   Caching Values - The caches need to be checked and updated.  This is currently done in the generated code.
   
   Metadata retrieval and Metadata implementation are hard requirements for external facing APIs.  They are currently exist as external facing APIs.  Many downstream projects depend on the ability to either customize existing metadata types or add their own metadata types.  When a down stream project adds their own metadata type, they need to be able to expose it to the larger system aka through metadata retrieval.  This is currently done by subclassing RelMetadataQuery and calling methods RelMetadataQueryBase.  An example of this can be seen in RelMetadataTest.MyRelMetadataQuery.
   
   So the metadata retrieval api is the only api need to for the rest of calcite core, however, downstream users of calcite need to be able to configure and extend the metadata implementation.  So some of RelMetadataQuery implementation must be exposed with the current setup.
   
   I support making RelMetadataQuery an interface so the metadata retrieval is clearly defined.  However, this is not the only public API necessary to expose.  Also, it is breaking change which does not separate the defaulting and validation from the RelMetadataQueryImpl.  If RelMetadataQuery was split into RelMetadataQuery(an interface), DefaultingAndValidatingRelMetadataQuery and HandlerBackedRelMetadataQuery, then down stream projects would have to implement 2 different RelMetadataQueries to add a custom metadata.  This api seems a bit convoluted and cumbersome.  Alternatively, breaking it up into class hierarchy of RelMetadataQuery -> DefaultingAndValidatingRelMetadataQuery ->  HandlerBackedRelMetadataQuery also feels a bit convoluted and is a breaking change.
   
   I also support removing the leaky abstraction of the metadata implementation to the rest of calcite core.  But changing this does not actually help things much other than remove a thread local that can cause odd behaviors in uncommon nesting scenarios. 




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



[GitHub] [calcite] jcamachor commented on pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jcamachor commented on pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#issuecomment-958403231


   @jamesstarr , what is the status of this PR? @jacques-n left some feedback but I think the PR was not synced to the main dev branch, it's not clear which issues are legit and need addressing, and which ones had already been solved.


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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r738683302



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/janino/CacheGenerator.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.janino;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.CyclicMetadataException;
+import org.apache.calcite.rel.metadata.DelegatingMetadataRel;
+import org.apache.calcite.rel.metadata.NullSentinel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.FlatLists;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Method;
+
+import static org.apache.calcite.rel.metadata.janino.CodeGeneratorUtil.argList;
+import static org.apache.calcite.rel.metadata.janino.CodeGeneratorUtil.paramList;
+
+/**
+ * Generates caching code for janino backed metadata.
+ */
+class CacheGenerator {

Review comment:
       Based on my initial analysis, it appears that all of this code can be removed from runtime generation. Is there a specific reason you don't think it can be? If you use objects throughout, I don't think it changes anything other than simplify things (and make them more debuggable). That being said, let's avoid  coupling the generification of metadatquery versus the generification of a the janino impl (separate prs).
   
   (Part of the argument against a pre-compiled generic approach would be to avoid boxing. In this case however, I think that point is moot is the the column key will always be boxed.

##########
File path: core/src/main/java/org/apache/calcite/plan/volcano/TopDownRuleDriver.java
##########
@@ -440,7 +440,7 @@ default boolean onProduce(RelNode node) {
       input.setExplored();
       for (RelSubset subset : input.getSet().subsets) {
         // Clear the LB cache as exploring state has changed.
-        input.getCluster().getMetadataQuery().clearCache(subset);
+        input.getCluster().getMetadataQuery().cache.clear(subset);

Review comment:
       This feels like a step backwards. It forces a RelMetadataQuery implementation to have a public field called cache of a certain type that exposes a method. Can you please revert it? Cache concerns should be internal (a relmetadataquery implementation may or may not have a cache and external users like VolcanoPlanner only need to be able to clear rows). If you need access in janino code, please pass in as part of object initialization, lambda or a subclass that exposes the property.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/TableMetadataCache.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Table;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.Map;
+
+/**
+ * Rel Metadata cache back by @see HashBasedTable table.
+ */
+public class TableMetadataCache implements MetadataCache {

Review comment:
       Let's move this to package protected visibility. It makes things clearer that this isn't supposed to be exposed as an api.

##########
File path: core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
##########
@@ -136,7 +134,7 @@ public RexBuilder getRexBuilder() {
     return rexBuilder;
   }
 
-  public @Nullable RelMetadataProvider getMetadataProvider() {
+  public RelMetadataProvider getMetadataProvider() {

Review comment:
       I think we should deprecate this method as part of this patch and remove any uses of it.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;

Review comment:
       This is potentially a change in behavior. Since the map is still visible, people may be relying on it (filling or clearing). Keeping the field but ignoring it is not a safe change. I think you have to do this change over two releases. Deprecate it in one release then stop using it/remove it in the next release.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;

Review comment:
       Note, that rather than exposing the cache as a public field to the world, I believe we can capture the same dynamic by simply passing a reference into the janino generated code (and anyone else who needs access to it).

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -114,13 +119,8 @@ protected RelMetadataQueryBase(@Nullable JaninoRelMetadataProvider metadataProvi
    * @param rel RelNode whose cached metadata should be removed
    * @return true if cache for the provided RelNode was not empty
    */
+  @Deprecated // to be removed before 2.0

Review comment:
       as mentioned elsewhere, please remove this deprecation.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/MetadataHandlerProvider.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+import org.apache.calcite.util.ControlFlowException;
+
+/**
+ * Provides {@link MetadataHandler} call sites and {@link MetadataCache} for
+ * {@link RelMetadataQuery}. The handlers provided are responsible for
+ * updating the cache stored in {@link RelMetadataQuery}.
+ */
+public interface MetadataHandlerProvider {

Review comment:
       I agree with this general approach. The concepts of code generation and ReviseableHandlerProvider are two separate concepts. I would actually name this more to that point e.g. "ReviseableHandlerProvider". As part of this introduction, can we make we remove JaninoRelMetadataProvider since now we really only need a ReviseableMetadataProvider that uses the JaninoReviseableHandlerProvider? (by remove I mean remove all code, uses and deprecate. It would still exist for compatibility reasons)

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/janino/CacheGenerator.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.janino;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.CyclicMetadataException;
+import org.apache.calcite.rel.metadata.DelegatingMetadataRel;
+import org.apache.calcite.rel.metadata.NullSentinel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.FlatLists;
+
+import com.google.common.collect.ImmutableList;
+
+import java.lang.reflect.Method;
+
+import static org.apache.calcite.rel.metadata.janino.CodeGeneratorUtil.argList;
+import static org.apache.calcite.rel.metadata.janino.CodeGeneratorUtil.paramList;
+
+/**
+ * Generates caching code for janino backed metadata.
+ */
+class CacheGenerator {

Review comment:
       Looking at this further, I think the janino cache refactoring should be separated into a separate pr after we get the updated apis merged. Can you pull this out for now?

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/janino/RelMetadataHandlerGenerator.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.janino;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.immutables.value.Value;
+
+import java.lang.reflect.Method;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Generates the {@link MetadataHandler} code.

Review comment:
       Per my other comments, can you please extract the janino code generation refactoring out of this patch and deal with it as a separate PR? Thanks

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       I'm not excited about depending on another provider interface in RelMetadataQuery (and introducing another public api). Can we come up with a way to extract this out? (Separate the concept of handlers from the relmetadataquery surface area.)

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQueryBase.java
##########
@@ -63,49 +61,56 @@
  * </ol>
  */
 public class RelMetadataQueryBase {
+
   //~ Instance fields --------------------------------------------------------
 
   /** Set of active metadata queries, and cache of previous results. */
-  public final Table<RelNode, List, Object> map = HashBasedTable.create();
 
-  public final @Nullable JaninoRelMetadataProvider metadataProvider;
+  @Deprecated // to be removed before 2.0
+  public final Table<RelNode, Object, Object> map;
+
+  public final MetadataCache cache;
+  protected final MetadataHandlerProvider metadataHandlerProvider;

Review comment:
       I'm against this change as it doesn't really reduce the required pattern/encapsulation between RelMetadataQuery and another provider concept.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/janino/CacheUtil.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.janino;
+
+import java.util.stream.IntStream;
+
+/**
+ * Functions used by generated code.
+ */
+public class CacheUtil {

Review comment:
       Can we make this package visible? Note, if you're having visibility problems, you can control what package the janino code produces in to address. Alternatively, declare this as a interface and have the janino generated code that needs access to it extend this interface.

##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -107,47 +106,69 @@
   private BuiltInMetadata.LowerBoundCost.Handler lowerBoundCostHandler;
 
   /**
-   * Creates the instance with {@link JaninoRelMetadataProvider} instance
-   * from {@link #THREAD_PROVIDERS} and {@link #EMPTY} as a prototype.
+   * Creates the instance using the Handler.classualt prototype.
    */
   protected RelMetadataQuery() {
-    this(castNonNull(THREAD_PROVIDERS.get()), EMPTY);
+    this(PROTOTYPE);
   }
 
   /** Creates and initializes the instance that will serve as a prototype for
    * all other instances. */
-  private RelMetadataQuery(@SuppressWarnings("unused") boolean dummy) {
-    super(null);
-    this.collationHandler = initialHandler(BuiltInMetadata.Collation.Handler.class);
-    this.columnOriginHandler = initialHandler(BuiltInMetadata.ColumnOrigin.Handler.class);
-    this.expressionLineageHandler = initialHandler(BuiltInMetadata.ExpressionLineage.Handler.class);
-    this.tableReferencesHandler = initialHandler(BuiltInMetadata.TableReferences.Handler.class);
-    this.columnUniquenessHandler = initialHandler(BuiltInMetadata.ColumnUniqueness.Handler.class);
-    this.cumulativeCostHandler = initialHandler(BuiltInMetadata.CumulativeCost.Handler.class);
-    this.distinctRowCountHandler = initialHandler(BuiltInMetadata.DistinctRowCount.Handler.class);
-    this.distributionHandler = initialHandler(BuiltInMetadata.Distribution.Handler.class);
-    this.explainVisibilityHandler = initialHandler(BuiltInMetadata.ExplainVisibility.Handler.class);
-    this.maxRowCountHandler = initialHandler(BuiltInMetadata.MaxRowCount.Handler.class);
-    this.minRowCountHandler = initialHandler(BuiltInMetadata.MinRowCount.Handler.class);
-    this.memoryHandler = initialHandler(BuiltInMetadata.Memory.Handler.class);
-    this.nonCumulativeCostHandler = initialHandler(BuiltInMetadata.NonCumulativeCost.Handler.class);
-    this.parallelismHandler = initialHandler(BuiltInMetadata.Parallelism.Handler.class);
+  protected RelMetadataQuery(MetadataHandlerProvider metadataHandlerProvider) {

Review comment:
       This `MetadataProvider` interface is really "a handler way to deal with metadata that is no longer coupled to janino". I agree that this abstraction makes sense but it feels too specific to be added as a RelMetadataQuery dependency. I'd like to see it instead added to some separate impl specific version of relmetadataquery e.g. . My best thought currently is modifying RelMetadataQuery to be a decorator on top of a new "AbstractRelMetdataQuery". Then the handler concepts can be moved to an implementation of AbstractRelMetdataQuery and RelMetadataQuery doesn't depend on a new implementation specific interface. Concepts like null -> sentinel conversion and convenience delegations (getColumnOrigin -> getColumnOrigins) can be done at the RelMetadataQuery level but other concepts should be removed.




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



[GitHub] [calcite] jacques-n commented on a change in pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#discussion_r742424624



##########
File path: core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java
##########
@@ -886,10 +919,59 @@ public Boolean isVisibleInExplain(RelNode rel,
     for (;;) {
       try {
         return lowerBoundCostHandler.getLowerBoundCost(rel, this, planner);
-      } catch (JaninoRelMetadataProvider.NoHandler e) {
+      } catch (MetadataHandlerProvider.NoHandler e) {
         lowerBoundCostHandler =
-            revise(BuiltInMetadata.LowerBoundCost.Handler.class);
+            metadataHandlerProvider.revise(BuiltInMetadata.LowerBoundCost.Handler.class);
       }
     }
   }
+
+  public static Builder<RelMetadataQuery> builder() {

Review comment:
       If we want to go with a builder, I think we should make a couple changes. (Not entirely sure a builder is yet necessary.)
   
   - This should probably be a method called `supplierBuilder()` or similar since it doesn't actually build the type of class it is contained within. I'd expect `RelMetdataQuery.builder()` to build a `RelMetadataQuery`, not `Supplier<RelMetadataQuery>`.
   - I think that we may want to add other ways of building in the future. For example I'm exploring the introduction of a lambda based metadata handling systems. In that situation, we may want to construct one of those using the same initial builder. As such I suggest a specialization strategy where calling withProviders returns a subtype of builder that has configuration options related to a provider based builder. Maybe in the future we have a separate one that works based on lambdas and we call withLambdas() to use it (and a call to that would specialize the builder to that type of metadataquery supplier.




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



[GitHub] [calcite] jamesstarr commented on pull request #2378: [CALCITE-4539] Customization of Metadata Handlers

Posted by GitBox <gi...@apache.org>.
jamesstarr commented on pull request #2378:
URL: https://github.com/apache/calcite/pull/2378#issuecomment-958435989


   > @jamesstarr , what is the status of this PR? @jacques-n left some feedback but I think the PR was not synced to the main dev branch, it's not clear which issues are legit and need addressing, and which ones had already been solved.
   
   @jcamachor, @Jacques-n said he would look at it on tomorrow.  Overall, I think I need to add some more javadocs and possibly clean up some naming, but I am waiting for feed back from @jacques-n  before I do the last bit of clean up.


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