You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by gv...@apache.org on 2019/12/13 13:03:46 UTC
[ignite] 01/01: IGNITE-12248: Apache Calcite based query execution
engine. Initial commit.
This is an automated email from the ASF dual-hosted git repository.
gvvinblade pushed a commit to branch ignite-12248
in repository https://gitbox.apache.org/repos/asf/ignite.git
commit 1958c3e542b8b0d8a9491678f5270cd53f56abd1
Author: Igor Seliverstov <gv...@gmail.com>
AuthorDate: Fri Dec 13 16:02:54 2019 +0300
IGNITE-12248: Apache Calcite based query execution engine.
Initial commit.
---
modules/calcite/README.txt | 35 +
modules/calcite/pom.xml | 112 ++
.../calcite/interpreter/InterpreterUtils.java | 28 +
.../apache/calcite/plan/volcano/VolcanoUtils.java | 34 +
.../query/calcite/CalciteQueryProcessor.java | 167 +++
.../query/calcite/cluster/MappingServiceImpl.java | 134 +++
.../query/calcite/exchange/EndMarker.java | 33 +
.../query/calcite/exchange/ExchangeProcessor.java | 36 +
.../processors/query/calcite/exchange/Inbox.java | 70 ++
.../processors/query/calcite/exchange/Outbox.java | 163 +++
.../query/calcite/exec/AbstractNode.java | 44 +
.../query/calcite/exec/ConsumerNode.java | 75 ++
.../processors/query/calcite/exec/FilterNode.java | 47 +
.../processors/query/calcite/exec/Implementor.java | 174 +++
.../processors/query/calcite/exec/JoinNode.java | 101 ++
.../processors/query/calcite/exec/Node.java | 27 +
.../processors/query/calcite/exec/ProjectNode.java | 47 +
.../query/calcite/exec/ScalarFactory.java | 151 +++
.../processors/query/calcite/exec/ScanNode.java | 70 ++
.../processors/query/calcite/exec/SingleNode.java | 33 +
.../processors/query/calcite/exec/Sink.java | 38 +
.../processors/query/calcite/exec/Source.java | 24 +
.../query/calcite/metadata/FragmentInfo.java | 75 ++
.../metadata/IgniteMdDerivedDistribution.java | 149 +++
.../calcite/metadata/IgniteMdDistribution.java | 123 ++
.../calcite/metadata/IgniteMdFragmentInfo.java | 110 ++
.../query/calcite/metadata/IgniteMetadata.java | 68 ++
.../calcite/metadata/LocationMappingException.java | 26 +
.../query/calcite/metadata/MappingService.java | 28 +
.../query/calcite/metadata/NodesMapping.java | 196 ++++
.../metadata/OptimisticPlanningException.java | 35 +
.../query/calcite/metadata/RelMetadataQueryEx.java | 107 ++
.../query/calcite/prepare/ContextValue.java | 44 +
.../query/calcite/prepare/DataContextImpl.java | 67 ++
.../calcite/prepare/DistributedExecution.java | 110 ++
.../query/calcite/prepare/IgnitePlanner.java | 436 +++++++
.../query/calcite/prepare/IgniteSqlValidator.java | 55 +
.../query/calcite/prepare/PlannerContext.java | 194 ++++
.../query/calcite/prepare/PlannerPhase.java | 60 +
.../query/calcite/prepare/PlannerType.java | 26 +
.../processors/query/calcite/prepare/Query.java | 73 ++
.../query/calcite/prepare/QueryExecution.java | 28 +
.../query/calcite/rel/IgniteConvention.java | 46 +
.../query/calcite/rel/IgniteExchange.java | 40 +
.../processors/query/calcite/rel/IgniteFilter.java | 40 +
.../processors/query/calcite/rel/IgniteJoin.java | 43 +
.../query/calcite/rel/IgniteProject.java | 42 +
.../query/calcite/rel/IgniteReceiver.java | 61 +
.../processors/query/calcite/rel/IgniteRel.java | 26 +
.../query/calcite/rel/IgniteRelVisitor.java | 38 +
.../processors/query/calcite/rel/IgniteSender.java | 57 +
.../query/calcite/rel/IgniteTableScan.java | 41 +
.../processors/query/calcite/rel/RelOp.java | 27 +
.../query/calcite/rule/FilterConverter.java | 62 +
.../query/calcite/rule/IgniteConverter.java | 74 ++
.../query/calcite/rule/JoinConverter.java | 68 ++
.../query/calcite/rule/ProjectConverter.java | 63 +
.../query/calcite/rule/TableScanConverter.java | 44 +
.../query/calcite/schema/CalciteSchemaHolder.java | 69 ++
.../query/calcite/schema/IgniteSchema.java | 88 ++
.../query/calcite/schema/IgniteTable.java | 137 +++
.../processors/query/calcite/serialize/Graph.java | 78 ++
.../query/calcite/serialize/GraphNode.java | 25 +
.../serialize/expression/CallExpression.java | 40 +
.../expression/DynamicParamExpression.java | 37 +
.../serialize/expression/ExpImplementor.java | 32 +
.../serialize/expression/ExpToRexTranslator.java | 100 ++
.../calcite/serialize/expression/Expression.java | 26 +
.../serialize/expression/InputRefExpression.java | 37 +
.../serialize/expression/LiteralExpression.java | 37 +
.../serialize/expression/LocalRefExpression.java | 37 +
.../serialize/expression/RexToExpTranslator.java | 101 ++
.../serialize/relation/ConversionContext.java | 36 +
.../calcite/serialize/relation/FilterNode.java | 58 +
.../serialize/relation/GraphToRelConverter.java | 90 ++
.../query/calcite/serialize/relation/JoinNode.java | 66 ++
.../calcite/serialize/relation/ProjectNode.java | 62 +
.../calcite/serialize/relation/ReceiverNode.java | 53 +
.../query/calcite/serialize/relation/RelGraph.java | 25 +
.../calcite/serialize/relation/RelGraphNode.java | 38 +
.../serialize/relation/RelToGraphConverter.java | 109 ++
.../calcite/serialize/relation/SenderNode.java | 56 +
.../serialize/relation/SerializedTraits.java | 65 ++
.../calcite/serialize/relation/TableScanNode.java | 44 +
.../query/calcite/serialize/type/DataType.java | 32 +
.../query/calcite/serialize/type/SimpleType.java | 59 +
.../query/calcite/serialize/type/StructType.java | 51 +
.../processors/query/calcite/splitter/Edge.java | 46 +
.../query/calcite/splitter/Fragment.java | 92 ++
.../query/calcite/splitter/QueryPlan.java | 76 ++
.../query/calcite/splitter/RelSource.java | 47 +
.../query/calcite/splitter/RelSourceImpl.java | 41 +
.../query/calcite/splitter/RelTarget.java | 29 +
.../query/calcite/splitter/RelTargetImpl.java | 48 +
.../query/calcite/splitter/Splitter.java | 122 ++
.../trait/AbstractDestinationFunctionFactory.java | 35 +
.../query/calcite/trait/AffinityFactory.java | 60 +
.../query/calcite/trait/AllTargetsFactory.java | 45 +
.../query/calcite/trait/DestinationFunction.java | 27 +
.../calcite/trait/DestinationFunctionFactory.java | 31 +
.../query/calcite/trait/DistributionTrait.java | 174 +++
.../query/calcite/trait/DistributionTraitDef.java | 78 ++
.../query/calcite/trait/HashFunctionFactory.java | 72 ++
.../query/calcite/trait/IgniteDistribution.java | 28 +
.../query/calcite/trait/IgniteDistributions.java | 272 +++++
.../query/calcite/trait/NoOpFactory.java | 41 +
.../query/calcite/trait/RandomTargetFactory.java | 47 +
.../query/calcite/trait/SingleTargetFactory.java | 48 +
.../query/calcite/type/IgniteTypeFactory.java | 33 +
.../query/calcite/type/IgniteTypeSystem.java | 28 +
.../processors/query/calcite/type/RowType.java | 140 +++
.../processors/query/calcite/util/Commons.java | 145 +++
.../query/calcite/util/IgniteMethod.java | 42 +
.../query/calcite/util/ListFieldsQueryCursor.java | 92 ++
.../query/calcite/util/TableScanIterator.java | 149 +++
.../query/calcite/CalciteQueryProcessorTest.java | 1203 ++++++++++++++++++++
.../query/calcite/exchange/OutboxTest.java | 146 +++
.../query/calcite/exec/ExecutionTest.java | 77 ++
.../ignite/testsuites/IgniteCalciteTestSuite.java | 35 +
.../ignite/internal/IgniteComponentType.java | 8 +
.../processors/query/GridQueryProcessor.java | 31 +-
.../processors/query/GridQueryTypeDescriptor.java | 5 +
.../internal/processors/query/QueryContext.java | 73 ++
.../internal/processors/query/QueryEngine.java | 61 +
.../processors/query/QueryTypeDescriptorImpl.java | 6 +-
.../query/schema/SchemaChangeListener.java | 35 +
.../GridInternalSubscriptionProcessor.java | 16 +
.../processors/query/h2/SchemaManager.java | 67 +-
parent/pom.xml | 2 +
pom.xml | 1 +
130 files changed, 9960 insertions(+), 7 deletions(-)
diff --git a/modules/calcite/README.txt b/modules/calcite/README.txt
new file mode 100644
index 0000000..e144a77
--- /dev/null
+++ b/modules/calcite/README.txt
@@ -0,0 +1,35 @@
+Apache Ignite Calcite Module
+--------------------------
+
+Apache Ignite Calcite module provides experimental Apache Calcite based query engine.
+
+To enable Calcite module when starting a standalone node, move 'optional/ignite-calcite' folder to
+'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will
+be added to classpath in this case.
+
+Note: At now some logic from ignite-indexing module is reused, this means ignite-indexing module also
+has to be present at classpath.
+
+Importing Calcite Module In Maven Project
+---------------------------------------
+
+If you are using Maven to manage dependencies of your project, you can add Calcite module
+dependency like this (replace '${ignite.version}' with actual Apache Ignite version you are
+interested in):
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+ http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ ...
+ <dependencies>
+ ...
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-calcite</artifactId>
+ <version>${ignite.version}</version>
+ </dependency>
+ ...
+ </dependencies>
+ ...
+</project>
diff --git a/modules/calcite/pom.xml b/modules/calcite/pom.xml
new file mode 100644
index 0000000..d654edb
--- /dev/null
+++ b/modules/calcite/pom.xml
@@ -0,0 +1,112 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+ 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.
+-->
+
+<!--
+ POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-parent</artifactId>
+ <version>1</version>
+ <relativePath>../../parent</relativePath>
+ </parent>
+
+ <artifactId>ignite-calcite</artifactId>
+ <version>2.8.0-SNAPSHOT</version>
+ <url>http://ignite.apache.org</url>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <!--
+ At now the new calcite engine reuses some logic
+ and doesn't work without "old" indexing module.
+ -->
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-indexing</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <version>${calcite.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-core</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.springframework</groupId>
+ <artifactId>spring-core</artifactId>
+ <version>${spring.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <version>${slf4j.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.springframework</groupId>
+ <artifactId>spring-beans</artifactId>
+ <version>${spring.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.springframework</groupId>
+ <artifactId>spring-context</artifactId>
+ <version>${spring.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+ <plugin>
+ <groupId>org.apache.felix</groupId>
+ <artifactId>maven-bundle-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java b/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java
new file mode 100644
index 0000000..edae3cc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.interpreter;
+
+import org.apache.calcite.DataContext;
+
+/**
+ *
+ */
+public class InterpreterUtils {
+ public static Context createContext(DataContext ctx) {
+ return new Context(ctx);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java b/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
new file mode 100644
index 0000000..a1874dc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.plan.volcano;
+
+import org.apache.calcite.plan.RelTraitSet;
+
+/**
+ *
+ */
+public class VolcanoUtils {
+ public static RelSubset subset(RelSubset subset, RelTraitSet traits) {
+ return subset.set.getOrCreateSubset(subset.getCluster(), traits.simplify());
+ }
+
+ public static VolcanoPlanner impatient(VolcanoPlanner planner) {
+ planner.impatient = true;
+
+ return planner;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
new file mode 100644
index 0000000..86a9a2e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
@@ -0,0 +1,167 @@
+/*
+ * 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;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.function.BiFunction;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.sql.fun.SqlLibrary;
+import org.apache.calcite.sql.fun.SqlLibraryOperatorTableFactory;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.calcite.cluster.MappingServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.DistributedExecution;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Query;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryExecution;
+import org.apache.ignite.internal.processors.query.calcite.schema.CalciteSchemaHolder;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
+import org.apache.ignite.resources.LoggerResource;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class CalciteQueryProcessor implements QueryEngine {
+ /** */
+ private final CalciteSchemaHolder schemaHolder = new CalciteSchemaHolder();
+
+ /** */
+ private final FrameworkConfig config;
+
+ /** */
+ private IgniteLogger log;
+
+ /** */
+ private GridKernalContext kernalContext;
+
+ public CalciteQueryProcessor() {
+ config = Frameworks.newConfigBuilder()
+ .parserConfig(SqlParser.configBuilder()
+ // Lexical configuration defines how identifiers are quoted, whether they are converted to upper or lower
+ // case when they are read, and whether identifiers are matched case-sensitively.
+ .setLex(Lex.MYSQL)
+ .build())
+ // Dialects support.
+ .operatorTable(SqlLibraryOperatorTableFactory.INSTANCE
+ .getOperatorTable(
+ SqlLibrary.STANDARD,
+ SqlLibrary.MYSQL))
+ // Context provides a way to store data within the planner session that can be accessed in planner rules.
+ .context(Contexts.of(this))
+ // Custom cost factory to use during optimization
+ .costFactory(null)
+ .typeSystem(IgniteTypeSystem.DEFAULT)
+ .build();
+ }
+
+ /**
+ * @param log Logger.
+ */
+ @LoggerResource
+ public void setLogger(IgniteLogger log) {
+ this.log = log;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void start(@NotNull GridKernalContext ctx) {
+ kernalContext = ctx;
+
+ GridInternalSubscriptionProcessor prc = ctx.internalSubscriptionProcessor();
+
+ if (prc != null) // Stubbed context doesn't have such processor
+ prc.registerSchemaChangeListener(schemaHolder);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void stop() {
+ }
+
+ @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String query, Object... params) throws IgniteSQLException {
+ PlannerContext context = context(Commons.convert(ctx), query, params, this::buildContext);
+ QueryExecution execution = prepare(context);
+ FieldsQueryCursor<List<?>> cur = execution.execute();
+ return Collections.singletonList(cur);
+ }
+
+ public FrameworkConfig config() {
+ return config;
+ }
+
+ public IgniteLogger log() {
+ return log;
+ }
+
+ /** */
+ public IgnitePlanner planner(RelTraitDef[] traitDefs, PlannerContext ctx0) {
+ FrameworkConfig cfg = Frameworks.newConfigBuilder(config())
+ .defaultSchema(ctx0.schema())
+ .traitDefs(traitDefs)
+ .context(ctx0)
+ .build();
+
+ return new IgnitePlanner(cfg);
+ }
+
+ /**
+ * @param ctx External context.
+ * @param query Query string.
+ * @param params Query parameters.
+ * @return Query execution context.
+ */
+ PlannerContext context(@NotNull Context ctx, String query, Object[] params, BiFunction<Context, Query, PlannerContext> clo) { // Package private visibility for tests.
+ return clo.apply(Contexts.chain(ctx, config.getContext()), new Query(query, params));
+ }
+
+ private PlannerContext buildContext(@NotNull Context parent, @NotNull Query query) {
+ return PlannerContext.builder()
+ .logger(log)
+ .kernalContext(kernalContext)
+ .queryProcessor(this)
+ .parentContext(parent)
+ .query(query)
+ .schema(schemaHolder.schema())
+ .topologyVersion(readyAffinityVersion())
+ .mappingService(new MappingServiceImpl(kernalContext))
+ .build();
+ }
+
+ private QueryExecution prepare(PlannerContext ctx) {
+ return new DistributedExecution(ctx);
+ }
+
+ private AffinityTopologyVersion readyAffinityVersion() {
+ return kernalContext.cache().context().exchange().readyAffinityVersion();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
new file mode 100644
index 0000000..3e91219
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.cluster;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping.DEDUPLICATED;
+
+/**
+ *
+ */
+public class MappingServiceImpl implements MappingService {
+ private final GridKernalContext ctx;
+
+ public MappingServiceImpl(GridKernalContext ctx) {
+ this.ctx = ctx;
+ }
+
+ @Override public NodesMapping local() {
+ return new NodesMapping(Collections.singletonList(ctx.discovery().localNode().id()), null, DEDUPLICATED);
+ }
+
+ @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+ List<ClusterNode> nodes = ctx.discovery().discoCache(topVer).serverNodes();
+
+ return new NodesMapping(Commons.transform(nodes, ClusterNode::id), null, DEDUPLICATED);
+ }
+
+ @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+ GridCacheContext<?,?> cctx = ctx.cache().context().cacheContext(cacheId);
+
+ return cctx.isReplicated() ? replicatedLocation(cctx, topVer) : partitionedLocation(cctx, topVer);
+ }
+
+ private NodesMapping partitionedLocation(GridCacheContext<?,?> cctx, AffinityTopologyVersion topVer) {
+ byte flags = NodesMapping.HAS_PARTITIONED_CACHES;
+
+ List<List<ClusterNode>> assignments = cctx.affinity().assignments(topVer);
+ List<List<UUID>> res;
+
+ if (cctx.config().getWriteSynchronizationMode() == CacheWriteSynchronizationMode.PRIMARY_SYNC) {
+ res = new ArrayList<>(assignments.size());
+
+ for (List<ClusterNode> partNodes : assignments)
+ res.add(F.isEmpty(partNodes) ? Collections.emptyList() : Collections.singletonList(F.first(partNodes).id()));
+ }
+ else if (!cctx.topology().rebalanceFinished(topVer)) {
+ res = new ArrayList<>(assignments.size());
+
+ flags |= NodesMapping.HAS_MOVING_PARTITIONS;
+
+ for (int part = 0; part < assignments.size(); part++) {
+ List<ClusterNode> partNodes = assignments.get(part);
+ List<UUID> partIds = new ArrayList<>(partNodes.size());
+
+ for (ClusterNode node : partNodes) {
+ if (cctx.topology().partitionState(node.id(), part) == GridDhtPartitionState.OWNING)
+ partIds.add(node.id());
+ }
+
+ res.add(partIds);
+ }
+ }
+ else
+ res = Commons.transform(assignments, nodes -> Commons.transform(nodes, ClusterNode::id));
+
+ return new NodesMapping(null, res, flags);
+ }
+
+ private NodesMapping replicatedLocation(GridCacheContext<?,?> cctx, AffinityTopologyVersion topVer) {
+ byte flags = NodesMapping.HAS_REPLICATED_CACHES;
+
+ if (cctx.config().getNodeFilter() != null)
+ flags |= NodesMapping.PARTIALLY_REPLICATED;
+
+ GridDhtPartitionTopology topology = cctx.topology();
+
+ List<ClusterNode> nodes = cctx.discovery().discoCache(topVer).cacheGroupAffinityNodes(cctx.cacheId());
+ List<UUID> res;
+
+ if (!topology.rebalanceFinished(topVer)) {
+ flags |= NodesMapping.PARTIALLY_REPLICATED;
+
+ res = new ArrayList<>(nodes.size());
+
+ int parts = topology.partitions();
+
+ for (ClusterNode node : nodes) {
+ if (isOwner(node.id(), topology, parts))
+ res.add(node.id());
+ }
+ }
+ else
+ res = Commons.transform(nodes, ClusterNode::id);
+
+ return new NodesMapping(res, null, flags);
+ }
+
+ private boolean isOwner(UUID nodeId, GridDhtPartitionTopology topology, int parts) {
+ for (int p = 0; p < parts; p++) {
+ if (topology.partitionState(nodeId, p) != GridDhtPartitionState.OWNING)
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
new file mode 100644
index 0000000..ecbfef2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class EndMarker implements Serializable {
+ public static final EndMarker INSTANCE = new EndMarker();
+
+ private EndMarker(){}
+
+ private Object readResolve() throws ObjectStreamException {
+ return INSTANCE;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
new file mode 100644
index 0000000..803417c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+
+/**
+ *
+ */
+public interface ExchangeProcessor {
+ int BATCH_SIZE = 200;
+ int PER_NODE_BATCH_COUNT = 10;
+
+ <T> Outbox<T> register(Outbox<T> outbox);
+ <T> void unregister(Outbox<T> outbox);
+ <T> Inbox<T> register(Inbox<T> inbox);
+ <T> void unregister(Inbox<T> inbox);
+ void send(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId, List<?> rows);
+ void acknowledge(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
new file mode 100644
index 0000000..bd515a5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exec.SingleNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.Sink;
+import org.apache.ignite.internal.processors.query.calcite.exec.Source;
+
+/**
+ * TODO
+ */
+public class Inbox<T> implements SingleNode<T> {
+ private final GridCacheVersion queryId;
+ private final long exchangeId;
+
+ private Sink<T> target;
+ private Collection<UUID> sources;
+ private Comparator<T> comparator;
+ private ExchangeProcessor srvc;
+
+ public Inbox(GridCacheVersion queryId, long exchangeId) {
+ this.queryId = queryId;
+ this.exchangeId = exchangeId;
+ }
+
+ public void bind(Sink<T> target, Collection<UUID> sources, Comparator<T> comparator) {
+ this.target = target;
+ this.sources = sources;
+ this.comparator = comparator;
+ }
+
+ void init(ExchangeProcessor srvc) {
+ this.srvc = srvc;
+ }
+
+ @Override public void signal() {
+ // No-op.
+ }
+
+ @Override public void sources(List<Source> sources) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Sink<T> sink(int idx) {
+ throw new UnsupportedOperationException();
+ }
+
+ public void push(UUID source, int batchId, List<?> rows) {
+
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
new file mode 100644
index 0000000..f118330
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exec.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.SingleNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.Sink;
+import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T> {
+ private final Map<UUID, Destination> perNode = new HashMap<>();
+
+ private final GridCacheVersion queryId;
+ private final long exchangeId;
+ private final Collection<UUID> targets;
+ private final DestinationFunction function;
+
+ private ExchangeProcessor srvc;
+
+ public Outbox(GridCacheVersion queryId, long exchangeId, Collection<UUID> targets, DestinationFunction function) {
+ super(Sink.noOp());
+ this.queryId = queryId;
+ this.exchangeId = exchangeId;
+
+ this.targets = targets;
+ this.function = function;
+ }
+
+ public void init(ExchangeProcessor srvc) {
+ this.srvc = srvc;
+
+ srvc.register(this);
+
+ signal();
+ }
+
+ public void acknowledge(UUID nodeId, int batchId) {
+ perNode.get(nodeId).acknowledge(batchId);
+ }
+
+ @Override public Sink<T> sink(int idx) {
+ if (idx != 0)
+ throw new IndexOutOfBoundsException();
+
+ return this;
+ }
+
+ @Override public boolean push(T row) {
+ List<UUID> nodes = function.destination(row);
+
+ if (F.isEmpty(nodes))
+ return true;
+
+ List<Destination> destinations = new ArrayList<>(nodes.size());
+
+ for (UUID node : nodes) {
+ Destination dest = perNode.computeIfAbsent(node, Destination::new);
+
+ if (!dest.ready()) {
+ dest.needSignal();
+
+ return false;
+ }
+
+ destinations.add(dest);
+ }
+
+ for (Destination dest : destinations)
+ dest.add(row);
+
+ return true;
+ }
+
+ @Override public void end() {
+ for (UUID node : targets)
+ perNode.computeIfAbsent(node, Destination::new).end();
+
+ srvc.unregister(this);
+ }
+
+ private final class Destination {
+ private final UUID nodeId;
+
+ private int hwm = -1;
+ private int lwm = -1;
+
+ private ArrayList<Object> curr = new ArrayList<>(ExchangeProcessor.BATCH_SIZE + 1); // extra space for end marker;
+
+ private boolean needSignal;
+
+ private Destination(UUID nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ public void add(T row) {
+ if (curr.size() == ExchangeProcessor.BATCH_SIZE) {
+ assert ready() && srvc != null;
+
+ srvc.send(queryId, exchangeId, nodeId, ++hwm, curr);
+
+ curr = new ArrayList<>(ExchangeProcessor.BATCH_SIZE);
+ }
+
+ curr.add(row);
+ }
+
+ public void end() {
+ curr.add(EndMarker.INSTANCE);
+
+ assert srvc != null;
+
+ srvc.send(queryId, exchangeId, nodeId, hwm, curr);
+
+ curr = null;
+ hwm = Integer.MAX_VALUE;
+ }
+
+ boolean ready() {
+ return hwm - lwm < ExchangeProcessor.PER_NODE_BATCH_COUNT || curr.size() < ExchangeProcessor.BATCH_SIZE;
+ }
+
+ void acknowledge(int id) {
+ if (lwm < id) {
+ lwm = id;
+
+ if (needSignal) {
+ needSignal = false;
+
+ signal();
+ }
+ }
+ }
+
+ public void needSignal() {
+ needSignal = true;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
new file mode 100644
index 0000000..66ba475
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ *
+ */
+public abstract class AbstractNode<T> implements Node<T> {
+ protected final Sink<T> target;
+ protected List<Source> sources;
+
+ protected AbstractNode(Sink<T> target) {
+ this.target = target;
+ }
+
+ @Override public void sources(List<Source> sources) {
+ this.sources = Collections.unmodifiableList(sources);
+ }
+
+ public void signal(int idx) {
+ sources.get(idx).signal();
+ }
+
+ @Override public void signal() {
+ sources.forEach(Source::signal);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
new file mode 100644
index 0000000..100f21f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.ArrayDeque;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+
+/**
+ *
+ */
+public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]> {
+ private static final int DEFAULT_BUFFER_SIZE = 1000;
+ private static final Object[] END = new Object[0];
+
+ private ArrayDeque<Object[]> buff;
+
+ public ConsumerNode() {
+ super(Sink.noOp());
+
+ buff = new ArrayDeque<>(DEFAULT_BUFFER_SIZE);
+ }
+
+ @Override public Sink<Object[]> sink(int idx) {
+ if (idx != 0)
+ throw new IndexOutOfBoundsException();
+
+ return this;
+ }
+
+ @Override public boolean push(Object[] row) {
+ if (buff.size() == DEFAULT_BUFFER_SIZE)
+ return false;
+
+ buff.add(row);
+
+ return true;
+ }
+
+ @Override public void end() {
+ buff.add(END);
+ }
+
+ @Override public boolean hasNext() {
+ if (buff.isEmpty())
+ signal();
+
+ return buff.peek() != END;
+ }
+
+ @Override public Object[] next() {
+ if (buff.isEmpty())
+ signal();
+
+ if(!hasNext())
+ throw new NoSuchElementException();
+
+ return Objects.requireNonNull(buff.poll());
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java
new file mode 100644
index 0000000..3075ee4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.function.Predicate;
+
+/**
+ *
+ */
+public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
+ private final Predicate<Object[]> predicate;
+
+ public FilterNode(Sink<Object[]> target, Predicate<Object[]> predicate) {
+ super(target);
+
+ this.predicate = predicate;
+ }
+
+ @Override public Sink<Object[]> sink(int idx) {
+ if (idx != 0)
+ throw new IndexOutOfBoundsException();
+
+ return this;
+ }
+
+ @Override public boolean push(Object[] row) {
+ return !predicate.test(row) || target.push(row);
+ }
+
+ @Override public void end() {
+ target.end();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
new file mode 100644
index 0000000..cc8ed64
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
@@ -0,0 +1,174 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.UUID;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exchange.Outbox;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
+import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunctionFactory;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+import static org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue.PLANNER_CONTEXT;
+import static org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue.QUERY_ID;
+
+/**
+ *
+ */
+public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<IgniteRel, Node<Object[]>> {
+ private final PlannerContext ctx;
+ private final DataContext root;
+ private final ScalarFactory factory;
+ private Deque<Sink<Object[]>> stack;
+
+ public Implementor(DataContext root) {
+ this.root = root;
+
+ ctx = PLANNER_CONTEXT.get(root);
+ factory = new ScalarFactory(new RexBuilder(ctx.typeFactory()));
+ stack = new ArrayDeque<>();
+ }
+
+ @Override public Node<Object[]> visit(IgniteSender rel) {
+ assert stack.isEmpty();
+
+ GridCacheVersion id = QUERY_ID.get(root);
+ long exchangeId = rel.target().exchangeId();
+ NodesMapping mapping = rel.target().mapping();
+ List<UUID> targets = mapping.nodes();
+ IgniteDistribution distribution = rel.target().distribution();
+ DestinationFunctionFactory destFactory = distribution.destinationFunctionFactory();
+ DestinationFunction function = destFactory.create(ctx, mapping, ImmutableIntList.copyOf(distribution.getKeys()));
+
+ Outbox<Object[]> res = new Outbox<>(id, exchangeId, targets, function);
+
+ stack.push(res.sink());
+
+ res.source(source(rel.getInput()));
+
+ return res;
+ }
+
+ @Override public Node<Object[]> visit(IgniteFilter rel) {
+ assert !stack.isEmpty();
+
+ FilterNode res = new FilterNode(stack.pop(), factory.filterPredicate(root, rel.getCondition(), rel.getRowType()));
+
+ stack.push(res.sink());
+
+ res.source(source(rel.getInput()));
+
+ return res;
+ }
+
+ @Override public Node<Object[]> visit(IgniteProject rel) {
+ assert !stack.isEmpty();
+
+ ProjectNode res = new ProjectNode(stack.pop(), factory.projectExpression(root, rel.getProjects(), rel.getInput().getRowType()));
+
+ stack.push(res.sink());
+
+ res.source(source(rel.getInput()));
+
+ return res;
+ }
+
+ @Override public Node<Object[]> visit(IgniteJoin rel) {
+ assert !stack.isEmpty();
+
+ JoinNode res = new JoinNode(stack.pop(), factory.joinExpression(root, rel.getCondition(), rel.getLeft().getRowType(), rel.getRight().getRowType()));
+
+ stack.push(res.sink(1));
+ stack.push(res.sink(0));
+
+ res.sources(sources(rel.getInputs()));
+
+ return res;
+ }
+
+ @Override public Node<Object[]> visit(IgniteTableScan rel) {
+ assert !stack.isEmpty();
+
+ Iterable<Object[]> source = rel.getTable().unwrap(ScannableTable.class).scan(root);
+
+ return new ScanNode(stack.pop(), source);
+ }
+
+ @Override public Node<Object[]> visit(IgniteReceiver rel) {
+ throw new AssertionError(); // TODO
+ }
+
+ @Override public Node<Object[]> visit(IgniteExchange rel) {
+ throw new AssertionError();
+ }
+
+ @Override public Node<Object[]> visit(IgniteRel other) {
+ throw new AssertionError();
+ }
+
+ private Source source(RelNode rel) {
+ if (rel.getConvention() != IgniteConvention.INSTANCE)
+ throw new IllegalStateException("INTERPRETABLE is required.");
+
+ return ((IgniteRel) rel).accept(this);
+ }
+
+ private List<Source> sources(List<RelNode> rels) {
+ ArrayList<Source> res = new ArrayList<>(rels.size());
+
+ for (RelNode rel : rels) {
+ res.add(source(rel));
+ }
+
+ return res;
+ }
+
+ @Override public Node<Object[]> go(IgniteRel rel) {
+ if (rel instanceof IgniteSender)
+ return visit((IgniteSender) rel);
+
+ ConsumerNode res = new ConsumerNode();
+
+ stack.push(res.sink());
+
+ res.source(source(rel));
+
+ return res;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java
new file mode 100644
index 0000000..5188148
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.function.BiFunction;
+
+/**
+ *
+ */
+public class JoinNode extends AbstractNode<Object[]> {
+ private final BiFunction<Object[], Object[], Object[]> expression;
+ private final ArraySink<Object[]> left;
+ private final ArraySink<Object[]> right;
+
+ private int leftIdx;
+ private int rightIdx;
+ private boolean end;
+
+ public JoinNode(Sink<Object[]> target, BiFunction<Object[], Object[], Object[]> expression) {
+ super(target);
+ this.expression = expression;
+
+ left = new ArraySink<>();
+ right = new ArraySink<>();
+ }
+
+ @Override public Sink<Object[]> sink(int idx) {
+ switch (idx) {
+ case 0:
+ return left;
+ case 1:
+ return right;
+ default:
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @Override public void signal() {
+ if (end)
+ return;
+
+ if (left.end && right.end)
+ tryFlush();
+
+ assert sources != null && sources.size() == 2;
+
+ if (!left.end)
+ signal(0);
+ if (!right.end)
+ signal(1);
+ }
+
+ public void tryFlush() {
+ if (left.end && right.end) {
+ for (int i = leftIdx; i < left.size(); i++) {
+ for (int j = rightIdx; j < right.size(); j++) {
+ Object[] row = expression.apply(left.get(i), right.get(j));
+
+ if (row != null && !target.push(row)) {
+ leftIdx = i;
+ rightIdx = j;
+
+ return;
+ }
+ }
+ }
+
+ end = true;
+ target.end();
+ }
+ }
+
+ private final class ArraySink<T> extends ArrayList<T> implements Sink<T> {
+ private boolean end;
+
+ @Override public boolean push(T row) {
+ return add(row);
+ }
+
+ @Override public void end() {
+ end = true;
+
+ tryFlush();
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
new file mode 100644
index 0000000..2472e8e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.List;
+
+/**
+ *
+ */
+public interface Node<T> extends Source {
+ Sink<T> sink(int idx);
+ void sources(List<Source> sources);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
new file mode 100644
index 0000000..d7b7e57
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.function.Function;
+
+/**
+ *
+ */
+public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
+ private final Function<Object[], Object[]> projection;
+
+ public ProjectNode(Sink<Object[]> target, Function<Object[], Object[]> projection) {
+ super(target);
+
+ this.projection = projection;
+ }
+
+ @Override public Sink<Object[]> sink(int idx) {
+ if (idx != 0)
+ throw new IndexOutOfBoundsException();
+
+ return this;
+ }
+
+ @Override public boolean push(Object[] row) {
+ return target.push(projection.apply(row));
+ }
+
+ @Override public void end() {
+ target.end();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
new file mode 100644
index 0000000..e3ff512
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
@@ -0,0 +1,151 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import com.google.common.collect.ImmutableList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.InterpreterUtils;
+import org.apache.calcite.interpreter.JaninoRexCompiler;
+import org.apache.calcite.interpreter.Scalar;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ *
+ */
+public class ScalarFactory {
+ private final JaninoRexCompiler rexCompiler;
+ private final RexBuilder builder;
+
+ public ScalarFactory(RexBuilder builder) {
+ rexCompiler = new JaninoRexCompiler(builder);
+ this.builder = builder;
+ }
+
+ public <T> Predicate<T> filterPredicate(DataContext root, RexNode filter, RelDataType rowType) {
+ Scalar scalar = rexCompiler.compile(ImmutableList.of(filter), rowType);
+ Context ctx = InterpreterUtils.createContext(root);
+
+ return new FilterPredicate<>(ctx, scalar);
+ }
+
+ public <T> Function<T, T> projectExpression(DataContext root, List<RexNode> projects, RelDataType rowType) {
+ Scalar scalar = rexCompiler.compile(projects, rowType);
+ Context ctx = InterpreterUtils.createContext(root);
+ int count = projects.size();
+
+ return new ProjectExpression<>(ctx, scalar, count);
+ }
+
+ public <T> BiFunction<T, T, T> joinExpression(DataContext root, RexNode expression, RelDataType leftType, RelDataType rightType) {
+ RelDataType rowType = combinedType(leftType, rightType);
+
+ Scalar scalar = rexCompiler.compile(ImmutableList.of(expression), rowType);
+ Context ctx = InterpreterUtils.createContext(root);
+ ctx.values = new Object[rowType.getFieldCount()];
+
+ return new JoinExpression<>(ctx, scalar);
+ }
+
+ private RelDataType combinedType(RelDataType... types) {
+ RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(builder.getTypeFactory());
+
+ for (RelDataType type : types)
+ typeBuilder.addAll(type.getFieldList());
+
+ return typeBuilder.build();
+ }
+
+ private static class FilterPredicate<T> implements Predicate<T> {
+ private final Context ctx;
+ private final Scalar scalar;
+ private final Object[] vals;
+
+ private FilterPredicate(Context ctx, Scalar scalar) {
+ this.ctx = ctx;
+ this.scalar = scalar;
+
+ vals = new Object[1];
+ }
+
+ @Override public boolean test(T r) {
+ ctx.values = (Object[]) r;
+ scalar.execute(ctx, vals);
+ return (Boolean) vals[0];
+ }
+ }
+
+ private static class JoinExpression<T> implements BiFunction<T, T, T> {
+ private final Object[] vals;
+ private final Context ctx;
+ private final Scalar scalar;
+
+ private Object[] left0;
+
+ private JoinExpression(Context ctx, Scalar scalar) {
+ this.ctx = ctx;
+ this.scalar = scalar;
+
+ vals = new Object[1];
+ }
+
+ @Override public T apply(T left, T right) {
+ if (left0 != left) {
+ left0 = (Object[]) left;
+ System.arraycopy(left0, 0, ctx.values, 0, left0.length);
+ }
+
+ Object[] right0 = (Object[]) right;
+ System.arraycopy(right0, 0, ctx.values, left0.length, right0.length);
+
+ scalar.execute(ctx, vals);
+
+ if ((Boolean) vals[0])
+ return (T) Arrays.copyOf(ctx.values, ctx.values.length);
+
+ return null;
+ }
+ }
+
+ private static class ProjectExpression<T> implements Function<T, T> {
+ private final Context ctx;
+ private final Scalar scalar;
+ private final int count;
+
+ private ProjectExpression(Context ctx, Scalar scalar, int count) {
+ this.ctx = ctx;
+ this.scalar = scalar;
+ this.count = count;
+ }
+
+ @Override public T apply(T r) {
+ ctx.values = (Object[]) r;
+ Object[] res = new Object[count];
+ scalar.execute(ctx, res);
+
+ return (T) res;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
new file mode 100644
index 0000000..6acaa2b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ *
+ */
+public class ScanNode implements SingleNode<Object[]> {
+ private static final Object[] END = new Object[0];
+
+ /** */
+ private final Sink<Object[]> target;
+ private final Iterable<Object[]> source;
+
+ private Iterator<Object[]> it;
+ private Object[] row;
+
+ public ScanNode(Sink<Object[]> target, Iterable<Object[]> source) {
+ this.target = target;
+ this.source = source;
+ }
+
+ @Override public void signal() {
+ if (row == END)
+ return;
+
+ if (row != null && !target.push(row))
+ return;
+
+ row = null;
+
+ if (it == null)
+ it = source.iterator();
+
+ while (it.hasNext()) {
+ row = it.next();
+
+ if (!target.push(row))
+ return;
+ }
+
+ row = END;
+ target.end();
+ }
+
+ @Override public void sources(List<Source> sources) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Sink<Object[]> sink(int idx) {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
new file mode 100644
index 0000000..a0476a0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ *
+ */
+public interface SingleNode<T> extends Node<T> {
+ default Sink<T> sink() {
+ return Objects.requireNonNull(sink(0));
+ }
+
+ default void source(Source source) {
+ sources(Collections.singletonList(source));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
new file mode 100644
index 0000000..842a973
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+/**
+ *
+ */
+public interface Sink<T> {
+ Sink NO_OP = new Sink() {
+ @Override public boolean push(Object row) {
+ return true;
+ }
+
+ @Override public void end() {}
+ };
+
+ boolean push(T row);
+ void end();
+
+ @SuppressWarnings("unchecked")
+ static <T> Sink<T> noOp() {
+ return NO_OP;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
new file mode 100644
index 0000000..95262c7
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exec;
+
+/**
+ *
+ */
+public interface Source {
+ void signal();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
new file mode 100644
index 0000000..de15e4d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
+
+/**
+ *
+ */
+public class FragmentInfo {
+ private final NodesMapping mapping;
+ private final ImmutableList<Pair<IgniteReceiver, RelSource>> sources;
+
+ public FragmentInfo(Pair<IgniteReceiver, RelSource> source) {
+ this(ImmutableList.of(source), null);
+ }
+
+ public FragmentInfo(NodesMapping mapping) {
+ this(null, mapping);
+ }
+
+ public FragmentInfo(ImmutableList<Pair<IgniteReceiver, RelSource>> sources, NodesMapping mapping) {
+ this.sources = sources;
+ this.mapping = mapping;
+ }
+
+ public NodesMapping mapping() {
+ return mapping;
+ }
+
+ public ImmutableList<Pair<IgniteReceiver, RelSource>> sources() {
+ return sources;
+ }
+
+ public FragmentInfo merge(FragmentInfo other) throws LocationMappingException {
+ return new FragmentInfo(
+ merge(sources(), other.sources()),
+ merge(mapping(), other.mapping()));
+ }
+
+ private static NodesMapping merge(NodesMapping left, NodesMapping right) throws LocationMappingException {
+ if (left == null)
+ return right;
+ if (right == null)
+ return left;
+
+ return left.mergeWith(right);
+ }
+
+ private static <T> ImmutableList<T> merge(ImmutableList<T> left, ImmutableList<T> right) {
+ if (left == null)
+ return right;
+ if (right == null)
+ return left;
+
+ return ImmutableList.<T>builder().addAll(left).addAll(right).build();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDerivedDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDerivedDistribution.java
new file mode 100644
index 0000000..6858b13
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDerivedDistribution.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.AbstractConverter;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.plan.volcano.VolcanoUtils;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.DerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class IgniteMdDerivedDistribution implements MetadataHandler<DerivedDistribution> {
+ /** */
+ private static final ThreadLocal<Convention> REQUESTED_CONVENTION = ThreadLocal.withInitial(() -> Convention.NONE);
+
+ public static final RelMetadataProvider SOURCE =
+ ReflectiveRelMetadataProvider.reflectiveSource(
+ IgniteMethod.DERIVED_DISTRIBUTIONS.method(), new IgniteMdDerivedDistribution());
+
+ @Override public MetadataDef<DerivedDistribution> getDef() {
+ return DerivedDistribution.DEF;
+ }
+
+ public List<IgniteDistribution> deriveDistributions(AbstractConverter rel, RelMetadataQuery mq) {
+ return Collections.emptyList();
+ }
+
+ public List<IgniteDistribution> deriveDistributions(RelNode rel, RelMetadataQuery mq) {
+ return F.asList(IgniteMdDistribution._distribution(rel, mq));
+ }
+
+ public List<IgniteDistribution> deriveDistributions(IgniteRel rel, RelMetadataQuery mq) {
+ return F.asList(IgniteMdDistribution._distribution(rel, mq));
+ }
+
+ public List<IgniteDistribution> deriveDistributions(LogicalTableScan rel, RelMetadataQuery mq) {
+ return F.asList(IgniteMdDistribution._distribution(rel, mq));
+ }
+
+ public List<IgniteDistribution> deriveDistributions(LogicalValues rel, RelMetadataQuery mq) {
+ return F.asList(IgniteMdDistribution._distribution(rel, mq));
+ }
+
+ public List<IgniteDistribution> deriveDistributions(LogicalProject rel, RelMetadataQuery mq) {
+ Mappings.TargetMapping mapping =
+ Project.getPartialMapping(rel.getInput().getRowType().getFieldCount(), rel.getProjects());
+
+ return Commons.transform(_deriveDistributions(rel.getInput(), mq), i -> i.apply(mapping));
+ }
+
+ public List<IgniteDistribution> deriveDistributions(SingleRel rel, RelMetadataQuery mq) {
+ if (rel instanceof IgniteRel)
+ return deriveDistributions((IgniteRel)rel, mq);
+
+ return _deriveDistributions(rel.getInput(), mq);
+ }
+
+ public List<IgniteDistribution> deriveDistributions(RelSubset rel, RelMetadataQuery mq) {
+ rel = VolcanoUtils.subset(rel, rel.getTraitSet().replace(REQUESTED_CONVENTION.get()));
+
+ HashSet<IgniteDistribution> res = new HashSet<>();
+
+ for (RelNode rel0 : rel.getRels())
+ res.addAll(_deriveDistributions(rel0, mq));
+
+ if (F.isEmpty(res)) {
+ RelSubset newRel = VolcanoUtils.subset(rel, rel.getTraitSet().replace(Convention.NONE));
+
+ if (newRel != rel) {
+ for (RelNode rel0 : newRel.getRels())
+ res.addAll(_deriveDistributions(rel0, mq));
+ }
+ }
+
+ return new ArrayList<>(res);
+ }
+
+ public List<IgniteDistribution> deriveDistributions(HepRelVertex rel, RelMetadataQuery mq) {
+ return _deriveDistributions(rel.getCurrentRel(), mq);
+ }
+
+ public List<IgniteDistribution> deriveDistributions(LogicalFilter rel, RelMetadataQuery mq) {
+ return _deriveDistributions(rel.getInput(), mq);
+ }
+
+ public List<IgniteDistribution> deriveDistributions(LogicalJoin rel, RelMetadataQuery mq) {
+ List<IgniteDistribution> left = _deriveDistributions(rel.getLeft(), mq);
+ List<IgniteDistribution> right = _deriveDistributions(rel.getRight(), mq);
+
+ return Commons.transform(IgniteDistributions.suggestJoin(left, right, rel.analyzeCondition(), rel.getJoinType()),
+ IgniteDistributions.BiSuggestion::out);
+ }
+
+ private static List<IgniteDistribution> _deriveDistributions(RelNode rel, RelMetadataQuery mq) {
+ return RelMetadataQueryEx.wrap(mq).derivedDistributions(rel);
+ }
+
+ public static List<IgniteDistribution> deriveDistributions(RelNode rel, Convention convention, RelMetadataQuery mq) {
+ try {
+ REQUESTED_CONVENTION.set(convention);
+
+ return _deriveDistributions(rel, mq);
+ }
+ finally {
+ REQUESTED_CONVENTION.remove();
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
new file mode 100644
index 0000000..9673835
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import java.util.List;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Exchange;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static org.apache.calcite.rel.RelDistribution.Type.ANY;
+
+/**
+ *
+ */
+public class IgniteMdDistribution implements MetadataHandler<BuiltInMetadata.Distribution> {
+ public static final RelMetadataProvider SOURCE =
+ ReflectiveRelMetadataProvider.reflectiveSource(
+ BuiltInMethod.DISTRIBUTION.method, new IgniteMdDistribution());
+
+ @Override public MetadataDef<BuiltInMetadata.Distribution> getDef() {
+ return BuiltInMetadata.Distribution.DEF;
+ }
+
+ public IgniteDistribution distribution(RelNode rel, RelMetadataQuery mq) {
+ return DistributionTraitDef.INSTANCE.getDefault();
+ }
+
+ public IgniteDistribution distribution(Filter filter, RelMetadataQuery mq) {
+ return filter(mq, filter.getInput(), filter.getCondition());
+ }
+
+ public IgniteDistribution distribution(Project project, RelMetadataQuery mq) {
+ return project(mq, project.getInput(), project.getProjects());
+ }
+
+ public IgniteDistribution distribution(Join join, RelMetadataQuery mq) {
+ return join(mq, join.getLeft(), join.getRight(), join.analyzeCondition(), join.getJoinType());
+ }
+
+ public IgniteDistribution distribution(RelSubset rel, RelMetadataQuery mq) {
+ return rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+ }
+
+ public IgniteDistribution distribution(TableScan rel, RelMetadataQuery mq) {
+ return rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+ }
+
+ public IgniteDistribution distribution(Values values, RelMetadataQuery mq) {
+ return IgniteDistributions.broadcast();
+ }
+
+ public IgniteDistribution distribution(Exchange exchange, RelMetadataQuery mq) {
+ return (IgniteDistribution) exchange.distribution;
+ }
+
+ public IgniteDistribution distribution(HepRelVertex rel, RelMetadataQuery mq) {
+ return _distribution(rel.getCurrentRel(), mq);
+ }
+
+ public static IgniteDistribution project(RelMetadataQuery mq, RelNode input, List<? extends RexNode> projects) {
+ return project(input.getRowType(), _distribution(input, mq), projects);
+ }
+
+ public static IgniteDistribution project(RelDataType inType, IgniteDistribution inDistr, List<? extends RexNode> projects) {
+ return inDistr.apply(Project.getPartialMapping(inType.getFieldCount(), projects));
+ }
+
+ public static IgniteDistribution filter(RelMetadataQuery mq, RelNode input, RexNode condition) {
+ return _distribution(input, mq);
+ }
+
+ public static IgniteDistribution join(RelMetadataQuery mq, RelNode left, RelNode right, JoinInfo joinInfo, JoinRelType joinType) {
+ return join(_distribution(left, mq), _distribution(right, mq), joinInfo, joinType);
+ }
+
+ public static IgniteDistribution join(IgniteDistribution left, IgniteDistribution right, JoinInfo joinInfo, JoinRelType joinType) {
+ return F.first(IgniteDistributions.suggestJoin(left, right, joinInfo, joinType)).out();
+ }
+
+ public static IgniteDistribution _distribution(RelNode rel, RelMetadataQuery mq) {
+ IgniteDistribution distr = rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+
+ if (distr.getType() != ANY)
+ return distr;
+
+ return (IgniteDistribution) mq.distribution(rel);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
new file mode 100644
index 0000000..c5f941e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.FragmentMetadata;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Edge;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+/**
+ *
+ */
+public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
+ public static final RelMetadataProvider SOURCE =
+ ReflectiveRelMetadataProvider.reflectiveSource(
+ IgniteMethod.FRAGMENT_INFO.method(), new IgniteMdFragmentInfo());
+
+ @Override public MetadataDef<FragmentMetadata> getDef() {
+ return FragmentMetadata.DEF;
+ }
+
+ public FragmentInfo getFragmentInfo(RelNode rel, RelMetadataQuery mq) {
+ throw new AssertionError();
+ }
+
+ public FragmentInfo getFragmentInfo(RelSubset rel, RelMetadataQuery mq) {
+ throw new AssertionError();
+ }
+
+ public FragmentInfo getFragmentInfo(SingleRel rel, RelMetadataQuery mq) {
+ return fragmentInfo(rel.getInput(), mq);
+ }
+
+ public FragmentInfo getFragmentInfo(Join rel, RelMetadataQuery mq) {
+ mq = RelMetadataQueryEx.wrap(mq);
+
+ FragmentInfo left = fragmentInfo(rel.getLeft(), mq);
+ FragmentInfo right = fragmentInfo(rel.getRight(), mq);
+
+ try {
+ return left.merge(right);
+ }
+ catch (LocationMappingException e) {
+ // a replicated cache is cheaper to redistribute
+ if (!left.mapping().hasPartitionedCaches())
+ throw planningException(rel, e, true);
+ else if (!right.mapping().hasPartitionedCaches())
+ throw planningException(rel, e, false);
+
+ // both sub-trees have partitioned sources, less cost is better
+ RelOptCluster cluster = rel.getCluster();
+
+ RelOptCost leftCost = rel.getLeft().computeSelfCost(cluster.getPlanner(), mq);
+ RelOptCost rightCost = rel.getRight().computeSelfCost(cluster.getPlanner(), mq);
+
+ throw planningException(rel, e, leftCost.isLe(rightCost));
+ }
+ }
+
+ public FragmentInfo getFragmentInfo(IgniteReceiver rel, RelMetadataQuery mq) {
+ return new FragmentInfo(Pair.of(rel, rel.source()));
+ }
+
+ public FragmentInfo getFragmentInfo(IgniteTableScan rel, RelMetadataQuery mq) {
+ return rel.getTable().unwrap(IgniteTable.class).fragmentInfo(Commons.plannerContext(rel));
+ }
+
+ public static FragmentInfo fragmentInfo(RelNode rel, RelMetadataQuery mq) {
+ return RelMetadataQueryEx.wrap(mq).getFragmentLocation(rel);
+ }
+
+ private OptimisticPlanningException planningException(BiRel rel, Exception cause, boolean splitLeft) {
+ String msg = "Failed to calculate physical distribution";
+
+ if (splitLeft)
+ return new OptimisticPlanningException(msg, new Edge(rel, rel.getLeft(), 0), cause);
+
+ return new OptimisticPlanningException(msg, new Edge(rel, rel.getRight(), 1), cause);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java
new file mode 100644
index 0000000..5579a14
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
+import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+/**
+ *
+ */
+public class IgniteMetadata {
+ public static final RelMetadataProvider METADATA_PROVIDER =
+ ChainedRelMetadataProvider.of(
+ ImmutableList.of(
+ IgniteMdDerivedDistribution.SOURCE,
+ IgniteMdDistribution.SOURCE,
+ IgniteMdFragmentInfo.SOURCE,
+ DefaultRelMetadataProvider.INSTANCE));
+
+ public interface FragmentMetadata extends Metadata {
+ MetadataDef<FragmentMetadata> DEF = MetadataDef.of(FragmentMetadata.class,
+ FragmentMetadata.Handler.class, IgniteMethod.FRAGMENT_INFO.method());
+
+ /** Determines how the rows are distributed. */
+ FragmentInfo getFragmentInfo();
+
+ /** Handler API. */
+ interface Handler extends MetadataHandler<FragmentMetadata> {
+ FragmentInfo getFragmentInfo(RelNode r, RelMetadataQuery mq);
+ }
+ }
+
+ public interface DerivedDistribution extends Metadata {
+ MetadataDef<DerivedDistribution> DEF = MetadataDef.of(DerivedDistribution.class,
+ DerivedDistribution.Handler.class, IgniteMethod.DERIVED_DISTRIBUTIONS.method());
+
+ List<IgniteDistribution> deriveDistributions();
+
+ /** Handler API. */
+ interface Handler extends MetadataHandler<DerivedDistribution> {
+ List<IgniteDistribution> deriveDistributions(RelNode r, RelMetadataQuery mq);
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/LocationMappingException.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/LocationMappingException.java
new file mode 100644
index 0000000..e1d884a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/LocationMappingException.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+/**
+ *
+ */
+public class LocationMappingException extends Exception {
+ public LocationMappingException(String message) {
+ super(message);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
new file mode 100644
index 0000000..9472cd1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+
+/**
+ *
+ */
+public interface MappingService {
+ NodesMapping local(); // returns local node with single partition
+ NodesMapping random(AffinityTopologyVersion topVer); // returns random distribution, partitions count depends on nodes count
+ NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer); // returns cache distribution
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
new file mode 100644
index 0000000..b63e3e2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
@@ -0,0 +1,196 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+public class NodesMapping implements Serializable {
+ public static final byte HAS_MOVING_PARTITIONS = 1;
+ public static final byte HAS_REPLICATED_CACHES = 1 << 1;
+ public static final byte HAS_PARTITIONED_CACHES = 1 << 2;
+ public static final byte PARTIALLY_REPLICATED = 1 << 3;
+ public static final byte DEDUPLICATED = 1 << 4;
+
+ private final List<UUID> nodes;
+ private final List<List<UUID>> assignments;
+ private final byte flags;
+
+ public NodesMapping(List<UUID> nodes, List<List<UUID>> assignments, byte flags) {
+ this.nodes = nodes;
+ this.assignments = assignments;
+ this.flags = flags;
+ }
+
+ public List<UUID> nodes() {
+ return nodes;
+ }
+
+ public List<List<UUID>> assignments() {
+ return assignments;
+ }
+
+ public NodesMapping mergeWith(NodesMapping other) throws LocationMappingException {
+ byte flags = (byte) (this.flags | other.flags);
+
+ if ((flags & PARTIALLY_REPLICATED) == 0)
+ return new NodesMapping(U.firstNotNull(nodes, other.nodes), mergeAssignments(other, null), flags);
+
+ List<UUID> nodes;
+
+ if (this.nodes == null)
+ nodes = other.nodes;
+ else if (other.nodes == null)
+ nodes = this.nodes;
+ else
+ nodes = Commons.intersect(this.nodes, other.nodes);
+
+ if (nodes != null && nodes.isEmpty())
+ throw new LocationMappingException("Failed to map fragment to location.");
+
+ return new NodesMapping(nodes, mergeAssignments(other, nodes), flags);
+ }
+
+ public NodesMapping deduplicate() {
+ if (!excessive())
+ return this;
+
+ if (assignments == null) {
+ UUID node = nodes.get(ThreadLocalRandom.current().nextInt(nodes.size()));
+
+ return new NodesMapping(Collections.singletonList(node), null, (byte)(flags | DEDUPLICATED));
+ }
+
+ HashSet<UUID> nodes0 = new HashSet<>();
+ List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
+
+ for (List<UUID> partNodes : assignments) {
+ UUID node = F.first(partNodes);
+
+ if (node == null)
+ assignments0.add(Collections.emptyList());
+ else {
+ assignments0.add(Collections.singletonList(node));
+
+ nodes0.add(node);
+ }
+ }
+
+ return new NodesMapping(new ArrayList<>(nodes0), assignments0, (byte)(flags | DEDUPLICATED));
+ }
+
+ public int[] partitions(UUID node) {
+ if (assignments == null)
+ return null;
+
+ GridIntList parts = new GridIntList(assignments.size());
+
+ for (int i = 0; i < assignments.size(); i++) {
+ List<UUID> assignment = assignments.get(i);
+ if (Objects.equals(node, F.first(assignment)))
+ parts.add(i);
+ }
+
+ return parts.array();
+ }
+
+ public boolean excessive() {
+ return (flags & DEDUPLICATED) == 0;
+ }
+
+ public boolean hasMovingPartitions() {
+ return (flags & HAS_MOVING_PARTITIONS) == HAS_MOVING_PARTITIONS;
+ }
+
+ public boolean hasReplicatedCaches() {
+ return (flags & HAS_REPLICATED_CACHES) == HAS_REPLICATED_CACHES;
+ }
+
+ public boolean hasPartitionedCaches() {
+ return (flags & HAS_PARTITIONED_CACHES) == HAS_PARTITIONED_CACHES;
+ }
+
+ public boolean partiallyReplicated() {
+ return (flags & PARTIALLY_REPLICATED) == PARTIALLY_REPLICATED;
+ }
+
+ private List<List<UUID>> mergeAssignments(NodesMapping other, List<UUID> nodes) throws LocationMappingException {
+ byte flags = (byte) (this.flags | other.flags); List<List<UUID>> left = assignments, right = other.assignments;
+
+ if (left == null && right == null)
+ return null; // nothing to intersect;
+
+ if (left == null || right == null || (flags & HAS_MOVING_PARTITIONS) == 0) {
+ List<List<UUID>> assignments = U.firstNotNull(left, right);
+
+ if (nodes == null || (flags & PARTIALLY_REPLICATED) == 0)
+ return assignments;
+
+ List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
+ HashSet<UUID> nodesSet = new HashSet<>(nodes);
+
+ for (List<UUID> partNodes : assignments) {
+ List<UUID> partNodes0 = new ArrayList<>(partNodes.size());
+
+ for (UUID partNode : partNodes) {
+ if (nodesSet.contains(partNode))
+ partNodes0.add(partNode);
+ }
+
+ if (partNodes0.isEmpty())
+ throw new LocationMappingException("Failed to map fragment to location.");
+
+ assignments0.add(partNodes0);
+ }
+
+ return assignments0;
+ }
+
+ List<List<UUID>> assignments = new ArrayList<>(left.size());
+ HashSet<UUID> nodesSet = nodes != null ? new HashSet<>(nodes) : null;
+
+ for (int i = 0; i < left.size(); i++) {
+ List<UUID> leftNodes = left.get(i), partNodes = new ArrayList<>(leftNodes.size());
+ HashSet<UUID> rightNodesSet = new HashSet<>(right.get(i));
+
+ for (UUID partNode : leftNodes) {
+ if (rightNodesSet.contains(partNode) && (nodesSet == null || nodesSet.contains(partNode)))
+ partNodes.add(partNode);
+ }
+
+ if (partNodes.isEmpty())
+ throw new LocationMappingException("Failed to map fragment to location.");
+
+ assignments.add(partNodes);
+ }
+
+ return assignments;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
new file mode 100644
index 0000000..f0d3fe3
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import org.apache.ignite.internal.processors.query.calcite.splitter.Edge;
+
+/**
+ *
+ */
+public class OptimisticPlanningException extends RuntimeException{
+ private final Edge edge;
+
+ public OptimisticPlanningException(String message, Edge edge, Throwable cause) {
+ super(message, cause);
+ this.edge = edge;
+ }
+
+ public Edge edge() {
+ return edge;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
new file mode 100644
index 0000000..74d7682
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.metadata;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public class RelMetadataQueryEx extends RelMetadataQuery {
+ private static final RelMetadataQueryEx PROTO = new RelMetadataQueryEx();
+ public static final JaninoRelMetadataProvider PROVIDER = JaninoRelMetadataProvider.of(IgniteMetadata.METADATA_PROVIDER);
+
+ static {
+ PROVIDER.register(ImmutableList.of(
+ IgniteExchange.class,
+ IgniteReceiver.class,
+ IgniteSender.class,
+ IgniteFilter.class,
+ IgniteProject.class,
+ IgniteJoin.class,
+ IgniteTableScan.class));
+ }
+
+ private IgniteMetadata.FragmentMetadata.Handler sourceDistributionHandler;
+ private IgniteMetadata.DerivedDistribution.Handler derivedDistributionsHandler;
+
+ @SuppressWarnings("MethodOverridesStaticMethodOfSuperclass")
+ public static RelMetadataQueryEx instance() {
+ return new RelMetadataQueryEx(PROTO);
+ }
+
+ public static RelMetadataQueryEx wrap(@NotNull RelMetadataQuery mq) {
+ if (mq.getClass() == RelMetadataQueryEx.class)
+ return (RelMetadataQueryEx) mq;
+
+ return new RelMetadataQueryEx(mq);
+ }
+
+ private RelMetadataQueryEx(@NotNull RelMetadataQueryEx parent) {
+ super(PROVIDER, parent);
+
+ sourceDistributionHandler = parent.sourceDistributionHandler;
+ derivedDistributionsHandler = parent.derivedDistributionsHandler;
+ }
+
+ private RelMetadataQueryEx(@NotNull RelMetadataQuery parent) {
+ super(PROVIDER, parent);
+
+ sourceDistributionHandler = PROTO.sourceDistributionHandler;
+ derivedDistributionsHandler = PROTO.derivedDistributionsHandler;
+ }
+
+ private RelMetadataQueryEx() {
+ super(JaninoRelMetadataProvider.DEFAULT, RelMetadataQuery.EMPTY);
+
+ sourceDistributionHandler = initialHandler(IgniteMetadata.FragmentMetadata.Handler.class);
+ derivedDistributionsHandler = initialHandler(IgniteMetadata.DerivedDistribution.Handler.class);
+ }
+
+ public FragmentInfo getFragmentLocation(RelNode rel) {
+ for (;;) {
+ try {
+ return sourceDistributionHandler.getFragmentInfo(rel, this);
+ } catch (JaninoRelMetadataProvider.NoHandler e) {
+ sourceDistributionHandler = revise(e.relClass, IgniteMetadata.FragmentMetadata.DEF);
+ }
+ }
+ }
+
+ public List<IgniteDistribution> derivedDistributions(RelNode rel) {
+ for (;;) {
+ try {
+ return derivedDistributionsHandler.deriveDistributions(rel, this);
+ } catch (JaninoRelMetadataProvider.NoHandler e) {
+ derivedDistributionsHandler = revise(e.relClass, IgniteMetadata.DerivedDistribution.DEF);
+ }
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
new file mode 100644
index 0000000..0fe3dc5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.DataContext;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+
+/**
+ *
+ */
+public enum ContextValue {
+ QUERY_ID("_query_id", GridCacheVersion.class),
+ PLANNER_CONTEXT("_planner_context", PlannerContext.class);
+
+ private final String valueName;
+ private final Class type;
+
+ ContextValue(String valueName, Class type) {
+ this.valueName = valueName;
+ this.type = type;
+ }
+
+ public String valueName() {
+ return valueName;
+ }
+
+ public <T> T get(DataContext ctx) {
+ return (T) type.cast(ctx.get(valueName));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
new file mode 100644
index 0000000..3c2d14c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
@@ -0,0 +1,67 @@
+/*
+ * 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.Map;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
+
+/**
+ *
+ */
+public class DataContextImpl implements DataContext {
+ /** */
+ private final PlannerContext ctx;
+
+ /** */
+ private final Map<String, Object> params;
+
+ /**
+ * @param params Parameters.
+ * @param ctx Query context.
+ */
+ public DataContextImpl(Map<String, Object> params, PlannerContext ctx) {
+ this.params = params;
+ this.ctx = ctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override public SchemaPlus getRootSchema() {
+ return ctx.schema();
+ }
+
+ /** {@inheritDoc} */
+ @Override public JavaTypeFactory getTypeFactory() {
+ return ctx.typeFactory();
+ }
+
+ /** {@inheritDoc} */
+ @Override public QueryProvider getQueryProvider() {
+ return ctx.queryProvider();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object get(String name) {
+ if (ContextValue.PLANNER_CONTEXT.valueName().equals(name))
+ return ctx;
+
+ return params.get(name);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
new file mode 100644
index 0000000..b81961f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
@@ -0,0 +1,110 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelDistributions;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.util.ListFieldsQueryCursor;
+
+/**
+ *
+ */
+public class DistributedExecution implements QueryExecution {
+ /** */
+ private final PlannerContext ctx;
+
+ /**
+ * @param ctx Query context.
+ */
+ public DistributedExecution(PlannerContext ctx) {
+ this.ctx = ctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override public FieldsQueryCursor<List<?>> execute() {
+ CalciteQueryProcessor proc = ctx.queryProcessor();
+ Query query = ctx.query();
+
+ RelTraitDef[] traitDefs = {
+ RelDistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE,
+ RelCollationTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)) {
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getTraitSet()
+ .replace(relRoot.collation)
+ .replace(IgniteConvention.INSTANCE)
+ .replace(RelDistributions.ANY)
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ } catch (SqlParseException | ValidationException e) {
+ String msg = "Failed to parse query.";
+
+ proc.log().error(msg, e);
+
+ throw new IgniteSQLException(msg, IgniteQueryErrorCode.PARSING, e);
+ } catch (Exception e) {
+ String msg = "Failed to create query execution graph.";
+
+ proc.log().error(msg, e);
+
+ throw new IgniteSQLException(msg, IgniteQueryErrorCode.UNKNOWN, e);
+ }
+
+ // TODO physical plan.
+
+ return new ListFieldsQueryCursor<>(relRoot.rel.getRowType(), Linq4j.emptyEnumerable(), Arrays::asList);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
new file mode 100644
index 0000000..2f35a65
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
@@ -0,0 +1,436 @@
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import java.io.Reader;
+import java.util.List;
+import java.util.Properties;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.plan.volcano.VolcanoUtils;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
+import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexExecutor;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql2rel.RelDecorrelator;
+import org.apache.calcite.sql2rel.SqlRexConvertletTable;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.Graph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.GraphToRelConverter;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Splitter;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
+ private final SqlOperatorTable operatorTable;
+ private final ImmutableList<Program> programs;
+ private final FrameworkConfig frameworkConfig;
+ private final Context context;
+ private final CalciteConnectionConfig connectionConfig;
+ private final ImmutableList<RelTraitDef> traitDefs;
+ private final SqlParser.Config parserConfig;
+ private final SqlToRelConverter.Config sqlToRelConverterConfig;
+ private final SqlRexConvertletTable convertletTable;
+ private final RexExecutor executor;
+ private final SchemaPlus defaultSchema;
+ private final JavaTypeFactory typeFactory;
+
+ private boolean open;
+
+ private RelOptPlanner planner;
+ private RelMetadataProvider metadataProvider;
+ private SqlValidator validator;
+
+ /**
+ * @param config Framework config.
+ */
+ public IgnitePlanner(FrameworkConfig config) {
+ frameworkConfig = config;
+ defaultSchema = config.getDefaultSchema();
+ operatorTable = config.getOperatorTable();
+ programs = config.getPrograms();
+ parserConfig = config.getParserConfig();
+ sqlToRelConverterConfig = config.getSqlToRelConverterConfig();
+ traitDefs = config.getTraitDefs();
+ convertletTable = config.getConvertletTable();
+ executor = config.getExecutor();
+ context = config.getContext();
+ connectionConfig = connConfig();
+
+ RelDataTypeSystem typeSystem = connectionConfig
+ .typeSystem(RelDataTypeSystem.class, IgniteTypeSystem.DEFAULT);
+
+ typeFactory = new IgniteTypeFactory(typeSystem);
+
+ Commons.plannerContext(context).planner(this);
+ }
+
+ private CalciteConnectionConfig connConfig() {
+ CalciteConnectionConfig unwrapped = context.unwrap(CalciteConnectionConfig.class);
+ if (unwrapped != null)
+ return unwrapped;
+
+ Properties properties = new Properties();
+ properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
+ String.valueOf(parserConfig.caseSensitive()));
+ properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
+ String.valueOf(frameworkConfig.getParserConfig().conformance()));
+ return new CalciteConnectionConfigImpl(properties);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelTraitSet getEmptyTraitSet() {
+ return planner.emptyTraitSet();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ reset();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void reset() {
+ planner = null;
+ metadataProvider = null;
+ validator = null;
+
+ RelMetadataQuery.THREAD_PROVIDERS.remove();
+
+ open = false;
+ }
+
+ private void ready() {
+ if (!open) {
+ planner = VolcanoUtils.impatient(new VolcanoPlanner(frameworkConfig.getCostFactory(), context));
+ planner.setExecutor(executor);
+ metadataProvider = new CachingRelMetadataProvider(IgniteMetadata.METADATA_PROVIDER, planner);
+
+ validator = new IgniteSqlValidator(operatorTable(), createCatalogReader(), typeFactory, conformance());
+ validator.setIdentifierExpansion(true);
+
+ for (RelTraitDef def : traitDefs) {
+ planner.addRelTraitDef(def);
+ }
+
+ open = true;
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public SqlNode parse(Reader reader) throws SqlParseException {
+ return SqlParser.create(reader, parserConfig).parseStmt();
+ }
+
+ /** {@inheritDoc} */
+ @Override public SqlNode validate(SqlNode sqlNode) throws ValidationException {
+ ready();
+
+ try {
+ return validator.validate(sqlNode);
+ }
+ catch (RuntimeException e) {
+ throw new ValidationException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode) throws ValidationException {
+ ready();
+
+ SqlNode validatedNode = validate(sqlNode);
+ RelDataType type = validator.getValidatedNodeType(validatedNode);
+ return Pair.of(validatedNode, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelNode convert(SqlNode sql) {
+ return rel(sql).rel;
+ }
+
+ public RelNode convert(RelGraph graph) {
+ ready();
+
+ RelOptCluster cluster = createCluster(createRexBuilder());
+ RelBuilder relBuilder = createRelBuilder(cluster, createCatalogReader());
+
+ return new GraphToRelConverter(this, relBuilder, operatorTable).convert(graph);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelRoot rel(SqlNode sql) {
+ ready();
+
+ RexBuilder rexBuilder = createRexBuilder();
+ RelOptCluster cluster = createCluster(rexBuilder);
+ SqlToRelConverter.Config config = SqlToRelConverter.configBuilder()
+ .withConfig(sqlToRelConverterConfig)
+ .withTrimUnusedFields(false)
+ .withConvertTableAccess(false)
+ .build();
+ SqlToRelConverter sqlToRelConverter =
+ new SqlToRelConverter(this, validator, createCatalogReader(), cluster, convertletTable, config);
+ RelRoot root = sqlToRelConverter.convertQuery(sql, false, true);
+ root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true));
+ RelBuilder relBuilder = createRelBuilder(cluster, null);
+ root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel, relBuilder));
+ return root;
+ }
+
+ public QueryPlan plan(RelNode rel) {
+ ready();
+
+ if (rel.getConvention() != IgniteConvention.INSTANCE)
+ throw new IllegalArgumentException("Physical node is required.");
+
+ return new Splitter().go((IgniteRel) rel);
+ }
+
+ public Graph graph(RelNode rel) {
+ ready();
+
+ if (rel.getConvention() != IgniteConvention.INSTANCE)
+ throw new IllegalArgumentException("Physical node is required.");
+
+ return new RelToGraphConverter().go((IgniteRel) rel);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
+ ready();
+
+ SqlParser parser = SqlParser.create(queryString, parserConfig);
+ SqlNode sqlNode;
+ try {
+ sqlNode = parser.parseQuery();
+ }
+ catch (SqlParseException e) {
+ throw new RuntimeException("parse failed", e);
+ }
+
+ SqlConformance conformance = conformance();
+ CalciteCatalogReader catalogReader =
+ createCatalogReader().withSchemaPath(schemaPath);
+ SqlValidator validator = new IgniteSqlValidator(operatorTable(), catalogReader, typeFactory, conformance);
+ validator.setIdentifierExpansion(true);
+
+ RexBuilder rexBuilder = createRexBuilder();
+ RelOptCluster cluster = createCluster(rexBuilder);
+ SqlToRelConverter.Config config = SqlToRelConverter
+ .configBuilder()
+ .withConfig(sqlToRelConverterConfig)
+ .withTrimUnusedFields(false)
+ .withConvertTableAccess(false)
+ .build();
+ SqlToRelConverter sqlToRelConverter =
+ new SqlToRelConverter(this, validator,
+ catalogReader, cluster, convertletTable, config);
+
+ RelRoot root = sqlToRelConverter.convertQuery(sqlNode, true, false);
+ RelRoot root2 = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true));
+ RelBuilder relBuilder = createRelBuilder(cluster, null);
+ return root2.withRel(RelDecorrelator.decorrelateQuery(root.rel, relBuilder));
+ }
+
+ private RelOptCluster createCluster(RexBuilder rexBuilder) {
+ RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
+
+ cluster.setMetadataProvider(metadataProvider);
+ RelMetadataQuery.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(metadataProvider));
+
+ return cluster;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelNode transform(int programIdx, RelTraitSet targetTraits, RelNode rel) {
+ ready();
+
+ RelTraitSet toTraits = targetTraits.simplify();
+
+ rel.accept(new MetaDataProviderModifier(metadataProvider));
+
+ return programs.get(programIdx).run(planner, rel, toTraits, ImmutableList.of(), ImmutableList.of());
+ }
+
+ public RelNode transform(PlannerType plannerType, PlannerPhase plannerPhase, RelNode input, RelTraitSet targetTraits) {
+ ready();
+
+ RelTraitSet toTraits = targetTraits.simplify();
+
+ input.accept(new MetaDataProviderModifier(metadataProvider));
+
+ RelNode output;
+
+ switch (plannerType) {
+ case HEP:
+ final HepProgramBuilder programBuilder = new HepProgramBuilder();
+
+ for (RelOptRule rule : plannerPhase.getRules(Commons.plannerContext(context))) {
+ programBuilder.addRuleInstance(rule);
+ }
+
+ final HepPlanner hepPlanner =
+ new HepPlanner(programBuilder.build(), context, true, null, RelOptCostImpl.FACTORY);
+
+ hepPlanner.setRoot(input);
+
+ if (!input.getTraitSet().equals(targetTraits))
+ hepPlanner.changeTraits(input, toTraits);
+
+ output = hepPlanner.findBestExp();
+
+ break;
+ case VOLCANO:
+ Program program = Programs.of(plannerPhase.getRules(Commons.plannerContext(context)));
+
+ output = program.run(planner, input, toTraits,
+ ImmutableList.of(), ImmutableList.of());
+
+ break;
+ default:
+ throw new AssertionError("Unknown planner type: " + plannerType);
+ }
+
+ return output;
+ }
+
+ /** {@inheritDoc} */
+ @Override public JavaTypeFactory getTypeFactory() {
+ return typeFactory;
+ }
+
+ private SqlConformance conformance() {
+ return connectionConfig.conformance();
+ }
+
+ private SqlOperatorTable operatorTable() {
+ return operatorTable;
+ }
+
+ private RexBuilder createRexBuilder() {
+ return new RexBuilder(typeFactory);
+ }
+
+ private RelBuilder createRelBuilder(RelOptCluster cluster, RelOptSchema schema) {
+ return sqlToRelConverterConfig.getRelBuilderFactory().create(cluster, schema);
+ }
+
+ private CalciteCatalogReader createCatalogReader() {
+ SchemaPlus rootSchema = rootSchema(defaultSchema);
+
+ return new CalciteCatalogReader(
+ CalciteSchema.from(rootSchema),
+ CalciteSchema.from(defaultSchema).path(null),
+ typeFactory, connectionConfig);
+ }
+
+ private static SchemaPlus rootSchema(SchemaPlus schema) {
+ for (; ; ) {
+ if (schema.getParentSchema() == null) {
+ return schema;
+ }
+ schema = schema.getParentSchema();
+ }
+ }
+
+ /** */
+ private static class MetaDataProviderModifier extends RelShuttleImpl {
+ /** */
+ private final RelMetadataProvider metadataProvider;
+
+ /** */
+ private MetaDataProviderModifier(RelMetadataProvider metadataProvider) {
+ this.metadataProvider = metadataProvider;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelNode visit(TableScan scan) {
+ scan.getCluster().setMetadataProvider(metadataProvider);
+ return super.visit(scan);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelNode visit(TableFunctionScan scan) {
+ scan.getCluster().setMetadataProvider(metadataProvider);
+ return super.visit(scan);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelNode visit(LogicalValues values) {
+ values.getCluster().setMetadataProvider(metadataProvider);
+ return super.visit(values);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected RelNode visitChild(RelNode parent, int i, RelNode child) {
+ child.accept(this);
+ parent.getCluster().setMetadataProvider(metadataProvider);
+ return parent;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
new file mode 100644
index 0000000..74bd1df
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
@@ -0,0 +1,55 @@
+/*
+ * 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 org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+
+/** Validator. */
+public class IgniteSqlValidator extends SqlValidatorImpl {
+ public IgniteSqlValidator(SqlOperatorTable opTab,
+ CalciteCatalogReader catalogReader, JavaTypeFactory typeFactory,
+ SqlConformance conformance) {
+ super(opTab, catalogReader, typeFactory, conformance);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected RelDataType getLogicalSourceRowType(
+ RelDataType sourceRowType, SqlInsert insert) {
+ final RelDataType superType =
+ super.getLogicalSourceRowType(sourceRowType, insert);
+ return ((JavaTypeFactory) typeFactory).toSql(superType);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected RelDataType getLogicalTargetRowType(
+ RelDataType targetRowType, SqlInsert insert) {
+ final RelDataType superType =
+ super.getLogicalTargetRowType(targetRowType, insert);
+ return ((JavaTypeFactory) typeFactory).toSql(superType);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
new file mode 100644
index 0000000..352980f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.exchange.ExchangeProcessor;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+
+/**
+ *
+ */
+public final class PlannerContext implements Context {
+ private final Context parentContext;
+ private final Query query;
+ private final AffinityTopologyVersion topologyVersion;
+ private final SchemaPlus schema;
+ private final IgniteLogger logger;
+ private final GridKernalContext kernalContext;
+ private final CalciteQueryProcessor queryProcessor;
+ private final MappingService mappingService;
+ private final ExchangeProcessor exchangeProcessor;
+
+ private IgnitePlanner planner;
+
+ private PlannerContext(Context parentContext, Query query, AffinityTopologyVersion topologyVersion,
+ SchemaPlus schema, IgniteLogger logger, GridKernalContext kernalContext, CalciteQueryProcessor queryProcessor, MappingService mappingService,
+ ExchangeProcessor exchangeProcessor) {
+ this.parentContext = parentContext;
+ this.query = query;
+ this.topologyVersion = topologyVersion;
+ this.schema = schema;
+ this.logger = logger;
+ this.kernalContext = kernalContext;
+ this.queryProcessor = queryProcessor;
+ this.mappingService = mappingService;
+ this.exchangeProcessor = exchangeProcessor;
+ }
+
+ public Query query() {
+ return query;
+ }
+
+ public AffinityTopologyVersion topologyVersion() {
+ return topologyVersion;
+ }
+
+ public SchemaPlus schema() {
+ return schema;
+ }
+
+ public IgniteLogger logger() {
+ return logger;
+ }
+
+ public GridKernalContext kernalContext() {
+ return kernalContext;
+ }
+
+ public CalciteQueryProcessor queryProcessor() {
+ return queryProcessor;
+ }
+
+ void planner(IgnitePlanner planner) {
+ this.planner = planner;
+ }
+
+ public IgnitePlanner planner() {
+ return planner;
+ }
+
+ public MappingService mappingService() {
+ return mappingService;
+ }
+
+ public ExchangeProcessor exchangeProcessor() {
+ return exchangeProcessor;
+ }
+
+ // Helper methods
+
+ public JavaTypeFactory typeFactory() {
+ return planner.getTypeFactory();
+ }
+
+ public NodesMapping mapForLocal() {
+ return mappingService.local();
+ }
+
+ public NodesMapping mapForRandom(AffinityTopologyVersion topVer) {
+ return mappingService.random(topVer);
+ }
+
+ public NodesMapping mapForCache(int cacheId, AffinityTopologyVersion topVer) {
+ return mappingService.distributed(cacheId, topVer);
+ }
+
+ public QueryProvider queryProvider() {
+ return null; // TODO
+ }
+
+ @Override public <C> C unwrap(Class<C> aClass) {
+ if (aClass == getClass())
+ return aClass.cast(this);
+
+ return parentContext.unwrap(aClass);
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public static class Builder {
+ private Context parentContext;
+ private Query query;
+ private AffinityTopologyVersion topologyVersion;
+ private SchemaPlus schema;
+ private IgniteLogger logger;
+ private GridKernalContext kernalContext;
+ private CalciteQueryProcessor queryProcessor;
+ private MappingService mappingService;
+ private ExchangeProcessor exchangeProcessor;
+
+ public Builder parentContext(Context parentContext) {
+ this.parentContext = parentContext;
+ return this;
+ }
+
+ public Builder query(Query query) {
+ this.query = query;
+ return this;
+ }
+
+ public Builder topologyVersion(AffinityTopologyVersion topologyVersion) {
+ this.topologyVersion = topologyVersion;
+ return this;
+ }
+
+ public Builder schema(SchemaPlus schema) {
+ this.schema = schema;
+ return this;
+ }
+
+ public Builder logger(IgniteLogger logger) {
+ this.logger = logger;
+ return this;
+ }
+
+ public Builder kernalContext(GridKernalContext kernalContext) {
+ this.kernalContext = kernalContext;
+ return this;
+ }
+
+ public Builder queryProcessor(CalciteQueryProcessor queryProcessor) {
+ this.queryProcessor = queryProcessor;
+ return this;
+ }
+
+ public Builder mappingService(MappingService mappingService) {
+ this.mappingService = mappingService;
+ return this;
+ }
+
+ public Builder exchangeProcessor(ExchangeProcessor exchangeProcessor) {
+ this.exchangeProcessor = exchangeProcessor;
+ return this;
+ }
+
+ public PlannerContext build() {
+ return new PlannerContext(parentContext, query, topologyVersion, schema, logger, kernalContext, queryProcessor, mappingService, exchangeProcessor);
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
new file mode 100644
index 0000000..14ef394
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
@@ -0,0 +1,60 @@
+/*
+ * 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 org.apache.calcite.rel.rules.SubQueryRemoveRule;
+import org.apache.calcite.tools.RuleSet;
+import org.apache.calcite.tools.RuleSets;
+import org.apache.ignite.internal.processors.query.calcite.rule.FilterConverter;
+import org.apache.ignite.internal.processors.query.calcite.rule.JoinConverter;
+import org.apache.ignite.internal.processors.query.calcite.rule.ProjectConverter;
+import org.apache.ignite.internal.processors.query.calcite.rule.TableScanConverter;
+
+/**
+ *
+ */
+public enum PlannerPhase {
+ /** */
+ SUBQUERY_REWRITE("Sub-queries rewrites") {
+ @Override public RuleSet getRules(PlannerContext ctx) {
+ return RuleSets.ofList(
+ SubQueryRemoveRule.FILTER,
+ SubQueryRemoveRule.PROJECT,
+ SubQueryRemoveRule.JOIN);
+ }
+ },
+
+ /** */
+ OPTIMIZATION("Main optimization phase") {
+ @Override public RuleSet getRules(PlannerContext ctx) {
+ return RuleSets.ofList(
+ TableScanConverter.INSTANCE,
+ JoinConverter.INSTANCE,
+ ProjectConverter.INSTANCE,
+ FilterConverter.INSTANCE);
+ }
+ };
+
+ public final String description;
+
+ PlannerPhase(String description) {
+ this.description = description;
+ }
+
+ public abstract RuleSet getRules(PlannerContext ctx);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerType.java
new file mode 100644
index 0000000..cddc4d5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerType.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.prepare;
+
+/**
+ *
+ */
+public enum PlannerType {
+ HEP,
+ VOLCANO;
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java
new file mode 100644
index 0000000..d18c82c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java
@@ -0,0 +1,73 @@
+/*
+ * 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.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class Query {
+ private final String sql;
+ private final Object[] params;
+
+ public Query(String sql, Object[] params) {
+ this.sql = sql;
+ this.params = params;
+ }
+
+ public String sql() {
+ return sql;
+ }
+
+ public Object[] params() {
+ return params;
+ }
+
+ public Map<String, Object> params(Map<String, Object> stashed) {
+ Map<String, Object> res = new HashMap<>(stashed);
+ if (!F.isEmpty(params)) {
+ for (int i = 0; i < params.length; i++) {
+ res.put("?" + i, params[i]);
+ }
+ }
+ return res;
+ }
+
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (!(o instanceof Query))
+ return false;
+
+ Query query = (Query) o;
+
+ if (!sql.equals(query.sql))
+ return false;
+ return Arrays.equals(params, query.params);
+ }
+
+ @Override public int hashCode() {
+ int result = sql.hashCode();
+ result = 31 * result + Arrays.hashCode(params);
+ return result;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
new file mode 100644
index 0000000..bcfed58
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
@@ -0,0 +1,28 @@
+/*
+ * 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.ignite.cache.query.FieldsQueryCursor;
+
+/**
+ *
+ */
+public interface QueryExecution {
+ FieldsQueryCursor<List<?>> execute();
+}
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
new file mode 100644
index 0000000..a0f7878
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.AbstractConverter;
+
+/**
+ *
+ */
+public class IgniteConvention extends Convention.Impl {
+ public static final IgniteConvention INSTANCE = new IgniteConvention();
+
+ private IgniteConvention() {
+ super("IGNITE", IgniteRel.class);
+ }
+
+ @Override public ConventionTraitDef getTraitDef() {
+ return ConventionTraitDef.INSTANCE;
+ }
+
+ @Override public void register(RelOptPlanner planner) {
+ planner.addRule(AbstractConverter.ExpandConversionRule.INSTANCE);
+ }
+
+ @Override public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+ return toTraits.contains(INSTANCE) && fromTraits.contains(INSTANCE);
+ }
+}
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
new file mode 100644
index 0000000..9d01fbb
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Exchange;
+
+/**
+ *
+ */
+public class IgniteExchange extends Exchange implements IgniteRel {
+ public IgniteExchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RelDistribution distribution) {
+ super(cluster, traitSet, input, distribution);
+ }
+
+ @Override public Exchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution) {
+ return new IgniteExchange(getCluster(), traitSet, newInput, newDistribution);
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java
new file mode 100644
index 0000000..c408769
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ *
+ */
+public class IgniteFilter extends Filter implements IgniteRel {
+ public IgniteFilter(RelOptCluster cluster, RelTraitSet traits, RelNode input, RexNode condition) {
+ super(cluster, traits, input, condition);
+ }
+
+ @Override public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+ return new IgniteFilter(getCluster(), traitSet, input, condition);
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java
new file mode 100644
index 0000000..b1ad44b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import java.util.Set;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ *
+ */
+public class IgniteJoin extends Join implements IgniteRel {
+ public IgniteJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+ super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+ }
+
+ @Override public Join copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+ return new IgniteJoin(getCluster(), traitSet, left, right, condition, variablesSet, joinType);
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java
new file mode 100644
index 0000000..42d2af2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ *
+ */
+public class IgniteProject extends Project implements IgniteRel {
+ public IgniteProject(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
+ super(cluster, traits, input, projects, rowType);
+ }
+
+ @Override public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType) {
+ return new IgniteProject(getCluster(), traitSet, input, projects, rowType);
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+}
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
new file mode 100644
index 0000000..545c6e8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public class IgniteReceiver extends AbstractRelNode implements IgniteRel {
+ private RelSource source;
+
+ public IgniteReceiver(RelOptCluster cluster, RelTraitSet traits, RelDataType rowType, RelSource source) {
+ super(cluster, traits);
+
+ this.rowType = rowType;
+ this.source = source;
+ }
+
+ @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ return new IgniteReceiver(getCluster(), traitSet, rowType, source);
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+
+ public RelSource source() {
+ return source;
+ }
+
+ public void source(RelSource source) {
+ this.source = source;
+ }
+
+ public IgniteDistribution distribution() {
+ return getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java
new file mode 100644
index 0000000..8e1567a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ *
+ */
+public interface IgniteRel extends RelNode {
+ <T> T accept(IgniteRelVisitor<T> visitor);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
new file mode 100644
index 0000000..6c6ffa1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+/**
+ *
+ */
+public interface IgniteRelVisitor<T> {
+ T visit(IgniteSender rel);
+
+ T visit(IgniteFilter rel);
+
+ T visit(IgniteProject rel);
+
+ T visit(IgniteJoin rel);
+
+ T visit(IgniteTableScan rel);
+
+ T visit(IgniteReceiver rel);
+
+ T visit(IgniteExchange rel);
+
+ T visit(IgniteRel other);
+}
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
new file mode 100644
index 0000000..4763c38
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelTarget;
+
+/**
+ *
+ */
+public class IgniteSender extends SingleRel implements IgniteRel {
+ private RelTarget target;
+
+ public IgniteSender(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelTarget target) {
+ super(cluster, traits, input);
+
+ this.target = target;
+ }
+
+ public IgniteSender(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+ super(cluster, traits, input);
+ }
+
+ @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ return new IgniteSender(getCluster(), traitSet, sole(inputs), target);
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+
+ public RelTarget target() {
+ return target;
+ }
+
+ public void target(RelTarget target) {
+ this.target = target;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java
new file mode 100644
index 0000000..59af858
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+
+/**
+ *
+ */
+public class IgniteTableScan extends TableScan implements IgniteRel {
+ public IgniteTableScan(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+ super(cluster, traitSet, table);
+ }
+
+ @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ return this;
+ }
+
+ @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+ return visitor.visit(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java
new file mode 100644
index 0000000..fb073c9
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java
@@ -0,0 +1,27 @@
+/*
+ * 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.rel;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ *
+ */
+public interface RelOp<T extends RelNode, R> {
+ R go(T rel);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/FilterConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/FilterConverter.java
new file mode 100644
index 0000000..56587ba
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/FilterConverter.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rule;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class FilterConverter extends IgniteConverter {
+ public static final ConverterRule INSTANCE = new FilterConverter();
+
+ public FilterConverter() {
+ super(LogicalFilter.class, "FilterConverter");
+ }
+
+ @Override protected List<RelNode> convert0(RelNode rel) {
+ LogicalFilter filter = (LogicalFilter) rel;
+
+ RelNode input = convert(filter.getInput(), IgniteConvention.INSTANCE);
+
+ RelOptCluster cluster = rel.getCluster();
+ RelMetadataQuery mq = cluster.getMetadataQuery();
+
+ List<IgniteDistribution> distrs = IgniteMdDerivedDistribution.deriveDistributions(input, IgniteConvention.INSTANCE, mq);
+
+ return Commons.transform(distrs, d -> create(filter, input, d));
+ }
+
+ private static IgniteFilter create(LogicalFilter filter, RelNode input, IgniteDistribution distr) {
+ RelTraitSet traits = filter.getTraitSet()
+ .replace(distr)
+ .replace(IgniteConvention.INSTANCE);
+
+ return new IgniteFilter(filter.getCluster(), traits, convert(input, distr), filter.getCondition());
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/IgniteConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/IgniteConverter.java
new file mode 100644
index 0000000..4653fec
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/IgniteConverter.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rule;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public abstract class IgniteConverter extends ConverterRule {
+ protected IgniteConverter(Class<? extends RelNode> clazz, String descriptionPrefix) {
+ super(clazz, Convention.NONE, IgniteConvention.INSTANCE, descriptionPrefix);
+ }
+
+ @Override public void onMatch(RelOptRuleCall call) {
+ RelNode rel = call.rel(0);
+ if (rel.getTraitSet().contains(Convention.NONE)) {
+ List<RelNode> rels = convert0(rel);
+ if (F.isEmpty(rels))
+ return;
+
+ Map<RelNode, RelNode> equiv = ImmutableMap.of();
+
+ if (rels.size() > 1) {
+ equiv = new HashMap<>();
+
+ for (int i = 1; i < rels.size(); i++) {
+ equiv.put(rels.get(i), rel);
+ }
+ }
+
+ call.transformTo(F.first(rels), equiv);
+ }
+ }
+
+ @Override public RelNode convert(RelNode rel) {
+ List<RelNode> converted = convert0(rel);
+
+ if (converted.size() > 1) {
+ RelOptPlanner planner = rel.getCluster().getPlanner();
+
+ for (int i = 1; i < converted.size(); i++)
+ planner.ensureRegistered(converted.get(i), rel);
+ }
+
+ return F.first(converted);
+ }
+
+ protected abstract List<RelNode> convert0(RelNode rel);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverter.java
new file mode 100644
index 0000000..b10fe20
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverter.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rule;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class JoinConverter extends IgniteConverter {
+ public static final ConverterRule INSTANCE = new JoinConverter();
+
+ public JoinConverter() {
+ super(LogicalJoin.class, "JoinConverter");
+ }
+
+ @Override protected List<RelNode> convert0(RelNode rel) {
+ LogicalJoin join = (LogicalJoin) rel;
+
+ RelNode left = convert(join.getLeft(), IgniteConvention.INSTANCE);
+ RelNode right = convert(join.getRight(), IgniteConvention.INSTANCE);
+
+ RelOptCluster cluster = join.getCluster();
+ RelMetadataQuery mq = cluster.getMetadataQuery();
+
+ List<IgniteDistribution> leftTraits = IgniteMdDerivedDistribution.deriveDistributions(left, IgniteConvention.INSTANCE, mq);
+ List<IgniteDistribution> rightTraits = IgniteMdDerivedDistribution.deriveDistributions(left, IgniteConvention.INSTANCE, mq);
+
+ List<IgniteDistributions.BiSuggestion> suggestions = IgniteDistributions.suggestJoin(leftTraits, rightTraits, join.analyzeCondition(), join.getJoinType());
+
+ return Commons.transform(suggestions, s -> create(join, left, right, s));
+ }
+
+ private static RelNode create(LogicalJoin join, RelNode left, RelNode right, IgniteDistributions.BiSuggestion suggest) {
+ left = convert(left, suggest.left());
+ right = convert(right, suggest.right());
+
+ RelTraitSet traitSet = join.getTraitSet().replace(IgniteConvention.INSTANCE).replace(suggest.out());
+
+ return new IgniteJoin(join.getCluster(), traitSet, left, right, join.getCondition(), join.getVariablesSet(), join.getJoinType());
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ProjectConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ProjectConverter.java
new file mode 100644
index 0000000..2fe1147
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ProjectConverter.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rule;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class ProjectConverter extends IgniteConverter {
+ public static final ConverterRule INSTANCE = new ProjectConverter();
+
+ public ProjectConverter() {
+ super(LogicalProject.class, "ProjectConverter");
+ }
+
+ @Override protected List<RelNode> convert0(RelNode rel) {
+ LogicalProject project = (LogicalProject) rel;
+
+ RelNode input = convert(project.getInput(), IgniteConvention.INSTANCE);
+
+ RelOptCluster cluster = rel.getCluster();
+ RelMetadataQuery mq = cluster.getMetadataQuery();
+
+ List<IgniteDistribution> distrs = IgniteMdDerivedDistribution.deriveDistributions(input, IgniteConvention.INSTANCE, mq);
+
+ return Commons.transform(distrs, d -> create(project, input, d));
+ }
+
+ private static IgniteProject create(LogicalProject project, RelNode input, IgniteDistribution distr) {
+ RelTraitSet traits = project.getTraitSet()
+ .replace(IgniteMdDistribution.project(input.getRowType(), distr, project.getProjects()))
+ .replace(IgniteConvention.INSTANCE);
+
+ return new IgniteProject(project.getCluster(), traits, convert(input, distr), project.getProjects(), project.getRowType());
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableScanConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableScanConverter.java
new file mode 100644
index 0000000..2d5aea0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableScanConverter.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rule;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class TableScanConverter extends IgniteConverter {
+ public static final ConverterRule INSTANCE = new TableScanConverter();
+
+ public TableScanConverter() {
+ super(LogicalTableScan.class, "TableScanConverter");
+ }
+
+ @Override protected List<RelNode> convert0(RelNode rel) {
+ LogicalTableScan scan = (LogicalTableScan) rel;
+
+ RelTraitSet traitSet = scan.getTraitSet().replace(IgniteConvention.INSTANCE);
+ return F.asList(new IgniteTableScan(rel.getCluster(), traitSet, scan.getTable()));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
new file mode 100644
index 0000000..bad106f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
@@ -0,0 +1,69 @@
+/*
+ * 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.schema;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+
+/**
+ *
+ */
+public class CalciteSchemaHolder implements SchemaChangeListener {
+ private final Map<String, IgniteSchema> schemas = new HashMap<>();
+ private volatile SchemaPlus schema;
+
+ public void schema(SchemaPlus schema) {
+ this.schema = schema;
+ }
+
+ public SchemaPlus schema() {
+ return schema;
+ }
+
+ @Override public synchronized void onSchemaCreate(String schemaName) {
+ schemas.putIfAbsent(schemaName, new IgniteSchema(schemaName));
+ rebuild();
+ }
+
+ @Override public synchronized void onSchemaDrop(String schemaName) {
+ schemas.remove(schemaName);
+ rebuild();
+ }
+
+ @Override public synchronized void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
+ schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeCreate(typeDescriptor, cacheInfo);
+ rebuild();
+ }
+
+ @Override public synchronized void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
+ schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeDrop(typeDescriptor, cacheInfo);
+ rebuild();
+ }
+
+ private void rebuild() {
+ SchemaPlus schema = Frameworks.createRootSchema(false);
+ schemas.forEach(schema::add);
+ schema(schema.getSubSchema(QueryUtils.DFLT_SCHEMA));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
new file mode 100644
index 0000000..271c5f6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
@@ -0,0 +1,88 @@
+/*
+ * 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.schema;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class IgniteSchema extends AbstractSchema {
+ /** */
+ private final String schemaName;
+
+ /** */
+ private final Map<String, Table> tableMap = new ConcurrentHashMap<>();
+
+ public IgniteSchema(String schemaName) {
+ this.schemaName = schemaName;
+ }
+
+ public String getName() {
+ return schemaName;
+ }
+
+ @Override protected Map<String, Table> getTableMap() {
+ return Collections.unmodifiableMap(tableMap);
+ }
+
+ /**
+ * Callback method.
+ *
+ * @param typeDesc Query type descriptor.
+ * @param cacheInfo Cache info.
+ */
+ public void onSqlTypeCreate(GridQueryTypeDescriptor typeDesc, GridCacheContextInfo cacheInfo) {
+ Object identityKey = cacheInfo.config().getCacheMode() == CacheMode.PARTITIONED ?
+ cacheInfo.cacheContext().group().affinity().similarAffinityKey() : null;
+
+ addTable(new IgniteTable(typeDesc.tableName(), cacheInfo.name(), Commons.rowType(typeDesc), identityKey));
+ }
+
+ /**
+ * Callback method.
+ *
+ * @param typeDesc Query type descriptor.
+ * @param cacheInfo Cache info.
+ */
+ public void onSqlTypeDrop(GridQueryTypeDescriptor typeDesc, GridCacheContextInfo cacheInfo) {
+ removeTable(typeDesc.tableName());
+ }
+
+ /**
+ * @param table Table.
+ */
+ public void addTable(IgniteTable table) {
+ tableMap.put(table.tableName(), table);
+ }
+
+ /**
+ * @param tableName Table name.
+ */
+ public void removeTable(String tableName) {
+ tableMap.remove(tableName);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
new file mode 100644
index 0000000..56e9302
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
@@ -0,0 +1,137 @@
+/*
+ * 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.schema;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentInfo;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.AffinityFactory;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.type.RowType;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+/** */
+public class IgniteTable extends AbstractTable implements TranslatableTable, ScannableTable {
+ private final String tableName;
+ private final String cacheName;
+ private final RowType rowType;
+ private final Object identityKey;
+
+ public IgniteTable(String tableName, String cacheName, RowType rowType, Object identityKey) {
+ this.tableName = tableName;
+ this.cacheName = cacheName;
+ this.rowType = rowType;
+ this.identityKey = identityKey;
+ }
+
+ /**
+ * @return Table name;
+ */
+ public String tableName() {
+ return tableName;
+ }
+
+ /**
+ * @return Cache name.
+ */
+ public String cacheName() {
+ return cacheName;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+ return rowType.asRelDataType(typeFactory);
+ }
+
+ @Override public Statistic getStatistic() {
+ return new TableStatistics();
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
+ RelOptCluster cluster = context.getCluster();
+ RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
+ .replaceIf(DistributionTraitDef.INSTANCE, this::getDistribution);
+
+ return new LogicalTableScan(cluster, traitSet, relOptTable);
+ }
+
+ public IgniteDistribution getDistribution() {
+ Object key = identityKey();
+
+ if (key == null)
+ return IgniteDistributions.broadcast();
+
+ return IgniteDistributions.hash(rowType.distributionKeys(), new AffinityFactory(CU.cacheId(cacheName), key));
+ }
+
+ protected Object identityKey() {
+ return identityKey;
+ }
+
+ public FragmentInfo fragmentInfo(PlannerContext ctx) {
+ return new FragmentInfo(ctx.mapForCache(CU.cacheId(cacheName), ctx.topologyVersion()));
+ }
+
+ @Override public Enumerable<Object[]> scan(DataContext root) {
+ throw new AssertionError(); // TODO
+ }
+
+ private class TableStatistics implements Statistic {
+ @Override public Double getRowCount() {
+ return null;
+ }
+
+ @Override public boolean isKey(ImmutableBitSet columns) {
+ return false;
+ }
+
+ @Override public List<RelReferentialConstraint> getReferentialConstraints() {
+ return ImmutableList.of();
+ }
+
+ @Override public List<RelCollation> getCollations() {
+ return ImmutableList.of();
+ }
+
+ @Override public RelDistribution getDistribution() {
+ return IgniteTable.this.getDistribution();
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java
new file mode 100644
index 0000000..0427a5f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.GridIntList;
+
+/**
+ *
+ */
+public class Graph<T extends GraphNode> implements Serializable {
+ private final List<T> nodes = new ArrayList<>();
+ private final List<GridIntList> edges = new ArrayList<>();
+
+ public List<Ord<T>> nodes() {
+ return Ord.zip(nodes);
+ }
+
+ public List<GridIntList> edges() {
+ return Commons.transform(edges, GridIntList::copy);
+ }
+
+ public int addNode(int parentId, T node) {
+ int id = addNode(node);
+
+ addEdge(parentId, id);
+
+ return id;
+ }
+
+ public int addNode(T node) {
+ assert nodes.size() == edges.size();
+
+ int id = nodes.size();
+
+ nodes.add(node);
+ edges.add(new GridIntList());
+
+ return id;
+ }
+
+ public void addEdge(int parentId, int childId) {
+ assert parentId == -1 || (parentId >= 0 && parentId < edges.size());
+ assert nodes.size() == edges.size();
+
+ if (parentId != -1)
+ edges.get(parentId).add(childId);
+ }
+
+ public List<Ord<T>> children(int parentId) {
+ GridIntList children = edges.get(parentId);
+
+ ArrayList<Ord<T>> ords = new ArrayList<>(children.size());
+
+ for (int i = 0; i < children.size(); i++)
+ ords.add(Ord.of(children.get(i), nodes.get(children.get(i))));
+
+ return ords;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
new file mode 100644
index 0000000..82e3f22
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public interface GraphNode extends Serializable {
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
new file mode 100644
index 0000000..ab75446
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.util.List;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSyntax;
+
+/**
+ *
+ */
+public class CallExpression implements Expression {
+ public final String opName;
+ public final SqlSyntax opSyntax;
+ public final List<Expression> operands;
+
+ public CallExpression(SqlOperator op, List<Expression> operands) {
+ this.operands = operands;
+ opName = op.getName();
+ opSyntax = op.getSyntax();
+ }
+
+ @Override public <T> T implement(ExpImplementor<T> implementor) {
+ return implementor.implement(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
new file mode 100644
index 0000000..94b2b85
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class DynamicParamExpression implements Expression {
+ public final DataType type;
+ public final int index;
+
+ public DynamicParamExpression(RelDataType type, int index) {
+ this.type = DataType.fromType(type);
+ this.index = index;
+ }
+
+ @Override public <T> T implement(ExpImplementor<T> implementor) {
+ return implementor.implement(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
new file mode 100644
index 0000000..2997c7d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+/**
+ *
+ */
+public interface ExpImplementor<T> {
+ T implement(CallExpression callExpression);
+
+ T implement(InputRefExpression inputRefExpression);
+
+ T implement(LiteralExpression literalExpression);
+
+ T implement(LocalRefExpression localRefExpression);
+
+ T implement(DynamicParamExpression dynamicParamExpression);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java
new file mode 100644
index 0000000..946e483
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class ExpToRexTranslator implements ExpImplementor<RexNode> {
+ private final RexBuilder builder;
+ private final RelDataTypeFactory typeFactory;
+ private final Map<Pair<String, SqlSyntax>, SqlOperator> ops;
+
+ public ExpToRexTranslator(RexBuilder builder, RelDataTypeFactory typeFactory, SqlOperatorTable opTable) {
+ this.builder = builder;
+ this.typeFactory = typeFactory;
+
+ List<SqlOperator> opList = opTable.getOperatorList();
+
+ HashMap<Pair<String, SqlSyntax>, SqlOperator> ops = new HashMap<>(opList.size());
+
+ for (SqlOperator op : opList) {
+ ops.put(Pair.of(op.getName(), op.getSyntax()), op);
+ }
+
+ this.ops = ops;
+ }
+
+ public List<RexNode> translate(List<Expression> exps) {
+ if (F.isEmpty(exps))
+ return Collections.emptyList();
+
+ if (exps.size() == 1)
+ return F.asList(translate(F.first(exps)));
+
+ List<RexNode> res = new ArrayList<>(exps.size());
+
+ for (Expression exp : exps) {
+ res.add(exp.implement(this));
+ }
+
+ return res;
+ }
+
+ public RexNode translate(Expression exp) {
+ return exp.implement(this);
+ }
+
+ @Override public RexNode implement(CallExpression exp) {
+ return builder.makeCall(op(exp.opName, exp.opSyntax), translate(exp.operands));
+ }
+
+ @Override public RexNode implement(InputRefExpression exp) {
+ return builder.makeInputRef(exp.type.toRelDataType(typeFactory), exp.index);
+ }
+
+ @Override public RexNode implement(LiteralExpression exp) {
+ return builder.makeLiteral(exp.value, exp.type.toRelDataType(typeFactory), false);
+ }
+
+ @Override public RexNode implement(LocalRefExpression exp) {
+ return new RexLocalRef(exp.index, exp.type.toRelDataType(typeFactory));
+ }
+
+ @Override public RexNode implement(DynamicParamExpression exp) {
+ return builder.makeDynamicParam(exp.type.toRelDataType(typeFactory), exp.index);
+ }
+
+ private SqlOperator op(String name, SqlSyntax syntax) {
+ return ops.get(Pair.of(name, syntax));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java
new file mode 100644
index 0000000..7e92b6f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public interface Expression extends Serializable {
+ <T> T implement(ExpImplementor<T> implementor);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
new file mode 100644
index 0000000..b561b1e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class InputRefExpression implements Expression {
+ public final DataType type;
+ public final int index;
+
+ public InputRefExpression(RelDataType type, int index) {
+ this.type = DataType.fromType(type);
+ this.index = index;
+ }
+
+ @Override public <T> T implement(ExpImplementor<T> implementor) {
+ return implementor.implement(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java
new file mode 100644
index 0000000..038b142
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class LiteralExpression implements Expression {
+ public final DataType type;
+ public final Comparable value;
+
+ public LiteralExpression(RelDataType type, Comparable value) {
+ this.type = DataType.fromType(type);
+ this.value = value;
+ }
+
+ @Override public <T> T implement(ExpImplementor<T> implementor) {
+ return implementor.implement(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java
new file mode 100644
index 0000000..e6023b8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class LocalRefExpression implements Expression {
+ public final DataType type;
+ public final int index;
+
+ public LocalRefExpression(RelDataType type, int index) {
+ this.type = DataType.fromType(type);
+ this.index = index;
+ }
+
+ @Override public <T> T implement(ExpImplementor<T> implementor) {
+ return implementor.implement(this);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java
new file mode 100644
index 0000000..22db0db
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexVisitor;
+
+/**
+ *
+ */
+public class RexToExpTranslator implements RexVisitor<Expression> {
+ public List<Expression> translate(List<RexNode> operands) {
+ ArrayList<Expression> res = new ArrayList<>(operands.size());
+
+ for (RexNode operand : operands) {
+ res.add(translate(operand));
+ }
+
+ return res;
+ }
+
+ public Expression translate(RexNode rex) {
+ return rex.accept(this);
+ }
+
+ @Override public Expression visitInputRef(RexInputRef inputRef) {
+ return new InputRefExpression(inputRef.getType(), inputRef.getIndex());
+ }
+
+ @Override public Expression visitLocalRef(RexLocalRef localRef) {
+ return new LocalRefExpression(localRef.getType(), localRef.getIndex());
+ }
+
+ @Override public Expression visitLiteral(RexLiteral literal) {
+ return new LiteralExpression(literal.getType(), literal.getValue());
+ }
+
+ @Override public Expression visitCall(RexCall call) {
+ return new CallExpression(call.getOperator(), translate(call.getOperands()));
+ }
+
+ @Override public Expression visitOver(RexOver over) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Expression visitCorrelVariable(RexCorrelVariable correlVariable) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Expression visitDynamicParam(RexDynamicParam dynamicParam) {
+ return new DynamicParamExpression(dynamicParam.getType(), dynamicParam.getIndex());
+ }
+
+ @Override public Expression visitRangeRef(RexRangeRef rangeRef) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Expression visitFieldAccess(RexFieldAccess fieldAccess) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Expression visitSubQuery(RexSubQuery subQuery) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Expression visitTableInputRef(RexTableInputRef fieldRef) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override public Expression visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java
new file mode 100644
index 0000000..0d8399e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
+
+/**
+ *
+ */
+public interface ConversionContext extends RelOptTable.ToRelContext {
+ RelDataTypeFactory getTypeFactory();
+
+ RelOptSchema getSchema();
+
+ PlannerContext getContext();
+
+ ExpToRexTranslator getExpressionTranslator();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java
new file mode 100644
index 0000000..1fd3da2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class FilterNode extends RelGraphNode {
+ private final Expression condition;
+
+ private FilterNode(Expression condition) {
+ this.condition = condition;
+ }
+
+ public static FilterNode create(IgniteFilter rel, RexToExpTranslator expTranslator) {
+ return new FilterNode(expTranslator.translate(rel.getCondition()));
+ }
+
+ @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+ RelNode input = F.first(children);
+ RexNode condition = this.condition.implement(ctx.getExpressionTranslator());
+ RelOptCluster cluster = input.getCluster();
+ RelMetadataQuery mq = cluster.getMetadataQuery();
+
+ RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE)
+ .replaceIf(DistributionTraitDef.INSTANCE, () -> IgniteMdDistribution.filter(mq, input, condition));
+
+ return new IgniteFilter(cluster, traits, input, condition);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java
new file mode 100644
index 0000000..f7574bf
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class GraphToRelConverter implements ConversionContext {
+ private final RelOptTable.ViewExpander viewExpander;
+ private final RelBuilder relBuilder;
+ private final ExpToRexTranslator expTranslator;
+
+ public GraphToRelConverter(RelOptTable.ViewExpander viewExpander, RelBuilder relBuilder, SqlOperatorTable operatorTable) {
+ this.viewExpander = viewExpander;
+ this.relBuilder = relBuilder;
+
+ expTranslator = new ExpToRexTranslator(
+ relBuilder.getRexBuilder(),
+ getTypeFactory(),
+ operatorTable);
+ }
+
+ @Override public RelDataTypeFactory getTypeFactory() {
+ return getCluster().getTypeFactory();
+ }
+
+ @Override public RelOptSchema getSchema() {
+ return relBuilder.getRelOptSchema();
+ }
+
+ @Override public PlannerContext getContext() {
+ return Commons.plannerContext(getCluster().getPlanner().getContext());
+ }
+
+ @Override public ExpToRexTranslator getExpressionTranslator() {
+ return expTranslator;
+ }
+
+ @Override public RelOptCluster getCluster() {
+ return relBuilder.getCluster();
+ }
+
+ @Override public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
+ return viewExpander.expandView(rowType, queryString, schemaPath, viewPath);
+ }
+
+ public RelNode convert(RelGraph graph) {
+ return F.first(convertRecursive(this, graph, graph.nodes().subList(0, 1)));
+ }
+
+ private List<RelNode> convertRecursive(ConversionContext ctx, RelGraph graph, List<Ord<RelGraphNode>> src) {
+ ImmutableList.Builder<RelNode> b = ImmutableList.builder();
+
+ for (Ord<RelGraphNode> node : src)
+ b.add(node.e.toRel(ctx, convertRecursive(ctx, graph, graph.children(node.i))));
+
+ return b.build();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java
new file mode 100644
index 0000000..c26d2c7
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+
+/**
+ *
+ */
+public class JoinNode extends RelGraphNode {
+ private final Expression condition;
+ private final int[] variables;
+ private final JoinRelType joinType;
+
+ private JoinNode(RelTraitSet traits, Expression condition, int[] variables, JoinRelType joinType) {
+ super(traits);
+ this.condition = condition;
+ this.variables = variables;
+ this.joinType = joinType;
+ }
+
+ public static JoinNode create(IgniteJoin rel, RexToExpTranslator expTranslator) {
+ return new JoinNode(rel.getTraitSet(),
+ expTranslator.translate(rel.getCondition()),
+ rel.getVariablesSet().stream().mapToInt(CorrelationId::getId).toArray(),
+ rel.getJoinType());
+ }
+
+ @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+ assert children.size() == 2;
+
+ RelNode left = children.get(0);
+ RelNode right = children.get(1);
+
+ return new IgniteJoin(ctx.getCluster(),
+ traitSet.toTraitSet(ctx.getCluster()),
+ left,
+ right,
+ ctx.getExpressionTranslator().translate(condition),
+ Arrays.stream(variables).mapToObj(CorrelationId::new).collect(Collectors.toSet()),
+ joinType);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java
new file mode 100644
index 0000000..56c60df
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class ProjectNode extends RelGraphNode {
+ private final List<Expression> projects;
+ private final DataType dataType;
+
+ private ProjectNode(List<Expression> projects, DataType dataType) {
+ this.projects = projects;
+ this.dataType = dataType;
+ }
+
+ public static ProjectNode create(IgniteProject rel, RexToExpTranslator rexTranslator) {
+ return new ProjectNode(rexTranslator.translate(rel.getProjects()),
+ DataType.fromType(rel.getRowType()));
+ }
+
+ @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+ RelNode input = F.first(children);
+ List<RexNode> projects = ctx.getExpressionTranslator().translate(this.projects);
+ RelOptCluster cluster = input.getCluster();
+ RelMetadataQuery mq = cluster.getMetadataQuery();
+
+ RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE)
+ .replaceIf(DistributionTraitDef.INSTANCE, () -> IgniteMdDistribution.project(mq, input, projects));
+
+ return new IgniteProject(cluster, traits, input, projects, dataType.toRelDataType(ctx.getTypeFactory()));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java
new file mode 100644
index 0000000..96bc0dd
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSourceImpl;
+
+
+/**
+ *
+ */
+public class ReceiverNode extends RelGraphNode {
+ private final DataType dataType;
+ private final RelSource source;
+
+ private ReceiverNode(RelTraitSet traits, DataType dataType, RelSource source) {
+ super(traits);
+ this.dataType = dataType;
+ this.source = source;
+ }
+
+ public static ReceiverNode create(IgniteReceiver rel) {
+ RelSource source = new RelSourceImpl(rel.source().exchangeId(), rel.source().mapping());
+
+ return new ReceiverNode(rel.getTraitSet(), DataType.fromType(rel.getRowType()), source);
+ }
+
+ @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+ return new IgniteReceiver(ctx.getCluster(),
+ traitSet.toTraitSet(ctx.getCluster()),
+ dataType.toRelDataType(ctx.getTypeFactory()),
+ source);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java
new file mode 100644
index 0000000..bb38efe
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import org.apache.ignite.internal.processors.query.calcite.serialize.Graph;
+
+/**
+ *
+ */
+public class RelGraph extends Graph<RelGraphNode> {
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java
new file mode 100644
index 0000000..ab53320
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.calcite.serialize.GraphNode;
+
+/**
+ *
+ */
+public abstract class RelGraphNode implements GraphNode {
+ protected SerializedTraits traitSet;
+
+ protected RelGraphNode() {
+ }
+
+ protected RelGraphNode(RelTraitSet traits) {
+ traitSet = new SerializedTraits(traits);
+ }
+
+ public abstract RelNode toRel(ConversionContext ctx, List<RelNode> children);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java
new file mode 100644
index 0000000..56d77aa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.ArrayDeque;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class RelToGraphConverter implements RelOp<IgniteRel, RelGraph> {
+ private final RexToExpTranslator rexTranslator = new RexToExpTranslator();
+
+ private RelGraph graph;
+ private int curParent;
+
+ private static final class Item {
+ final int parentId;
+ final List<IgniteRel> children;
+
+ private Item(int parentId, List<IgniteRel> children) {
+ this.parentId = parentId;
+ this.children = children;
+ }
+ }
+
+ private final class ItemTranslator implements IgniteRelVisitor<Item> {
+ @Override public Item visit(IgniteFilter rel) {
+ return new Item(graph.addNode(curParent, FilterNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
+ }
+
+ @Override public Item visit(IgniteJoin rel) {
+ return new Item(graph.addNode(curParent, JoinNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
+ }
+
+ @Override public Item visit(IgniteProject rel) {
+ return new Item(graph.addNode(curParent, ProjectNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
+ }
+
+ @Override public Item visit(IgniteTableScan rel) {
+ return new Item(graph.addNode(curParent, TableScanNode.create(rel)), Commons.cast(rel.getInputs()));
+ }
+
+ @Override public Item visit(IgniteReceiver rel) {
+ return new Item(graph.addNode(curParent, ReceiverNode.create(rel)), Collections.emptyList());
+ }
+
+ @Override public Item visit(IgniteSender rel) {
+ return new Item(graph.addNode(curParent, SenderNode.create(rel)), Commons.cast(rel.getInputs()));
+ }
+
+ @Override public Item visit(IgniteRel rel) {
+ return rel.accept(this);
+ }
+
+ @Override public Item visit(IgniteExchange rel) {
+ throw new AssertionError("Unexpected node: " + rel);
+ }
+ }
+
+ @Override public RelGraph go(IgniteRel root) {
+ graph = new RelGraph();
+
+ ItemTranslator itemTranslator = new ItemTranslator();
+ Deque<Item> stack = new ArrayDeque<>();
+ stack.push(new Item(-1, F.asList(root)));
+
+ while (!stack.isEmpty()) {
+ Item item = stack.pop();
+
+ curParent = item.parentId;
+
+ for (IgniteRel child : item.children) {
+ stack.push(itemTranslator.visit(child));
+ }
+ }
+
+ return graph;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java
new file mode 100644
index 0000000..2940015
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelTarget;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class SenderNode extends RelGraphNode {
+ private final RelTarget target;
+
+ private SenderNode(RelTarget target) {
+ this.target = target;
+ }
+
+ public static SenderNode create(IgniteSender rel) {
+ return new SenderNode(rel.target());
+ }
+
+ @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+ RelNode input = F.first(children);
+ RelOptCluster cluster = input.getCluster();
+ RelMetadataQuery mq = cluster.getMetadataQuery();
+
+ RelTraitSet traits = cluster.traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replaceIf(DistributionTraitDef.INSTANCE, () -> IgniteMdDistribution._distribution(input, mq));
+
+ return new IgniteSender(cluster, traits, input, target);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SerializedTraits.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SerializedTraits.java
new file mode 100644
index 0000000..94efe86
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SerializedTraits.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class SerializedTraits implements Serializable {
+ private static final Byte CONVENTION = 0;
+
+ private final List<Serializable> traits;
+
+ public SerializedTraits(RelTraitSet traits) {
+ this.traits = Commons.transform(traits, this::toSerializable);
+ }
+
+ public RelTraitSet toTraitSet(RelOptCluster cluster) {
+ RelTraitSet traits = cluster.traitSet();
+
+ for (Serializable trait : this.traits)
+ traits.replace(fromSerializable(trait));
+
+ return traits.simplify();
+ }
+
+ private Serializable toSerializable(RelTrait trait) {
+ if (trait instanceof Serializable)
+ return (Serializable) trait;
+ if (trait == IgniteConvention.INSTANCE)
+ return CONVENTION;
+
+ throw new AssertionError();
+ }
+
+ private RelTrait fromSerializable(Serializable trait) {
+ if (trait instanceof RelTrait)
+ return (RelTrait) trait;
+ if (CONVENTION.equals(trait))
+ return IgniteConvention.INSTANCE;
+
+ throw new AssertionError();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java
new file mode 100644
index 0000000..097a8fa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+
+/**
+ *
+ */
+public class TableScanNode extends RelGraphNode {
+ private final List<String> tableName;
+
+ private TableScanNode(RelTraitSet traits, List<String> tableName) {
+ super(traits);
+ this.tableName = tableName;
+ }
+
+ public static TableScanNode create(IgniteTableScan rel) {
+ return new TableScanNode(rel.getTraitSet(), rel.getTable().getQualifiedName());
+ }
+
+ @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+ return new IgniteTableScan(ctx.getCluster(),
+ traitSet.toTraitSet(ctx.getCluster()),
+ ctx.getSchema().getTableForMember(tableName));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java
new file mode 100644
index 0000000..317c95c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.type;
+
+import java.io.Serializable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+
+/**
+ *
+ */
+public interface DataType extends Serializable {
+ static DataType fromType(RelDataType type) {
+ return type.isStruct() ? StructType.fromType(type) : SimpleType.fromType(type);
+ }
+
+ RelDataType toRelDataType(RelDataTypeFactory factory);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/SimpleType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/SimpleType.java
new file mode 100644
index 0000000..a5bb3f4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/SimpleType.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ *
+ */
+public class SimpleType implements DataType {
+ private final Class clazz;
+ private final SqlTypeName typeName;
+ private final int precision;
+ private final int scale;
+
+ public static SimpleType fromType(RelDataType type) {
+ assert !type.isStruct();
+
+ if (type instanceof RelDataTypeFactoryImpl.JavaType)
+ return new SimpleType(((RelDataTypeFactoryImpl.JavaType) type).getJavaClass(), null, 0, 0);
+
+ return new SimpleType(null, type.getSqlTypeName(), type.getPrecision(), type.getScale());
+ }
+
+ private SimpleType(Class clazz, SqlTypeName typeName, int precision, int scale) {
+ this.clazz = clazz;
+ this.typeName = typeName;
+ this.precision = precision;
+ this.scale = scale;
+ }
+
+ @Override public RelDataType toRelDataType(RelDataTypeFactory factory) {
+ if (clazz != null)
+ return factory.createJavaType(clazz);
+ if (typeName.allowsNoPrecNoScale())
+ return factory.createSqlType(typeName);
+ if (typeName.allowsPrecNoScale())
+ return factory.createSqlType(typeName, precision);
+
+ return factory.createSqlType(typeName, precision, scale);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/StructType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/StructType.java
new file mode 100644
index 0000000..c712754
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/StructType.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.type;
+
+import java.util.LinkedHashMap;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+/**
+ *
+ */
+public class StructType implements DataType {
+ private final LinkedHashMap<String, DataType> fields;
+
+ static StructType fromType(RelDataType type) {
+ assert type.isStruct();
+
+ LinkedHashMap<String, DataType> fields = new LinkedHashMap<>();
+
+ for (RelDataTypeField field : type.getFieldList()) {
+ fields.put(field.getName(), DataType.fromType(field.getType()));
+ }
+
+ return new StructType(fields);
+ }
+
+ private StructType(LinkedHashMap<String, DataType> fields) {
+ this.fields = fields;
+ }
+
+ @Override public RelDataType toRelDataType(RelDataTypeFactory factory) {
+ RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(factory);
+ fields.forEach((n,f) -> builder.add(n,f.toRelDataType(factory)));
+ return builder.build();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Edge.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Edge.java
new file mode 100644
index 0000000..a065dbe
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Edge.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ *
+ */
+public class Edge {
+ private final RelNode parent;
+ private final RelNode child;
+ private final int childIdx;
+
+ public Edge(RelNode parent, RelNode child, int childIdx) {
+ this.parent = parent;
+ this.child = child;
+ this.childIdx = childIdx;
+ }
+
+ public RelNode parent() {
+ return parent;
+ }
+
+ public RelNode child() {
+ return child;
+ }
+
+ public int childIdx() {
+ return childIdx;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Fragment.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Fragment.java
new file mode 100644
index 0000000..c812f66
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Fragment.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import com.google.common.collect.ImmutableList;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentInfo;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdFragmentInfo;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class Fragment implements RelSource {
+ private static final AtomicLong ID_GEN = new AtomicLong();
+
+ private final long exchangeId = ID_GEN.getAndIncrement();
+
+ private final RelNode root;
+
+ private NodesMapping mapping;
+
+ public Fragment(RelNode root) {
+ this.root = root;
+ }
+
+ public void init(PlannerContext ctx, RelMetadataQuery mq) {
+ FragmentInfo info = IgniteMdFragmentInfo.fragmentInfo(root, mq);
+
+ if (info.mapping() == null)
+ mapping = remote() ? ctx.mapForRandom(ctx.topologyVersion()) : ctx.mapForLocal();
+ else
+ mapping = info.mapping().deduplicate();
+
+ ImmutableList<Pair<IgniteReceiver, RelSource>> sources = info.sources();
+
+ if (!F.isEmpty(sources)) {
+ for (Pair<IgniteReceiver, RelSource> input : sources) {
+ IgniteReceiver receiver = input.left;
+ RelSource source = input.right;
+
+ source.init(mapping, receiver.distribution(), ctx, mq);
+ }
+ }
+ }
+
+ @Override public long exchangeId() {
+ return exchangeId;
+ }
+
+ @Override public void init(NodesMapping mapping, IgniteDistribution distribution, PlannerContext ctx, RelMetadataQuery mq) {
+ assert remote();
+
+ ((IgniteSender) root).target(new RelTargetImpl(exchangeId, mapping, distribution));
+
+ init(ctx, mq);
+ }
+
+ public RelNode root() {
+ return root;
+ }
+
+ @Override public NodesMapping mapping() {
+ return mapping;
+ }
+
+ private boolean remote() {
+ return root instanceof IgniteSender;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/QueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/QueryPlan.java
new file mode 100644
index 0000000..4a8ac5d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/QueryPlan.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.metadata.OptimisticPlanningException;
+import org.apache.ignite.internal.processors.query.calcite.metadata.RelMetadataQueryEx;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class QueryPlan {
+ private final List<Fragment> fragments;
+
+ public QueryPlan(List<Fragment> fragments) {
+ this.fragments = fragments;
+ }
+
+ public void init(PlannerContext ctx) {
+ int i = 0;
+
+ RelMetadataQueryEx mq = RelMetadataQueryEx.instance();
+
+ while (true) {
+ try {
+ F.first(fragments).init(ctx, mq);
+
+ break;
+ }
+ catch (OptimisticPlanningException e) {
+ if (++i > 3)
+ throw new IgniteSQLException("Failed to map query.", e);
+
+ Edge edge = e.edge();
+
+ RelNode parent = edge.parent();
+ RelNode child = edge.child();
+
+ RelOptCluster cluster = child.getCluster();
+ RelTraitSet traitSet = child.getTraitSet();
+
+ Fragment fragment = new Fragment(new IgniteSender(cluster, traitSet, child));
+
+ fragments.add(fragment);
+
+ parent.replaceInput(edge.childIdx(), new IgniteReceiver(cluster, traitSet, child.getRowType(), fragment));
+ }
+ }
+ }
+
+ public List<Fragment> fragments() {
+ return fragments;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSource.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSource.java
new file mode 100644
index 0000000..b4eb49e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSource.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public interface RelSource {
+ /**
+ * @return Exchange id, has to be unique in scope of query.
+ */
+ long exchangeId();
+
+ /**
+ * @return Source mapping.
+ */
+ NodesMapping mapping();
+
+ /**
+ * @param mapping Target mapping.
+ * @param distribution Target distribution.
+ * @param ctx Context.
+ * @param mq Metadata query instance.
+ */
+ default void init(NodesMapping mapping, IgniteDistribution distribution, PlannerContext ctx, RelMetadataQuery mq) {
+ // No-op
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSourceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSourceImpl.java
new file mode 100644
index 0000000..b0ca757
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSourceImpl.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+
+/**
+ *
+ */
+public class RelSourceImpl implements RelSource, Serializable {
+ private final long exchangeId;
+ private final NodesMapping mapping;
+
+ public RelSourceImpl(long exchangeId, NodesMapping mapping) {
+ this.exchangeId = exchangeId;
+ this.mapping = mapping;
+ }
+
+ @Override public long exchangeId() {
+ return exchangeId;
+ }
+
+ @Override public NodesMapping mapping() {
+ return mapping;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
new file mode 100644
index 0000000..b031b8e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public interface RelTarget {
+ long exchangeId();
+ NodesMapping mapping();
+ IgniteDistribution distribution();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTargetImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTargetImpl.java
new file mode 100644
index 0000000..a3c9b29
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTargetImpl.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public class RelTargetImpl implements RelTarget, Serializable {
+ private final long exchangeId;
+ private final NodesMapping mapping;
+ private final IgniteDistribution distribution;
+
+ public RelTargetImpl(long exchangeId, NodesMapping mapping, IgniteDistribution distribution) {
+ this.exchangeId = exchangeId;
+ this.mapping = mapping;
+ this.distribution = distribution;
+ }
+
+ @Override public long exchangeId() {
+ return exchangeId;
+ }
+
+ @Override public NodesMapping mapping() {
+ return mapping;
+ }
+
+ @Override public IgniteDistribution distribution() {
+ return distribution;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Splitter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Splitter.java
new file mode 100644
index 0000000..8dd3678
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Splitter.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.igniteRel;
+
+/**
+ *
+ */
+public class Splitter implements IgniteRelVisitor<IgniteRel>, RelOp<IgniteRel, QueryPlan> {
+ private List<Fragment> fragments;
+
+ @Override public QueryPlan go(IgniteRel root) {
+ fragments = new ArrayList<>();
+
+ fragments.add(new Fragment(visit(root)));
+
+ Collections.reverse(fragments);
+
+ return new QueryPlan(fragments);
+ }
+
+ @Override public IgniteRel visit(IgniteExchange rel) {
+ RelOptCluster cluster = rel.getCluster();
+ RelTraitSet outTraits = rel.getTraitSet();
+
+ IgniteRel input = visit(igniteRel(rel.getInput()));
+ RelTraitSet inTraits = input.getTraitSet();
+
+ Fragment fragment = new Fragment(new IgniteSender(cluster, inTraits, input));
+
+ fragments.add(fragment);
+
+ return new IgniteReceiver(cluster, outTraits, input.getRowType(), fragment);
+ }
+
+ @Override public IgniteRel visit(IgniteFilter rel) {
+ return visitChild(rel);
+ }
+
+ @Override public IgniteRel visit(IgniteProject rel) {
+ return visitChild(rel);
+ }
+
+ @Override public IgniteRel visit(IgniteJoin rel) {
+ return visitChildren(rel);
+ }
+
+ @Override public IgniteRel visit(IgniteTableScan rel) {
+ return rel;
+ }
+
+ @Override public IgniteRel visit(IgniteRel rel) {
+ return rel.accept(this);
+ }
+
+ @Override public IgniteRel visit(IgniteReceiver rel) {
+ throw new AssertionError("An attempt to split an already split task.");
+ }
+
+ @Override public IgniteRel visit(IgniteSender rel) {
+ throw new AssertionError("An attempt to split an already split task.");
+ }
+
+ private IgniteRel visitChildren(IgniteRel rel) {
+ for (Ord<RelNode> input : Ord.zip(rel.getInputs()))
+ visitChild(rel, input.i, igniteRel(input.e));
+
+ return rel;
+ }
+
+ /**
+ * Visits a single child of a parent.
+ */
+ private <T extends SingleRel & IgniteRel> IgniteRel visitChild(T rel) {
+ visitChild(rel, 0, igniteRel(rel.getInput()));
+
+ return rel;
+ }
+
+ /**
+ * Visits a particular child of a parent.
+ */
+ private void visitChild(IgniteRel parent, int i, IgniteRel child) {
+ IgniteRel child2 = visit(child);
+ if (child2 != child)
+ parent.replaceInput(i, child2);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AbstractDestinationFunctionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AbstractDestinationFunctionFactory.java
new file mode 100644
index 0000000..e6b6bb5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AbstractDestinationFunctionFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.util.Objects;
+
+/**
+ *
+ */
+public abstract class AbstractDestinationFunctionFactory implements DestinationFunctionFactory {
+ @Override public int hashCode() {
+ return Objects.hashCode(key());
+ }
+
+ @Override public boolean equals(Object obj) {
+ if (obj instanceof DestinationFunctionFactory)
+ return Objects.equals(key(), ((DestinationFunctionFactory) obj).key());
+
+ return false;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityFactory.java
new file mode 100644
index 0000000..f1dbbac
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.ToIntFunction;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+public final class AffinityFactory extends AbstractDestinationFunctionFactory {
+ private final int cacheId;
+ private final Object key;
+
+ public AffinityFactory(int cacheId, Object key) {
+ this.cacheId = cacheId;
+ this.key = key;
+ }
+
+ @Override public DestinationFunction create(PlannerContext ctx, NodesMapping mapping, ImmutableIntList keys) {
+ assert keys.size() == 1 && mapping != null && !F.isEmpty(mapping.assignments());
+
+ List<List<UUID>> assignments = mapping.assignments();
+
+ if (U.assertionsEnabled()) {
+ for (List<UUID> assignment : assignments) {
+ assert F.isEmpty(assignment) || assignment.size() == 1;
+ }
+ }
+
+ ToIntFunction<Object> rowToPart = ctx.kernalContext()
+ .cache().context().cacheContext(cacheId).affinity()::partition;
+
+ return row -> assignments.get(rowToPart.applyAsInt(((Object[]) row)[keys.getInt(0)]));
+ }
+
+ @Override public Object key() {
+ return key;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllTargetsFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllTargetsFactory.java
new file mode 100644
index 0000000..30ce8b8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllTargetsFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.io.ObjectStreamException;
+import java.util.List;
+import java.util.UUID;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public final class AllTargetsFactory extends AbstractDestinationFunctionFactory {
+ public static final DestinationFunctionFactory INSTANCE = new AllTargetsFactory();
+
+ @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+ List<UUID> nodes = m.nodes();
+
+ return r -> nodes;
+ }
+
+ @Override public Object key() {
+ return "AllTargetsFactory";
+ }
+
+ private Object readResolve() throws ObjectStreamException {
+ return INSTANCE;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
new file mode 100644
index 0000000..3d4dc41
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.util.List;
+import java.util.UUID;
+
+/**
+ *
+ */
+public interface DestinationFunction {
+ List<UUID> destination(Object row);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunctionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunctionFactory.java
new file mode 100644
index 0000000..d12ec15
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunctionFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.io.Serializable;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public interface DestinationFunctionFactory extends Serializable {
+ DestinationFunction create(PlannerContext ctx, NodesMapping mapping, ImmutableIntList keys);
+
+ Object key();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
new file mode 100644
index 0000000..5972a11
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
@@ -0,0 +1,174 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import com.google.common.collect.Ordering;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Objects;
+import org.apache.calcite.plan.RelMultipleTrait;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.mapping.Mappings;
+
+import static org.apache.calcite.rel.RelDistribution.Type.ANY;
+import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.RANDOM_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.RANGE_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.ROUND_ROBIN_DISTRIBUTED;
+
+/**
+ *
+ */
+public final class DistributionTrait implements IgniteDistribution, Serializable {
+ private static final Ordering<Iterable<Integer>> ORDERING =
+ Ordering.<Integer>natural().lexicographical();
+
+ private RelDistribution.Type type;
+ private ImmutableIntList keys;
+ private DestinationFunctionFactory functionFactory;
+
+ public DistributionTrait() {
+ }
+
+ public DistributionTrait(RelDistribution.Type type, ImmutableIntList keys, DestinationFunctionFactory functionFactory) {
+ if (type == RANGE_DISTRIBUTED || type == ROUND_ROBIN_DISTRIBUTED)
+ throw new IllegalArgumentException("Distribution type " + type + " is unsupported.");
+
+ this.type = type;
+ this.keys = keys;
+ this.functionFactory = functionFactory;
+ }
+
+ @Override public RelDistribution.Type getType() {
+ return type;
+ }
+
+ @Override public DestinationFunctionFactory destinationFunctionFactory() {
+ return functionFactory;
+ }
+
+ @Override public ImmutableIntList getKeys() {
+ return keys;
+ }
+
+ @Override public void register(RelOptPlanner planner) {}
+
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+
+ if (o instanceof DistributionTrait) {
+ DistributionTrait that = (DistributionTrait) o;
+
+ return type == that.type
+ && Objects.equals(keys, that.keys)
+ && Objects.equals(functionFactory.key(), that.functionFactory.key());
+ }
+
+ return false;
+ }
+
+ @Override public int hashCode() {
+ return Objects.hash(type, keys);
+ }
+
+ @Override public String toString() {
+ return type + (type == Type.HASH_DISTRIBUTED ? "[" + functionFactory.key() + "]" + keys : "");
+ }
+
+ @Override public DistributionTraitDef getTraitDef() {
+ return DistributionTraitDef.INSTANCE;
+ }
+
+ @Override public boolean satisfies(RelTrait trait) {
+ if (trait == this)
+ return true;
+
+ if (!(trait instanceof DistributionTrait))
+ return false;
+
+ DistributionTrait other = (DistributionTrait) trait;
+
+ if (other.getType() == ANY)
+ return true;
+
+ if (getType() == other.getType())
+ return getType() != HASH_DISTRIBUTED
+ || (Objects.equals(keys, other.keys)
+ && Objects.equals(functionFactory, other.functionFactory));
+
+ return other.getType() == RANDOM_DISTRIBUTED && getType() == HASH_DISTRIBUTED;
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ out.writeObject(type);
+ out.writeObject(keys.toIntArray());
+ out.writeObject(functionFactory);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ type = (Type) in.readObject();
+ keys = ImmutableIntList.of((int[])in.readObject());
+ functionFactory = (DestinationFunctionFactory) in.readObject();
+ }
+
+ private Object readResolve() throws ObjectStreamException {
+ return DistributionTraitDef.INSTANCE.canonize(this);
+ }
+
+ @Override public IgniteDistribution apply(Mappings.TargetMapping mapping) {
+ if (keys.isEmpty())
+ return this;
+
+ assert type == HASH_DISTRIBUTED;
+
+ List<Integer> newKeys = IgniteDistributions.projectDistributionKeys(mapping, keys);
+
+ return newKeys.size() == keys.size() ? IgniteDistributions.hash(newKeys, functionFactory) :
+ IgniteDistributions.random();
+ }
+
+ @Override public boolean isTop() {
+ return type == Type.ANY;
+ }
+
+ @Override public int compareTo(RelMultipleTrait o) {
+ // TODO is this method really needed??
+
+ final IgniteDistribution distribution = (IgniteDistribution) o;
+
+ if (type == distribution.getType()
+ && (type == Type.HASH_DISTRIBUTED
+ || type == Type.RANGE_DISTRIBUTED)) {
+ int cmp = ORDERING.compare(getKeys(), distribution.getKeys());
+
+ if (cmp == 0)
+ cmp = Integer.compare(functionFactory.key().hashCode(), distribution.destinationFunctionFactory().key().hashCode());
+
+ return cmp;
+ }
+
+ return type.compareTo(distribution.getType());
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTraitDef.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTraitDef.java
new file mode 100644
index 0000000..306b7aa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTraitDef.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Exchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+
+/**
+ *
+ */
+public class DistributionTraitDef extends RelTraitDef<IgniteDistribution> {
+ /** */
+ public static final DistributionTraitDef INSTANCE = new DistributionTraitDef();
+
+ @Override public Class<IgniteDistribution> getTraitClass() {
+ return IgniteDistribution.class;
+ }
+
+ @Override public String getSimpleName() {
+ return "distr";
+ }
+
+ @Override public RelNode convert(RelOptPlanner planner, RelNode rel, IgniteDistribution targetDist, boolean allowInfiniteCostConverters) {
+ if (rel.getConvention() == Convention.NONE)
+ return null;
+
+ RelDistribution srcDist = rel.getTraitSet().getTrait(INSTANCE);
+
+ if (srcDist == targetDist) // has to be interned
+ return rel;
+
+ switch(targetDist.getType()){
+ case HASH_DISTRIBUTED:
+ case BROADCAST_DISTRIBUTED:
+ case SINGLETON:
+ Exchange exchange = new IgniteExchange(rel.getCluster(), rel.getTraitSet().replace(targetDist), rel, targetDist);
+ RelNode newRel = planner.register(exchange, rel);
+ RelTraitSet newTraits = rel.getTraitSet().replace(targetDist);
+
+ if (!newRel.getTraitSet().equals(newTraits))
+ newRel = planner.changeTraits(newRel, newTraits);
+
+ return newRel;
+ case ANY:
+ return rel;
+ default:
+ return null;
+ }
+ }
+
+ @Override public boolean canConvert(RelOptPlanner planner, IgniteDistribution fromTrait, IgniteDistribution toTrait) {
+ return true;
+ }
+
+ @Override public IgniteDistribution getDefault() {
+ return IgniteDistributions.any();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/HashFunctionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/HashFunctionFactory.java
new file mode 100644
index 0000000..82c7325
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/HashFunctionFactory.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.io.ObjectStreamException;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.ToIntFunction;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+public final class HashFunctionFactory extends AbstractDestinationFunctionFactory {
+ public static final DestinationFunctionFactory INSTANCE = new HashFunctionFactory();
+
+ @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+ assert m != null && !F.isEmpty(m.assignments());
+
+ int[] fields = k.toIntArray();
+
+ ToIntFunction<Object> hashFun = r -> {
+ Object[] row = (Object[]) r;
+
+ if (row == null)
+ return 0;
+
+ int hash = 1;
+
+ for (int i : fields)
+ hash = 31 * hash + (row[i] == null ? 0 : row[i].hashCode());
+
+ return hash;
+ };
+
+ List<List<UUID>> assignments = m.assignments();
+
+ if (U.assertionsEnabled()) {
+ for (List<UUID> assignment : assignments) {
+ assert F.isEmpty(assignment) || assignment.size() == 1;
+ }
+ }
+
+ return r -> assignments.get(hashFun.applyAsInt(r) % assignments.size());
+ }
+
+ @Override public Object key() {
+ return "DefaultHashFunction";
+ }
+
+ private Object readResolve() throws ObjectStreamException {
+ return INSTANCE;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
new file mode 100644
index 0000000..5f3b175
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.util.mapping.Mappings;
+
+/**
+ *
+ */
+public interface IgniteDistribution extends RelDistribution {
+ DestinationFunctionFactory destinationFunctionFactory();
+ @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistributions.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistributions.java
new file mode 100644
index 0000000..3e3aa55
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistributions.java
@@ -0,0 +1,272 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.calcite.rel.RelDistribution.Type.ANY;
+import static org.apache.calcite.rel.RelDistribution.Type.BROADCAST_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.RANDOM_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.SINGLETON;
+import static org.apache.calcite.rel.core.JoinRelType.INNER;
+import static org.apache.calcite.rel.core.JoinRelType.LEFT;
+import static org.apache.calcite.rel.core.JoinRelType.RIGHT;
+
+/**
+ *
+ */
+public class IgniteDistributions {
+ private static final int BEST_CNT = 3;
+
+ private static final IgniteDistribution BROADCAST_DISTR = new DistributionTrait(BROADCAST_DISTRIBUTED, ImmutableIntList.of(), AllTargetsFactory.INSTANCE);
+ private static final IgniteDistribution SINGLETON_DISTR = new DistributionTrait(SINGLETON, ImmutableIntList.of(), SingleTargetFactory.INSTANCE);
+ private static final IgniteDistribution RANDOM_DISTR = new DistributionTrait(RANDOM_DISTRIBUTED, ImmutableIntList.of(), RandomTargetFactory.INSTANCE);
+ private static final IgniteDistribution ANY_DISTR = new DistributionTrait(ANY, ImmutableIntList.of(), NoOpFactory.INSTANCE);
+
+ public static IgniteDistribution any() {
+ return ANY_DISTR;
+ }
+
+ public static IgniteDistribution random() {
+ return RANDOM_DISTR;
+ }
+
+ public static IgniteDistribution single() {
+ return SINGLETON_DISTR;
+ }
+
+ public static IgniteDistribution broadcast() {
+ return BROADCAST_DISTR;
+ }
+
+ public static IgniteDistribution hash(List<Integer> keys) {
+ return DistributionTraitDef.INSTANCE.canonize(
+ new DistributionTrait(HASH_DISTRIBUTED, ImmutableIntList.copyOf(keys), HashFunctionFactory.INSTANCE));
+ }
+
+ public static IgniteDistribution hash(List<Integer> keys, DestinationFunctionFactory factory) {
+ return DistributionTraitDef.INSTANCE.canonize(
+ new DistributionTrait(HASH_DISTRIBUTED, ImmutableIntList.copyOf(keys), factory));
+ }
+
+ public static List<BiSuggestion> suggestJoin(IgniteDistribution leftIn, IgniteDistribution rightIn,
+ JoinInfo joinInfo, JoinRelType joinType) {
+ return topN(suggestJoin0(leftIn, rightIn, joinInfo, joinType), BEST_CNT);
+ }
+
+ public static List<BiSuggestion> suggestJoin(List<IgniteDistribution> leftIn, List<IgniteDistribution> rightIn,
+ JoinInfo joinInfo, JoinRelType joinType) {
+ HashSet<BiSuggestion> suggestions = new HashSet<>();
+
+ int bestCnt = 0;
+
+ for (IgniteDistribution leftIn0 : leftIn) {
+ for (IgniteDistribution rightIn0 : rightIn) {
+ for (BiSuggestion suggest : suggestJoin0(leftIn0, rightIn0, joinInfo, joinType)) {
+ if (suggestions.add(suggest) && suggest.needExchange == 0 && (++bestCnt) == BEST_CNT)
+ topN(new ArrayList<>(suggestions), BEST_CNT);
+ }
+ }
+ }
+
+ return topN(new ArrayList<>(suggestions), BEST_CNT);
+ }
+
+ private static ArrayList<BiSuggestion> suggestJoin0(IgniteDistribution leftIn, IgniteDistribution rightIn,
+ JoinInfo joinInfo, JoinRelType joinType) {
+ /*
+ * Distributions table:
+ *
+ * ===============INNER JOIN==============
+ * hash + hash = hash
+ * broadcast + hash = hash
+ * hash + broadcast = hash
+ * broadcast + broadcast = broadcast
+ * single + single = single
+ *
+ * ===============LEFT JOIN===============
+ * hash + hash = hash
+ * hash + broadcast = hash
+ * broadcast + broadcast = broadcast
+ * single + single = single
+ *
+ * ===============RIGHT JOIN==============
+ * hash + hash = hash
+ * broadcast + hash = hash
+ * broadcast + broadcast = broadcast
+ * single + single = single
+ *
+ * ===========FULL JOIN/CROSS JOIN========
+ * broadcast + broadcast = broadcast
+ * single + single = single
+ *
+ *
+ * others require redistribution
+ */
+
+ ArrayList<BiSuggestion> res = new ArrayList<>();
+
+ IgniteDistribution out, left, right;
+
+ if (joinType == LEFT || joinType == RIGHT || (joinType == INNER && !F.isEmpty(joinInfo.keys()))) {
+ HashSet<DestinationFunctionFactory> factories = U.newHashSet(3);
+
+ if (leftIn.getKeys().equals(joinInfo.leftKeys))
+ factories.add(leftIn.destinationFunctionFactory());
+
+ if (rightIn.getKeys().equals(joinInfo.rightKeys))
+ factories.add(rightIn.destinationFunctionFactory());
+
+ factories.add(HashFunctionFactory.INSTANCE);
+
+ for (DestinationFunctionFactory factory : factories) {
+ out = hash(joinInfo.leftKeys, factory);
+
+ left = hash(joinInfo.leftKeys, factory); right = hash(joinInfo.rightKeys, factory);
+ add(res, out, leftIn, rightIn, left, right);
+
+ if (joinType == INNER || joinType == LEFT) {
+ left = hash(joinInfo.leftKeys, factory); right = broadcast();
+ add(res, out, leftIn, rightIn, left, right);
+ }
+
+ if (joinType == INNER || joinType == RIGHT) {
+ left = broadcast(); right = hash(joinInfo.rightKeys, factory);
+ add(res, out, leftIn, rightIn, left, right);
+ }
+ }
+ }
+
+ out = left = right = broadcast();
+ add(res, out, leftIn, rightIn, left, right);
+
+ out = left = right = single();
+ add(res, out, leftIn, rightIn, left, right);
+
+ return res;
+ }
+
+ private static int add(ArrayList<BiSuggestion> dst, IgniteDistribution out, IgniteDistribution left, IgniteDistribution right,
+ IgniteDistribution newLeft, IgniteDistribution newRight) {
+ int exch = 0;
+
+ if (!left.satisfies(newLeft))
+ exch++;
+
+ if (!right.satisfies(newRight))
+ exch++;
+
+ dst.add(new BiSuggestion(out, newLeft, newRight, exch));
+
+ return exch;
+ }
+
+ private static List<BiSuggestion> topN(ArrayList<BiSuggestion> src, int n) {
+ Collections.sort(src);
+
+ return src.size() <= n ? src : src.subList(0, n);
+ }
+
+ public static List<Integer> projectDistributionKeys(Mappings.TargetMapping mapping, ImmutableIntList keys) {
+ if (mapping.getTargetCount() < keys.size())
+ return Collections.emptyList();
+
+ List<Integer> resKeys = new ArrayList<>(mapping.getTargetCount());
+
+ parent:
+ for (int i = 0; i < keys.size(); i++) {
+ int key = keys.getInt(i);
+
+ for (int j = 0; j < mapping.getTargetCount(); j++) {
+ if (mapping.getSourceOpt(j) == key) {
+ resKeys.add(j);
+
+ continue parent;
+ }
+ }
+
+ return Collections.emptyList();
+ }
+
+ return resKeys;
+ }
+
+ public static class BiSuggestion implements Comparable<BiSuggestion> {
+ private final IgniteDistribution out;
+ private final IgniteDistribution left;
+ private final IgniteDistribution right;
+ private final int needExchange;
+
+ public BiSuggestion(IgniteDistribution out, IgniteDistribution left, IgniteDistribution right, int needExchange) {
+ this.out = out;
+ this.left = left;
+ this.right = right;
+ this.needExchange = needExchange;
+ }
+
+ public IgniteDistribution out() {
+ return out;
+ }
+
+ public IgniteDistribution left() {
+ return left;
+ }
+
+ public IgniteDistribution right() {
+ return right;
+ }
+
+ public int needExchange() {
+ return needExchange;
+ }
+
+ @Override public int compareTo(@NotNull IgniteDistributions.BiSuggestion o) {
+ return Integer.compare(needExchange, o.needExchange);
+ }
+
+ @Override public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ BiSuggestion that = (BiSuggestion) o;
+
+ if (needExchange != that.needExchange) return false;
+ if (out != that.out) return false;
+ if (left != that.left) return false;
+ return right == that.right;
+ }
+
+ @Override public int hashCode() {
+ int result = out.hashCode();
+ result = 31 * result + left.hashCode();
+ result = 31 * result + right.hashCode();
+ result = 31 * result + needExchange;
+ return result;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/NoOpFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/NoOpFactory.java
new file mode 100644
index 0000000..7086277
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/NoOpFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.io.ObjectStreamException;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public final class NoOpFactory extends AbstractDestinationFunctionFactory {
+ public static final DestinationFunctionFactory INSTANCE = new NoOpFactory();
+
+ @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+ return null;
+ }
+
+ @Override public Object key() {
+ return "NoOpFactory";
+ }
+
+ private Object readResolve() throws ObjectStreamException {
+ return INSTANCE;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomTargetFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomTargetFactory.java
new file mode 100644
index 0000000..8a643c3
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomTargetFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.io.ObjectStreamException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public final class RandomTargetFactory extends AbstractDestinationFunctionFactory {
+ public static final DestinationFunctionFactory INSTANCE = new RandomTargetFactory();
+
+ @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+ List<UUID> nodes = m.nodes();
+
+ return r -> Collections.singletonList(nodes.get(ThreadLocalRandom.current().nextInt(nodes.size())));
+ }
+
+ @Override public Object key() {
+ return "RandomTargetFactory";
+ }
+
+ private Object readResolve() throws ObjectStreamException {
+ return INSTANCE;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/SingleTargetFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/SingleTargetFactory.java
new file mode 100644
index 0000000..8a8b27c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/SingleTargetFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.trait;
+
+import java.io.ObjectStreamException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public final class SingleTargetFactory extends AbstractDestinationFunctionFactory {
+ public static final DestinationFunctionFactory INSTANCE = new SingleTargetFactory();
+
+ @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+ List<UUID> nodes = Collections.singletonList(Objects.requireNonNull(F.first(m.nodes())));
+
+ return r -> nodes;
+ }
+
+ @Override public Object key() {
+ return "SingleTargetFactory";
+ }
+
+ private Object readResolve() throws ObjectStreamException {
+ return INSTANCE;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
new file mode 100644
index 0000000..83a2a19
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.type;
+
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+
+/**
+ *
+ */
+public class IgniteTypeFactory extends JavaTypeFactoryImpl {
+ public IgniteTypeFactory() {
+ super(IgniteTypeSystem.DEFAULT);
+ }
+
+ public IgniteTypeFactory(RelDataTypeSystem typeSystem) {
+ super(typeSystem);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java
new file mode 100644
index 0000000..6dbfd02
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.type;
+
+import java.io.Serializable;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+/**
+ *
+ */
+public class IgniteTypeSystem extends RelDataTypeSystemImpl implements Serializable {
+ public static final RelDataTypeSystem DEFAULT = new IgniteTypeSystem();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/RowType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/RowType.java
new file mode 100644
index 0000000..2133c56
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/RowType.java
@@ -0,0 +1,140 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.type;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.util.ImmutableIntList;
+
+/**
+ *
+ */
+public class RowType {
+ private final String[] fields;
+ private final Class[] types;
+ private final BitSet keyFields;
+ private final int affinityKey;
+
+ public RowType(String[] fields, Class[] types, BitSet keyFields, int affinityKey) {
+ this.fields = fields;
+ this.types = types;
+ this.keyFields = keyFields;
+ this.affinityKey = affinityKey;
+ }
+
+ public RelDataType asRelDataType(RelDataTypeFactory factory) {
+ RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(factory);
+
+ int len = fields.length;
+
+ for (int i = 0; i < len; i++)
+ builder.add(fields[i], factory.createJavaType(types[i]));
+
+ return builder.build();
+ }
+
+ public List<Integer> distributionKeys() {
+ return ImmutableIntList.of(affinityKey);
+ }
+
+ public boolean isKeyField(int idx) {
+ return keyFields.get(idx);
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public static class Builder {
+ private int affinityKey;
+ private final LinkedHashSet<String> fields;
+ private final BitSet keyFields;
+ private final ArrayList<Class> types;
+
+ private Builder() {
+ fields = new LinkedHashSet<>();
+ types = new ArrayList<>();
+ keyFields = new BitSet();
+
+ fields.add("_key"); types.add(Object.class);
+ fields.add("_val"); types.add(Object.class);
+ }
+
+ public Builder key(Class type) {
+ if (types.get(0) != Object.class && types.get(0) != type)
+ throw new IllegalStateException("Key type is already set.");
+
+ types.set(0, type);
+
+ return this;
+ }
+
+ public Builder val(Class type) {
+ if (types.get(1) != Object.class && types.get(1) != type)
+ throw new IllegalStateException("Value type is already set.");
+
+ types.set(1, type);
+
+ return this;
+ }
+
+ public Builder field(String name, Class type) {
+ if (!fields.add(name))
+ throw new IllegalStateException("Field name must be unique.");
+
+ types.add(type);
+
+ return this;
+ }
+
+ public Builder keyField(String name, Class type) {
+ if (!fields.add(name))
+ throw new IllegalStateException("Field name must be unique.");
+
+ types.add(type);
+
+ keyFields.set(types.size() - 1);
+
+ return this;
+ }
+
+ public Builder keyField(String name, Class type, boolean affinityKey) {
+ if (affinityKey && this.affinityKey > 0)
+ throw new IllegalStateException("Affinity key field must be unique.");
+
+ if (!fields.add(name))
+ throw new IllegalStateException("Field name must be unique.");
+
+ types.add(type);
+
+ keyFields.set(types.size() - 1);
+
+ if (affinityKey)
+ this.affinityKey = types.size() - 1;
+
+ return this;
+ }
+
+ public RowType build() {
+ return new RowType(fields.toArray(new String[0]), types.toArray(new Class[0]), keyFields, affinityKey);
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
new file mode 100644
index 0000000..1eb5c76
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
@@ -0,0 +1,145 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.type.RowType;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public final class Commons {
+ private Commons(){}
+
+ public static Context convert(QueryContext ctx) {
+ return ctx == null ? Contexts.empty() : Contexts.of(ctx.unwrap(Object[].class));
+ }
+
+ /** */
+ public static RowType rowType(GridQueryTypeDescriptor desc) {
+ RowType.Builder b = RowType.builder();
+
+ Map<String, Class<?>> fields = desc.fields();
+
+ b.key(desc.keyClass()).val(desc.valueClass());
+
+ for (Map.Entry<String, Class<?>> entry : fields.entrySet()) {
+ GridQueryProperty prop = desc.property(entry.getKey());
+
+ if (prop.key())
+ b.keyField(prop.name(), prop.type(), Objects.equals(desc.affinityKey(), prop.name()));
+ else
+ b.field(prop.name(), prop.type());
+ }
+
+ return b.build();
+ }
+
+ public static RelOptRuleOperand any(Class<? extends RelNode> first, Class<? extends RelNode> second) {
+ return RelOptRule.operand(first, RelOptRule.operand(second, RelOptRule.any()));
+ }
+
+ public static <T> List<T> intersect(List<T> left, List<T> right) {
+ if (F.isEmpty(left) || F.isEmpty(right))
+ return Collections.emptyList();
+ else if (left.size() > right.size())
+ return intersect0(right, left);
+ else
+ return intersect0(left, right);
+ }
+
+ public static <T> List<T> intersect0(List<T> left, List<T> right) {
+ List<T> res = new ArrayList<>(Math.min(left.size(), right.size()));
+ HashSet<T> set = new HashSet<>(left);
+
+ for (T t : right) {
+ if (set.contains(t))
+ res.add(t);
+ }
+
+ return res;
+ }
+
+ public static <T> List<T> concat(List<T> col, T... elements) {
+ ArrayList<T> res = new ArrayList<>(col.size() + elements.length);
+
+ res.addAll(col);
+ res.addAll(Arrays.asList(elements));
+
+ return res;
+ }
+
+ @SuppressWarnings("unchecked")
+ public static <T> List<T> cast(List<?> src) {
+ return (List)src;
+ }
+
+ public static <T,R> List<R> transform(@NotNull List<T> src, @NotNull Function<T,R> mapFun) {
+ List<R> list = new ArrayList<>(src.size());
+
+ for (T t : src)
+ list.add(mapFun.apply(t));
+
+ return list;
+ }
+
+ public static <T,R> Set<R> transform(@NotNull Set<T> src, @NotNull Function<T,R> mapFun) {
+ Set<R> set = new HashSet<>(src.size());
+
+ for (T t : src)
+ set.add(mapFun.apply(t));
+
+ return set;
+ }
+
+ public static PlannerContext plannerContext(RelNode rel) {
+ return plannerContext(rel.getCluster().getPlanner().getContext());
+ }
+
+ public static PlannerContext plannerContext(Context ctx) {
+ return Objects.requireNonNull(ctx.unwrap(PlannerContext.class));
+ }
+
+ public static IgniteRel igniteRel(RelNode rel) {
+ if (rel.getConvention() != IgniteConvention.INSTANCE)
+ throw new AssertionError("Unexpected node: " + rel);
+
+ return (IgniteRel) rel;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java
new file mode 100644
index 0000000..5104816
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.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.ignite.internal.processors.query.calcite.util;
+
+import java.lang.reflect.Method;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.DerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.FragmentMetadata;
+
+/**
+ *
+ */
+public enum IgniteMethod {
+ DERIVED_DISTRIBUTIONS(DerivedDistribution.class, "deriveDistributions"),
+ FRAGMENT_INFO(FragmentMetadata.class, "getFragmentInfo");
+
+ private final Method method;
+
+ IgniteMethod(Class clazz, String methodName, Class... argumentTypes) {
+ method = Types.lookupMethod(clazz, methodName, argumentTypes);
+ }
+
+ /** */
+ public Method method() {
+ return method;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/ListFieldsQueryCursor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/ListFieldsQueryCursor.java
new file mode 100644
index 0000000..c46e762
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/ListFieldsQueryCursor.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public class ListFieldsQueryCursor<T> implements FieldsQueryCursor<List<?>> {
+ /** */
+ private final RelDataType rowType;
+
+ /** */
+ private final Enumerable<T> enumerable;
+
+ /** */
+ private final Function<T, List<?>> converter;
+
+ /** */
+ private Iterator<T> it;
+
+ /**
+ * @param rowType Row data type description.
+ * @param enumerable Rows source.
+ * @param converter Row converter.
+ */
+ public ListFieldsQueryCursor(RelDataType rowType, Enumerable<T> enumerable, Function<T, List<?>> converter) {
+ this.rowType = rowType;
+ this.enumerable = enumerable;
+ this.converter = converter;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getFieldName(int idx) {
+ return rowType.getFieldList().get(idx).getName();
+ }
+
+ /** {@inheritDoc} */
+ @Override public int getColumnsCount() {
+ return rowType.getFieldCount();
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<List<?>> getAll() {
+ return StreamSupport.stream(enumerable.spliterator(), false)
+ .map(converter)
+ .collect(Collectors.toList());
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ closeIterator();
+ }
+
+ /** {@inheritDoc} */
+ @NotNull @Override public Iterator<List<?>> iterator() {
+ closeIterator();
+
+ return F.iterator(it = enumerable.iterator(), converter::apply, true);
+ }
+
+ private void closeIterator() {
+ if (it instanceof AutoCloseable)
+ U.closeQuiet((AutoCloseable)it);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/TableScanIterator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/TableScanIterator.java
new file mode 100644
index 0000000..64f05dc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/TableScanIterator.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.util;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UNDEFINED_CACHE_ID;
+
+/**
+ *
+ */
+public class TableScanIterator<T> extends GridCloseableIteratorAdapter<T> {
+ private final int cacheId;
+ private final Iterator<GridDhtLocalPartition> parts;
+ private final Function<CacheDataRow, T> typeWrapper;
+ private final Predicate<CacheDataRow> typeFilter;
+
+ /**
+ *
+ */
+ private GridCursor<? extends CacheDataRow> cur;
+ /**
+ *
+ */
+ private GridDhtLocalPartition curPart;
+
+ /**
+ *
+ */
+ private T next;
+
+ public TableScanIterator(int cacheId, Iterator<GridDhtLocalPartition> parts, Function<CacheDataRow, T> typeWrapper,
+ Predicate<CacheDataRow> typeFilter) {
+ this.cacheId = cacheId;
+ this.parts = parts;
+ this.typeWrapper = typeWrapper;
+ this.typeFilter = typeFilter;
+ }
+
+ @Override
+ protected T onNext() {
+ if (next == null)
+ throw new NoSuchElementException();
+
+ T next = this.next;
+
+ this.next = null;
+
+ return next;
+ }
+
+ @Override
+ protected boolean onHasNext() throws IgniteCheckedException {
+ if (next != null)
+ return true;
+
+ while (true) {
+ if (cur == null) {
+ if (parts.hasNext()) {
+ GridDhtLocalPartition part = parts.next();
+
+ if (!reservePartition(part))
+ throw new IgniteSQLException("Failed to reserve partition, please retry on stable topology.");
+
+ IgniteCacheOffheapManager.CacheDataStore ds = part.dataStore();
+
+ cur = cacheId == UNDEFINED_CACHE_ID ? ds.cursor() : ds.cursor(cacheId);
+ } else
+ break;
+ }
+
+ if (cur.next()) {
+ CacheDataRow row = cur.get();
+
+ if (!typeFilter.test(row))
+ continue;
+
+ next = typeWrapper.apply(row);
+
+ break;
+ } else {
+ cur = null;
+
+ releaseCurrentPartition();
+ }
+ }
+
+ return next != null;
+ }
+
+ /**
+ *
+ */
+ private void releaseCurrentPartition() {
+ GridDhtLocalPartition p = curPart;
+
+ assert p != null;
+
+ curPart = null;
+
+ p.release();
+ }
+
+ /**
+ *
+ */
+ private boolean reservePartition(GridDhtLocalPartition p) {
+ if (p != null && p.reserve()) {
+ curPart = p;
+
+ return true;
+ }
+
+ return false;
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override protected void onClose() {
+ if (curPart != null)
+ releaseCurrentPartition();
+ }
+}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
new file mode 100644
index 0000000..f0afa1a
--- /dev/null
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
@@ -0,0 +1,1203 @@
+/*
+ * 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;
+
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exec.ConsumerNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.Implementor;
+import org.apache.ignite.internal.processors.query.calcite.exec.Node;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue;
+import org.apache.ignite.internal.processors.query.calcite.prepare.DataContextImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerPhase;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerType;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Query;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Splitter;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.type.RowType;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.igniteRel;
+
+/**
+ *
+ */
+//@WithSystemProperty(key = "calcite.debug", value = "true")
+public class CalciteQueryProcessorTest extends GridCommonAbstractTest {
+
+ private GridTestKernalContext kernalContext;
+ private CalciteQueryProcessor proc;
+ private SchemaPlus schema;
+ private List<UUID> nodes;
+
+ private TestIgniteTable city;
+ private TestIgniteTable country;
+ private TestIgniteTable project;
+ private TestIgniteTable developer;
+
+ @Before
+ public void setup() {
+ kernalContext = new GridTestKernalContext(log);
+ proc = new CalciteQueryProcessor();
+ proc.setLogger(log);
+ proc.start(kernalContext);
+
+ IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
+
+ developer = new TestIgniteTable("Developer", "Developer",
+ RowType.builder()
+ .keyField("id", Integer.class, true)
+ .field("name", String.class)
+ .field("projectId", Integer.class)
+ .field("cityId", Integer.class)
+ .build(), Arrays.asList(
+ new Object[]{0, null, 0, "Igor", 0, 1},
+ new Object[]{1, null, 1, "Roman", 0, 0}
+ ));
+
+ project = new TestIgniteTable("Project", "Project",
+ RowType.builder()
+ .keyField("id", Integer.class, true)
+ .field("name", String.class)
+ .field("ver", Integer.class)
+ .build(), Arrays.asList(
+ new Object[]{0, null, 0, "Calcite", 1},
+ new Object[]{1, null, 1, "Ignite", 1}
+ ));
+
+ country = new TestIgniteTable("Country", "Country",
+ RowType.builder()
+ .keyField("id", Integer.class, true)
+ .field("name", String.class)
+ .field("countryCode", Integer.class)
+ .build(), Arrays.<Object[]>asList(
+ new Object[]{0, null, 0, "Russia", 7}
+ ));
+
+ city = new TestIgniteTable("City", "City",
+ RowType.builder()
+ .keyField("id", Integer.class, true)
+ .field("name", String.class)
+ .field("countryId", Integer.class)
+ .build(), Arrays.asList(
+ new Object[]{0, null, 0, "Moscow", 0},
+ new Object[]{1, null, 1, "Saint Petersburg", 0}
+ ));
+
+ publicSchema.addTable(developer);
+ publicSchema.addTable(project);
+ publicSchema.addTable(country);
+ publicSchema.addTable(city);
+
+ schema = Frameworks
+ .createRootSchema(false)
+ .add("PUBLIC", publicSchema);
+
+ nodes = new ArrayList<>(4);
+
+ for (int i = 0; i < 4; i++) {
+ nodes.add(UUID.randomUUID());
+ }
+ }
+
+ @Test
+ public void testLogicalPlan() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.projectId = p.id0 + 1" +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(query);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+ }
+
+ assertNotNull(relRoot.rel);
+ }
+
+ @Test
+ public void testLogicalPlanDefaultSchema() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM Project pp" +
+ ") p " +
+ "ON d.projectId = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(query);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+ }
+
+ assertNotNull(relRoot.rel);
+
+
+ }
+
+ @Test
+ public void testCorrelatedQuery() throws Exception {
+ String sql = "SELECT d.id, (SELECT p.name FROM Project p WHERE p.id = d.id) name, d.projectId " +
+ "FROM Developer d";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(query);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+ }
+
+ assertNotNull(relRoot.rel);
+ }
+
+ @Test
+ public void testHepPlaner() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.projectId = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(query);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot.rel);
+ }
+
+ @Test
+ public void testVolcanoPlanerDistributed() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.projectId = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(query);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot.rel);
+
+ RexToExpTranslator translator = new RexToExpTranslator();
+
+ Project proj = (Project) relRoot.rel.getInput(0);
+
+ List<Expression> expressions = translator.translate(proj.getProjects());
+
+ assertNotNull(expressions);
+ }
+
+ @Test
+ public void testPlanSerializationDeserialization() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.id = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ byte[] convertedBytes;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ RelNode rel = planner.convert(sqlNode);
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ assertNotNull(rel);
+
+ QueryPlan plan = planner.plan(rel);
+
+ assertNotNull(plan);
+
+ assertTrue(plan.fragments().size() == 2);
+
+ plan.init(ctx);
+
+ RelGraph graph = new RelToGraphConverter().go(igniteRel(plan.fragments().get(1).root()));
+
+ convertedBytes = new JdkMarshaller().marshal(graph);
+
+ assertNotNull(convertedBytes);
+ }
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)) {
+ assertNotNull(planner);
+
+ RelGraph graph = new JdkMarshaller().unmarshal(convertedBytes, getClass().getClassLoader());
+
+ assertNotNull(graph);
+
+ RelNode rel = planner.convert(graph);
+
+ assertNotNull(rel);
+ }
+ }
+
+ @Test
+ public void testSplitterCollocatedPartitionedPartitioned() throws Exception {
+ Object key = new Object();
+
+ developer.identityKey(key);
+ project.identityKey(key);
+
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.id = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot);
+
+ QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+ assertNotNull(plan);
+
+ plan.init(ctx);
+
+ assertNotNull(plan);
+
+ assertTrue(plan.fragments().size() == 2);
+ }
+
+ @Test
+ public void testPhysicalPlan() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.name0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.name as name0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.projectId = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{-10}, this::context);
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ RelRoot relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSetOf(IgniteConvention.INSTANCE);
+
+ RelNode phys = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ assertNotNull(phys);
+
+ Map<String, Object> params = ctx.query().params(F.asMap(ContextValue.QUERY_ID.valueName(), new GridCacheVersion()));
+
+ Implementor implementor = new Implementor(new DataContextImpl(params, ctx));
+
+ Node<Object[]> exec = implementor.go(igniteRel(phys));
+
+ assertNotNull(exec);
+
+ assertTrue(exec instanceof ConsumerNode);
+
+ ConsumerNode consumer = (ConsumerNode) exec;
+
+ assertTrue(consumer.hasNext());
+
+ ArrayList<Object[]> res = new ArrayList<>();
+
+ while (consumer.hasNext())
+ res.add(consumer.next());
+
+ assertFalse(res.isEmpty());
+
+ Assert.assertArrayEquals(new Object[]{0, "Igor", 0, "Calcite", 1}, res.get(0));
+ Assert.assertArrayEquals(new Object[]{1, "Roman", 0, "Calcite", 1}, res.get(1));
+ }
+ }
+
+ @Test
+ public void testSplitterCollocatedReplicatedReplicated() throws Exception {
+ String sql = "SELECT d.id, (d.id + 1) as id2, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.id = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ MappingService ms = new MappingService() {
+ @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+ return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+ }
+
+ @Override public NodesMapping local() {
+ return new NodesMapping(select(nodes, 0), null, (byte) 0);
+ }
+
+ @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+ if (cacheId == CU.cacheId("Developer"))
+ return new NodesMapping(select(nodes, 0,1,2), null, NodesMapping.HAS_REPLICATED_CACHES);
+ if (cacheId == CU.cacheId("Project"))
+ return new NodesMapping(select(nodes, 0,1,2), null, NodesMapping.HAS_REPLICATED_CACHES);
+
+ throw new AssertionError("Unexpected cache id:" + cacheId);
+ }
+ };
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot);
+
+ QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+ assertNotNull(plan);
+
+ plan.init(ctx);
+
+ assertNotNull(plan);
+
+ assertTrue(plan.fragments().size() == 2);
+ }
+
+ @Test
+ public void testSplitterCollocatedReplicatedAndPartitioned() throws Exception {
+ developer.identityKey(new Object());
+
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.id = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ MappingService ms = new MappingService() {
+ @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+ return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+ }
+
+ @Override public NodesMapping local() {
+ return new NodesMapping(select(nodes, 0), null, (byte) 0);
+ }
+
+ @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+ if (cacheId == CU.cacheId("Developer"))
+ return new NodesMapping(null, Arrays.asList(
+ select(nodes, 0,1),
+ select(nodes, 1,2),
+ select(nodes, 2,0),
+ select(nodes, 0,1),
+ select(nodes, 1,2)
+ ), NodesMapping.HAS_PARTITIONED_CACHES);
+ if (cacheId == CU.cacheId("Project"))
+ return new NodesMapping(select(nodes, 0,1), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+ throw new AssertionError("Unexpected cache id:" + cacheId);
+ }
+ };
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot);
+
+ QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+ assertNotNull(plan);
+
+ plan.init(ctx);
+
+ assertNotNull(plan);
+
+ assertTrue(plan.fragments().size() == 2);
+ }
+
+ @Test
+ public void testSplitterPartiallyCollocated() throws Exception {
+ developer.identityKey(new Object());
+
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.projectId = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ MappingService ms = new MappingService() {
+ @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+ return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+ }
+
+ @Override public NodesMapping local() {
+ return new NodesMapping(select(nodes, 0), null, (byte) 0);
+ }
+
+ @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+ if (cacheId == CU.cacheId("Developer"))
+ return new NodesMapping(null, Arrays.asList(
+ select(nodes, 1),
+ select(nodes, 2),
+ select(nodes, 2),
+ select(nodes, 0),
+ select(nodes, 1)
+ ), NodesMapping.HAS_PARTITIONED_CACHES);
+ if (cacheId == CU.cacheId("Project"))
+ return new NodesMapping(select(nodes, 0,1), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+ throw new AssertionError("Unexpected cache id:" + cacheId);
+ }
+ };
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot);
+
+ QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+ assertNotNull(plan);
+
+ plan.init(ctx);
+
+ assertNotNull(plan);
+
+ assertTrue(plan.fragments().size() == 3);
+ }
+
+ @Test
+ public void testSplitterNonCollocated() throws Exception {
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.projectId = p.ver0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ MappingService ms = new MappingService() {
+ @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+ return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+ }
+
+ @Override public NodesMapping local() {
+ return new NodesMapping(select(nodes, 0), null, (byte) 0);
+ }
+
+ @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+ if (cacheId == CU.cacheId("Developer"))
+ return new NodesMapping(select(nodes, 2), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+ else if (cacheId == CU.cacheId("Project"))
+ return new NodesMapping(select(nodes, 0,1), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+ throw new AssertionError("Unexpected cache id:" + cacheId);
+ }
+ };
+
+ PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+
+ assertNotNull(ctx);
+
+ RelTraitDef[] traitDefs = {
+ DistributionTraitDef.INSTANCE,
+ ConventionTraitDef.INSTANCE
+ };
+
+ RelRoot relRoot;
+
+ try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+ assertNotNull(planner);
+
+ Query query = ctx.query();
+
+ assertNotNull(planner);
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query.sql());
+
+ // Validate
+ sqlNode = planner.validate(sqlNode);
+
+ // Convert to Relational operators graph
+ relRoot = planner.rel(sqlNode);
+
+ RelNode rel = relRoot.rel;
+
+ // Transformation chain
+ rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .simplify();
+
+ rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+ }
+
+ assertNotNull(relRoot);
+
+ QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+ assertNotNull(plan);
+
+ plan.init(ctx);
+
+ assertNotNull(plan);
+
+ assertTrue(plan.fragments().size() == 3);
+ }
+
+ @Test
+ public void testSplitterPartiallyReplicated1() throws Exception {
+ developer.identityKey(new Object());
+
+ String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+ "FROM PUBLIC.Developer d JOIN (" +
+ "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+ ") p " +
+ "ON d.id = p.id0 " +
+ "WHERE (d.projectId + 1) > ?";
+
+ MappingService ms = new MappingService() {
+ @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+ return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+ }
+
+ @Override public NodesMapping local() {
+ return new NodesMapping(select(nodes, 0), null, (byte) 0);
+ }
+
+ @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+ if (cacheId == CU.cacheId("Developer"))
... 1036 lines suppressed ...