You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ko...@apache.org on 2021/11/15 11:41:15 UTC

[ignite-3] branch main updated: IGNITE-15833 Provide interfaces for SQL Extension API (#439)

This is an automated email from the ASF dual-hosted git repository.

korlov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new cc88e41  IGNITE-15833 Provide interfaces for SQL Extension API (#439)
cc88e41 is described below

commit cc88e410c83f34f380dc48fd28ef6afd09fc4b08
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Mon Nov 15 14:41:10 2021 +0300

    IGNITE-15833 Provide interfaces for SQL Extension API (#439)
---
 .../query/calcite/exec/ExecutionServiceImpl.java   |  16 +--
 .../CatalogUpdateListener.java}                    |  26 ++--
 .../query/calcite/extension/SqlExtension.java      | 149 +++++++++++++++++++++
 .../query/calcite/externalize/RelJson.java         |   4 +-
 .../query/calcite/prepare/FieldsMetadataImpl.java  |  52 -------
 .../query/calcite/rel/AbstractIgniteSpool.java     |   2 +-
 .../query/calcite/rel/IgniteAggregate.java         |   2 +-
 .../query/calcite/rel/IgniteConvention.java        |   2 +-
 .../query/calcite/rel/IgniteExchange.java          |   2 +-
 .../query/calcite/rel/IgniteHashIndexSpool.java    |   2 +-
 .../processors/query/calcite/rel/IgniteLimit.java  |   2 +-
 .../query/calcite/rel/IgniteReceiver.java          |   2 +-
 .../processors/query/calcite/rel/IgniteSender.java |   2 +-
 .../processors/query/calcite/rel/IgniteSort.java   |   2 +-
 .../query/calcite/rel/IgniteSortedIndexSpool.java  |   2 +-
 .../query/calcite/rel/IgniteTableFunctionScan.java |   2 +-
 .../query/calcite/rel/IgniteTableModify.java       |   2 +-
 .../query/calcite/rel/IgniteTableSpool.java        |   2 +-
 .../processors/query/calcite/rel/IgniteValues.java |   2 +-
 .../InternalIgniteRel.java}                        |  23 +---
 .../query/calcite/rel/SourceAwareIgniteRel.java    |   2 +-
 .../query/calcite/trait/TraitsAwareIgniteRel.java  |   4 +-
 22 files changed, 192 insertions(+), 112 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index 0d369e3..367535a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import static java.util.Collections.singletonList;
 import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
-import static org.apache.ignite.internal.processors.query.calcite.exec.PlannerHelper.optimize;
 import static org.apache.ignite.internal.processors.query.calcite.externalize.RelJsonReader.fromJson;
+import static org.apache.ignite.internal.processors.query.calcite.prepare.PlannerHelper.optimize;
 import static org.apache.ignite.internal.processors.query.calcite.util.Commons.FRAMEWORK_CONFIG;
 import static org.apache.ignite.internal.util.CollectionUtils.first;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
@@ -392,9 +392,9 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService {
         ValidationResult validated = planner.validateAndGetTypeMetadata(sqlNode);
 
         sqlNode = validated.sqlNode();
-
-        IgniteRel igniteRel = optimize(sqlNode, planner, LOG);
-
+        
+        IgniteRel igniteRel = optimize(sqlNode, planner);
+        
         // Split query plan to query fragments.
         List<Fragment> fragments = new Splitter().go(igniteRel);
 
@@ -448,8 +448,8 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService {
         sqlNode = planner.validate(sqlNode);
 
         // Convert to Relational operators graph
-        IgniteRel igniteRel = optimize(sqlNode, planner, LOG);
-
+        IgniteRel igniteRel = optimize(sqlNode, planner);
+        
         // Split query plan to query fragments.
         List<Fragment> fragments = new Splitter().go(igniteRel);
 
@@ -475,8 +475,8 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService {
         sql = planner.validate(sql);
 
         // Convert to Relational operators graph
-        IgniteRel igniteRel = optimize(sql, planner, LOG);
-
+        IgniteRel igniteRel = optimize(sql, planner);
+        
         String plan = RelOptUtil.toString(igniteRel, SqlExplainLevel.ALL_ATTRIBUTES);
 
         return new ExplainPlan(plan, explainFieldsMetadata(ctx));
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/extension/CatalogUpdateListener.java
similarity index 50%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/extension/CatalogUpdateListener.java
index dc0a62c..3a34759 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/extension/CatalogUpdateListener.java
@@ -2,11 +2,11 @@
  * 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 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
+ * 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,
@@ -15,23 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.prepare;
+package org.apache.ignite.internal.processors.query.calcite.extension;
 
-import java.util.List;
-import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.extension.SqlExtension.ExternalCatalog;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
 
 /**
- * FieldsMetadata interface.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+ * Listener used to notify {@link SchemaHolder} about any changes in the external catalogs.
  */
-public interface FieldsMetadata {
+public interface CatalogUpdateListener {
     /**
-     * Get result row type.
+     * Notify the {@link SchemaHolder} that provided catalog has been updated.
+     *
+     * @param catalog Catalog to notify the {@link SchemaHolder} about.
      */
-    RelDataType rowType();
-
-    /**
-     * Get result row origins (or where a field value comes from).
-     */
-    List<List<String>> origins();
+    void onCatalogUpdated(ExternalCatalog catalog);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/extension/SqlExtension.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/extension/SqlExtension.java
new file mode 100644
index 0000000..7fa84ed
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/extension/SqlExtension.java
@@ -0,0 +1,149 @@
+/*
+ * 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.ignite.internal.processors.query.calcite.extension;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerPhase;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Entry point to extend current sql engine with external storage or even custom execution.
+ */
+public interface SqlExtension {
+    /**
+     * Returns the name of the current extension.
+     *
+     * <p>This name will be used to distinguish between different
+     * extensions. Also the {@link CatalogUpdateListener} will register
+     * provided catalog with the same name.
+     *
+     * @return Name of the extension.
+     */
+    String name();
+
+    /**
+     * Initializes the extension before use.
+     *
+     * @param ignite Instance of the current Ignite node.
+     * @param catalogUpdateListener Listener to notify when new table or schema
+     *                              are available. Note: the catalog listener creates
+     *                              copy of the provided catalog, so its not enough
+     *                              to just update {@link ExternalCatalog catalog} or
+     *                              {@link ExternalSchema schema}, the listener should
+     *                              be called explicitly to register changes.
+     * @see ExternalSchema
+     * @see ExternalCatalog
+     */
+    void init(
+            Ignite ignite,
+            CatalogUpdateListener catalogUpdateListener
+    );
+
+    /**
+     * Returns a set of optimization rules for given optimization phase.
+     *
+     * @param phase Current optimization phase.
+     * @return Set of rules, or empty set if there are no rules for given phase.
+     */
+    Set<? extends RelOptRule> getOptimizerRules(PlannerPhase phase);
+    
+    /**
+     * Returns an implementor of relations provided by current extension.
+     *
+     * @param <RowT> Type of the rows used in current runtime.
+     * @return Implementor of the relations provided by current extension.
+     * @see org.apache.ignite.internal.processors.query.calcite.exec.RowHandler
+     * @see org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory
+     * @see RelImplementor
+     */
+    <RowT> RelImplementor<RowT> implementor();
+    
+    /**
+     * Returns colocation group for given relational tree.
+     *
+     * <p>It's guaranteed that this tree will only consist of the relations
+     * provided by the current extension.
+     *
+     * @param node Relational tree to find colocation for.
+     * @return Colocation of given relation tree.
+     */
+    ColocationGroup colocationGroup(IgniteRel node);
+    
+    /**
+     * Implementer to create execution nodes from provided relational nodes.
+     *
+     * <p>Should provide implementation for every physical node introduced by current extension.
+     *
+     * @param <RowT> Type of the rows used in current runtime.
+     * @see org.apache.ignite.internal.processors.query.calcite.exec.RowHandler
+     * @see org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory
+     */
+    interface RelImplementor<RowT> {
+        /**
+         * Converts given relational tree into an execution tree.
+         *
+         * <p>It's guaranteed that the tree will only consist of the relations
+         * provided by the current extension.
+         *
+         * @param ctx An execution context.
+         * @param node A root of the relational tree.
+         * @return A root of the resulting execution tree.
+         */
+        Node<RowT> implement(ExecutionContext<RowT> ctx, IgniteRel node);
+    }
+    
+    /**
+     * Represents an external SQL schema that is simply a group different tables.
+     */
+    interface ExternalSchema {
+        /** Returns list of all tables provided by current schema. */
+        List<String> tableNames();
+    
+        /**
+         * Returns table by its name.
+         *
+         * @param name Name of the table.
+         * @return The table, or {@code null} if there is no table with given name.
+         */
+        @Nullable RelOptTable table(String name);
+    }
+    
+    /**
+     * Represents an external SQL catalog that is simply a group different schemas.
+     */
+    interface ExternalCatalog {
+        /** Returns list of all schemas provided by current catalog. */
+        List<String> schemaNames();
+    
+        /**
+         * Returns schema by its name.
+         *
+         * @param name Name of the schema.
+         * @return The schema, or {@code null} if there is no schema with given name.
+         */
+        @Nullable ExternalSchema schema(String name);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java
index da9ffc5..41092ee 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/externalize/RelJson.java
@@ -99,7 +99,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlNameMatchers;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.InternalIgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
@@ -234,7 +234,7 @@ class RelJson {
     }
 
     String classToTypeName(Class<? extends RelNode> cls) {
-        if (IgniteRel.class.isAssignableFrom(cls)) {
+        if (InternalIgniteRel.class.isAssignableFrom(cls)) {
             return cls.getSimpleName();
         }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
deleted file mode 100644
index 2c918d5..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-import org.apache.calcite.rel.type.RelDataType;
-
-/**
- * FieldsMetadataImpl.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
- */
-public class FieldsMetadataImpl implements FieldsMetadata {
-    private final RelDataType rowType;
-
-    private final List<List<String>> origins;
-
-    /**
-     * Constructor.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
-     */
-    public FieldsMetadataImpl(RelDataType rowType, List<List<String>> origins) {
-        this.rowType = rowType;
-        this.origins = origins;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public RelDataType rowType() {
-        return rowType;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public List<List<String>> origins() {
-        return origins;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteSpool.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteSpool.java
index fb8002a..2d83206 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteSpool.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteSpool.java
@@ -26,7 +26,7 @@ import org.apache.calcite.rel.core.Spool;
 /**
  * Relational operator that returns the contents of a table.
  */
-public abstract class AbstractIgniteSpool extends Spool implements IgniteRel {
+public abstract class AbstractIgniteSpool extends Spool implements InternalIgniteRel {
     /**
      * Constructor.
      * AbstractIgniteSpool
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java
index 693c148..4e6f4e9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java
@@ -37,7 +37,7 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteC
  * IgniteAggregate.
  * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
  */
-public abstract class IgniteAggregate extends Aggregate implements IgniteRel {
+public abstract class IgniteAggregate extends Aggregate implements InternalIgniteRel {
     /** {@inheritDoc} */
     protected IgniteAggregate(
             RelOptCluster cluster,
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java
index f733666..8bd402d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java
@@ -30,7 +30,7 @@ public class IgniteConvention extends Convention.Impl {
     public static final IgniteConvention INSTANCE = new IgniteConvention();
 
     private IgniteConvention() {
-        super("IGNITE", IgniteRel.class);
+        super("IGNITE", InternalIgniteRel.class);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
index 7731897..1b3f311 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
@@ -37,7 +37,7 @@ import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribut
 /**
  * Relational expression that imposes a particular distribution on its input without otherwise changing its content.
  */
-public class IgniteExchange extends Exchange implements IgniteRel {
+public class IgniteExchange extends Exchange implements InternalIgniteRel {
     /**
      * Creates an Exchange.
      *
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashIndexSpool.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashIndexSpool.java
index 90c7c91..a6ce982 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashIndexSpool.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashIndexSpool.java
@@ -38,7 +38,7 @@ import org.apache.ignite.internal.processors.query.calcite.util.RexUtils;
 /**
  * Relational operator that returns the hashed contents of a table and allow to lookup rows by specified keys.
  */
-public class IgniteHashIndexSpool extends AbstractIgniteSpool implements IgniteRel {
+public class IgniteHashIndexSpool extends AbstractIgniteSpool implements InternalIgniteRel {
     /** Search row. */
     private final List<RexNode> searchRow;
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteLimit.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteLimit.java
index 4475844..15f3ab7 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteLimit.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteLimit.java
@@ -37,7 +37,7 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteC
  * IgniteLimit.
  * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
  */
-public class IgniteLimit extends SingleRel implements IgniteRel {
+public class IgniteLimit extends SingleRel implements InternalIgniteRel {
     /** In case the fetch value is a DYNAMIC_PARAM. */
     private static final double FETCH_IS_PARAM_FACTOR = 0.01;
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
index 4400302..83cf923 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
@@ -33,7 +33,7 @@ import org.apache.calcite.util.Pair;
 /**
  * Relational expression that receives elements from remote {@link IgniteSender}.
  */
-public class IgniteReceiver extends AbstractRelNode implements IgniteRel {
+public class IgniteReceiver extends AbstractRelNode implements InternalIgniteRel {
     private final long exchangeId;
 
     private final long sourceFragmentId;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java
index 0c18874..e861984 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java
@@ -33,7 +33,7 @@ import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribut
 /**
  * Relational expression that iterates over its input and sends elements to remote {@link IgniteReceiver}.
  */
-public class IgniteSender extends SingleRel implements IgniteRel {
+public class IgniteSender extends SingleRel implements InternalIgniteRel {
     private final long exchangeId;
 
     private final long targetFragmentId;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSort.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSort.java
index ef0a4c9..11551b7 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSort.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSort.java
@@ -40,7 +40,7 @@ import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 /**
  * Ignite sort operator.
  */
-public class IgniteSort extends Sort implements IgniteRel {
+public class IgniteSort extends Sort implements InternalIgniteRel {
     /**
      * Constructor.
      *
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSortedIndexSpool.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSortedIndexSpool.java
index 542ca72..ab05633 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSortedIndexSpool.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSortedIndexSpool.java
@@ -37,7 +37,7 @@ import org.apache.ignite.internal.processors.query.calcite.util.IndexConditions;
 /**
  * Relational operator that returns the sorted contents of a table and allow to lookup rows by specified bounds.
  */
-public class IgniteSortedIndexSpool extends AbstractIgniteSpool implements IgniteRel {
+public class IgniteSortedIndexSpool extends AbstractIgniteSpool implements InternalIgniteRel {
     private final RelCollation collation;
 
     /** Index condition. */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableFunctionScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableFunctionScan.java
index acb86e5..cc5b09a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableFunctionScan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableFunctionScan.java
@@ -36,7 +36,7 @@ import org.apache.calcite.rex.RexNode;
 /**
  * Relational operator for table function scan.
  */
-public class IgniteTableFunctionScan extends TableFunctionScan implements IgniteRel {
+public class IgniteTableFunctionScan extends TableFunctionScan implements InternalIgniteRel {
     /** Default estimate row count. */
     private static final int ESTIMATE_ROW_COUNT = 100;
     
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableModify.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableModify.java
index 76270ef..e0fa477 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableModify.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableModify.java
@@ -31,7 +31,7 @@ import org.apache.ignite.internal.processors.query.calcite.util.Commons;
  * IgniteTableModify.
  * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
  */
-public class IgniteTableModify extends TableModify implements IgniteRel {
+public class IgniteTableModify extends TableModify implements InternalIgniteRel {
     /**
      * Creates a {@code TableModify}.
      *
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableSpool.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableSpool.java
index 178b86a..7b0f70f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableSpool.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableSpool.java
@@ -32,7 +32,7 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteC
 /**
  * Relational operator that returns the contents of a table.
  */
-public class IgniteTableSpool extends AbstractIgniteSpool implements IgniteRel {
+public class IgniteTableSpool extends AbstractIgniteSpool implements InternalIgniteRel {
     /**
      * Constructor.
      * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteValues.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteValues.java
index 58f6f8f..ed2ba7e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteValues.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteValues.java
@@ -33,7 +33,7 @@ import org.apache.calcite.rex.RexLiteral;
  * IgniteValues.
  * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
  */
-public class IgniteValues extends Values implements IgniteRel {
+public class IgniteValues extends Values implements InternalIgniteRel {
     /**
      * Creates a new Values.
      *
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/InternalIgniteRel.java
similarity index 54%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/InternalIgniteRel.java
index dc0a62c..f0d70c5 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/InternalIgniteRel.java
@@ -2,11 +2,11 @@
  * 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 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
+ * 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,
@@ -15,23 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-import org.apache.calcite.rel.type.RelDataType;
+package org.apache.ignite.internal.processors.query.calcite.rel;
 
 /**
- * FieldsMetadata interface.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+ * Marker interface to segregate internal relations from those provided by extensions.
  */
-public interface FieldsMetadata {
-    /**
-     * Get result row type.
-     */
-    RelDataType rowType();
-
-    /**
-     * Get result row origins (or where a field value comes from).
-     */
-    List<List<String>> origins();
+public interface InternalIgniteRel extends IgniteRel {
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/SourceAwareIgniteRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/SourceAwareIgniteRel.java
index 961e655..cf36013 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/SourceAwareIgniteRel.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/SourceAwareIgniteRel.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.query.calcite.rel;
 /**
  * SourceAwareIgniteRel interface.
  */
-public interface SourceAwareIgniteRel extends IgniteRel {
+public interface SourceAwareIgniteRel extends InternalIgniteRel {
     long sourceId();
 
     IgniteRel clone(long sourceId);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java
index 53f12f2..91f5394 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java
@@ -25,14 +25,14 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.InternalIgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * TraitsAwareIgniteRel interface.
  * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
  */
-public interface TraitsAwareIgniteRel extends IgniteRel {
+public interface TraitsAwareIgniteRel extends InternalIgniteRel {
     /** {@inheritDoc} */
     @Override
     public default List<RelNode> derive(List<List<RelTraitSet>> inTraits) {