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 2020/01/29 13:46:48 UTC
[ignite] branch ignite-12248 updated: IGNITE-12448: Calcite
integration. Communication protocol. IGNITE-12449: Calcite integration.
Execution flow. This closes #7272
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
The following commit(s) were added to refs/heads/ignite-12248 by this push:
new 095804d IGNITE-12448: Calcite integration. Communication protocol. IGNITE-12449: Calcite integration. Execution flow. This closes #7272
095804d is described below
commit 095804d5dd4c170ab2dd23f7b060047fef3b4b17
Author: Igor Seliverstov <gv...@gmail.com>
AuthorDate: Wed Jan 29 16:46:30 2020 +0300
IGNITE-12448: Calcite integration. Communication protocol.
IGNITE-12449: Calcite integration. Execution flow.
This closes #7272
---
.../query/calcite/CalciteQueryProcessor.java | 275 ++-
.../query/calcite/cluster/MappingServiceImpl.java | 158 --
.../processors/query/calcite/exchange/Inbox.java | 105 --
.../processors/query/calcite/exchange/Outbox.java | 202 ---
.../query/calcite/exec/AbstractNode.java | 108 +-
.../query/calcite/exec/ConsumerNode.java | 238 ++-
.../calcite/{exchange => exec}/EndMarker.java | 2 +-
.../ExchangeService.java} | 61 +-
.../query/calcite/exec/ExchangeServiceImpl.java | 190 ++
.../query/calcite/exec/ExecutionContext.java | 154 ++
.../query/calcite/exec/ExecutionService.java} | 32 +-
.../query/calcite/exec/ExecutionServiceImpl.java | 825 +++++++++
.../processors/query/calcite/exec/FilterNode.java | 14 +-
.../processors/query/calcite/exec/Implementor.java | 152 +-
.../processors/query/calcite/exec/Inbox.java | 442 +++++
.../processors/query/calcite/exec/JoinNode.java | 42 +-
.../query/calcite/exec/MailboxRegistry.java | 92 +
.../query/calcite/exec/MailboxRegistryImpl.java | 132 ++
.../processors/query/calcite/exec/Node.java | 53 +-
.../processors/query/calcite/exec/Outbox.java | 293 ++++
.../processors/query/calcite/exec/ProjectNode.java | 12 +-
.../exec/{Source.java => QueryTaskExecutor.java} | 15 +-
.../query/calcite/exec/QueryTaskExecutorImpl.java | 74 +
.../query/calcite/exec/ScalarFactory.java | 175 +-
.../processors/query/calcite/exec/ScanNode.java | 79 +-
.../processors/query/calcite/exec/SingleNode.java | 9 +-
.../processors/query/calcite/exec/Sink.java | 22 +-
.../EndMarker.java => message/CalciteMessage.java} | 27 +-
.../calcite/message/CalciteMessageFactory.java | 53 +
.../ExecutionContextAware.java} | 24 +-
.../query/calcite/message/GenericRowMessage.java | 118 ++
.../query/calcite/message/InboxCancelMessage.java | 171 ++
.../MarshalableMessage.java} | 24 +-
.../MessageListener.java} | 13 +-
.../MessageService.java} | 32 +-
.../query/calcite/message/MessageServiceImpl.java | 287 ++++
.../query/calcite/message/MessageType.java | 90 +
.../message/QueryBatchAcknowledgeMessage.java | 173 ++
.../query/calcite/message/QueryBatchMessage.java | 246 +++
.../query/calcite/message/QueryCancelRequest.java | 100 ++
.../query/calcite/message/QueryStartRequest.java | 283 +++
.../query/calcite/message/QueryStartResponse.java | 174 ++
.../query/calcite/metadata/FragmentInfo.java | 10 +-
.../calcite/metadata/IgniteMdDistribution.java | 7 +-
.../calcite/metadata/IgniteMdFragmentInfo.java | 8 +-
.../query/calcite/metadata/MappingService.java | 25 +-
.../query/calcite/metadata/MappingServiceImpl.java | 78 +
.../query/calcite/metadata/NodesMapping.java | 16 +-
.../metadata/OptimisticPlanningException.java | 10 +
.../PartitionService.java} | 17 +-
.../calcite/metadata/PartitionServiceImpl.java | 63 +
.../query/calcite/metadata/RelMetadataQueryEx.java | 2 +-
.../processors/query/calcite/prepare/CacheKey.java | 78 +
.../query/calcite/prepare/ContextValue.java | 59 -
.../query/calcite/prepare/DataContextImpl.java | 67 -
.../calcite/prepare/DistributedExecution.java | 110 --
.../query/calcite/prepare/IgnitePlanner.java | 115 +-
.../query/calcite/prepare/IgniteSqlValidator.java | 37 +-
.../query/calcite/prepare/PlannerContext.java | 345 ----
.../query/calcite/prepare/PlannerPhase.java | 6 +-
.../query/calcite/prepare/PlanningContext.java | 342 ++++
.../processors/query/calcite/prepare/Query.java | 93 -
.../{QueryExecution.java => QueryPlanCache.java} | 17 +-
.../query/calcite/prepare/QueryPlanCacheImpl.java | 115 ++
.../{QueryExecution.java => QueryPlanFactory.java} | 14 +-
.../query/calcite/rel/IgniteReceiver.java | 9 +
.../DistributedTable.java} | 25 +-
.../query/calcite/schema/IgniteSchema.java | 31 +-
.../query/calcite/schema/IgniteTable.java | 195 ++-
.../GraphNode.java => schema/SchemaHolder.java} | 13 +-
...citeSchemaHolder.java => SchemaHolderImpl.java} | 69 +-
.../GraphNode.java => schema/SortedTable.java} | 13 +-
.../query/calcite/schema/TableDescriptor.java | 65 +
.../query/calcite/schema/TableDescriptorImpl.java | 131 ++
.../processors/query/calcite/serialize/Graph.java | 114 --
.../serialize/expression/CallExpression.java | 19 +-
.../expression/DynamicParamExpression.java | 3 +-
.../serialize/expression/ExpToRexTranslator.java | 2 +-
.../serialize/expression/InputRefExpression.java | 3 +-
.../serialize/expression/LiteralExpression.java | 10 +-
.../serialize/expression/LocalRefExpression.java | 3 +-
.../serialize/expression/RexToExpTranslator.java | 2 +-
.../serialize/relation/ConversionContext.java | 4 +-
.../calcite/serialize/relation/FilterNode.java | 5 +-
.../serialize/relation/GraphToRelConverter.java | 14 +-
.../query/calcite/serialize/relation/JoinNode.java | 5 +-
.../calcite/serialize/relation/ProjectNode.java | 8 +-
.../calcite/serialize/relation/ReceiverNode.java | 11 +-
.../query/calcite/serialize/relation/RelGraph.java | 92 +-
.../calcite/serialize/relation/RelGraphNode.java | 58 +-
.../calcite/serialize/relation/SenderNode.java | 5 +-
.../serialize/relation/SerializedTraits.java | 81 -
.../calcite/serialize/relation/TableScanNode.java | 14 +-
.../query/calcite/serialize/type/DataType.java | 16 -
.../query/calcite/serialize/type/StructType.java | 2 +-
.../serialize/type/{DataType.java => Types.java} | 21 +-
.../splitter/{Splitter.java => Cloner.java} | 114 +-
.../query/calcite/splitter/Fragment.java | 101 +-
.../query/calcite/splitter/QueryPlan.java | 37 +-
.../query/calcite/splitter/RelSource.java | 17 +-
.../query/calcite/splitter/RelSourceImpl.java | 12 +-
.../query/calcite/splitter/RelTarget.java | 10 +-
.../query/calcite/splitter/RelTargetImpl.java | 15 +-
.../query/calcite/splitter/Splitter.java | 62 +-
.../{DestinationFunction.java => AllNodes.java} | 31 +-
.../{DestinationFunction.java => Destination.java} | 6 +-
.../query/calcite/trait/DistributionFunction.java | 158 +-
.../query/calcite/trait/DistributionTrait.java | 7 +-
.../query/calcite/trait/IgniteDistributions.java | 24 +-
.../AbstractNode.java => trait/Partitioned.java} | 43 +-
.../{DestinationFunction.java => RandomNode.java} | 40 +-
.../query/calcite/type/IgniteTypeFactory.java | 77 +-
.../query/calcite/type/IgniteTypeSystem.java | 2 +-
.../processors/query/calcite/type/RowType.java | 213 ---
.../GraphNode.java => type/SystemType.java} | 8 +-
.../query/calcite/util/AbstractService.java} | 32 +-
.../processors/query/calcite/util/Commons.java | 111 +-
.../LifecycleAware.java} | 20 +-
.../query/calcite/util/ListFieldsQueryCursor.java | 37 +-
.../processors/query/calcite/util/NoOpPlanner.java | 228 +++
.../{exec/Source.java => util/Service.java} | 19 +-
.../processors/query/calcite/util/TableScan.java | 242 +++
.../query/calcite/util/TableScanIterator.java | 146 --
.../query/calcite/CalciteQueryProcessorTest.java | 1272 +-------------
.../processors/query/calcite/PlannerTest.java | 1815 ++++++++++++++++++++
.../query/calcite/exchange/OutboxTest.java | 185 --
.../query/calcite/exec/AbstractExecutionTest.java | 175 ++
.../calcite/exec/ContinuousExecutionTest.java | 146 ++
.../query/calcite/exec/ExecutionTest.java | 40 +-
.../processors/query/calcite/exec/OutboxTest.java | 179 ++
.../query/calcite/message/TestIoManager.java} | 25 +-
.../ignite/testsuites/IgniteCalciteTestSuite.java | 10 +-
modules/codegen/pom.xml | 6 +
.../ignite/codegen/MessageCodeGenerator.java | 3 +
.../ignite/internal/GridKernalContextImpl.java | 4 +-
.../ignite/internal/IgniteComponentType.java | 3 +-
.../org/apache/ignite/internal/IgniteKernal.java | 9 +-
.../managers/communication/GridIoManager.java | 2 +
.../managers/communication/GridIoPolicy.java | 3 +
.../internal/processors/pool/PoolProcessor.java | 4 +
.../processors/query/GridQueryProcessor.java | 31 +-
.../internal/processors/query/NoOpQueryEngine.java | 16 +-
.../internal/processors/query/QueryContext.java | 2 +-
.../internal/processors/query/QueryEngine.java | 16 +-
.../query/schema/SchemaChangeListener.java | 29 +-
.../apache/ignite/thread/SameThreadExecutor.java} | 28 +-
146 files changed, 9876 insertions(+), 4387 deletions(-)
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
index 9ab992b..fc87608 100644
--- 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
@@ -19,60 +19,53 @@ 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.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
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.exec.ExchangeService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistry;
+import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistryImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutorImpl;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCache;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCacheImpl;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolderImpl;
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.apache.ignite.internal.processors.query.calcite.util.LifecycleAware;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
import org.jetbrains.annotations.Nullable;
/**
*
*/
-public class CalciteQueryProcessor implements QueryEngine {
+public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
/** */
- private final CalciteSchemaHolder schemaHolder = new CalciteSchemaHolder();
-
- /** */
- private final FrameworkConfig config;
-
- /** */
- private IgniteLogger log;
-
- /** */
- private GridKernalContext kernalContext;
-
- /** */
- public CalciteQueryProcessor() {
- config = Frameworks.newConfigBuilder()
+ public static final FrameworkConfig FRAMEWORK_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)
+ .setLex(Lex.ORACLE)
.build())
// Dialects support.
.operatorTable(SqlLibraryOperatorTableFactory.INSTANCE
@@ -80,91 +73,201 @@ public class CalciteQueryProcessor implements QueryEngine {
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))
+ .context(Contexts.empty())
// Custom cost factory to use during optimization
.costFactory(null)
- .typeSystem(IgniteTypeSystem.DEFAULT)
+ .typeSystem(IgniteTypeSystem.INSTANCE)
.build();
+
+ /** */
+ private final QueryPlanCache queryPlanCache;
+
+ /** */
+ private final QueryTaskExecutor taskExecutor;
+
+ /** */
+ private final FailureProcessor failureProcessor;
+
+ /** */
+ private final PartitionService partitionService;
+
+ /** */
+ private final SchemaHolder schemaHolder;
+
+ /** */
+ private final MessageService messageService;
+
+ /** */
+ private final ExchangeService exchangeService;
+
+ /** */
+ private final MappingService mappingService;
+
+ /** */
+ private final MailboxRegistry mailboxRegistry;
+
+ /** */
+ private final ExecutionService executionService;
+
+ /**
+ * @param ctx Kernal context.
+ */
+ public CalciteQueryProcessor(GridKernalContext ctx) {
+ this(
+ ctx,
+ ctx.failure(),
+ new SchemaHolderImpl(ctx),
+ new QueryPlanCacheImpl(ctx),
+ new MailboxRegistryImpl(ctx),
+ new QueryTaskExecutorImpl(ctx),
+ new ExecutionServiceImpl(ctx),
+ new PartitionServiceImpl(ctx),
+ new MessageServiceImpl(ctx),
+ new MappingServiceImpl(ctx),
+ new ExchangeServiceImpl(ctx));
}
/**
- * @param log Logger.
+ * For tests purpose.
+ * @param ctx Kernal context.
+ * @param failureProcessor Failure processor.
+ * @param schemaHolder Schema holder.
+ * @param queryPlanCache Query cache;
+ * @param mailboxRegistry Mailbox registry.
+ * @param taskExecutor Task executor.
+ * @param executionService Execution service.
+ * @param partitionService Affinity service.
+ * @param messageService Message service.
+ * @param mappingService Mapping service.
+ * @param exchangeService Exchange service.
*/
- @LoggerResource
- public void setLogger(IgniteLogger log) {
- this.log = log;
+ CalciteQueryProcessor(GridKernalContext ctx, FailureProcessor failureProcessor, SchemaHolder schemaHolder, QueryPlanCache queryPlanCache, MailboxRegistry mailboxRegistry, QueryTaskExecutor taskExecutor, ExecutionService executionService, PartitionService partitionService, MessageService messageService,
+ MappingService mappingService, ExchangeService exchangeService) {
+ super(ctx);
+
+ this.failureProcessor = failureProcessor;
+ this.schemaHolder = schemaHolder;
+ this.queryPlanCache = queryPlanCache;
+ this.mailboxRegistry = mailboxRegistry;
+ this.taskExecutor = taskExecutor;
+ this.executionService = executionService;
+ this.partitionService = partitionService;
+ this.messageService = messageService;
+ this.mappingService = mappingService;
+ this.exchangeService = exchangeService;
}
- /** {@inheritDoc} */
- @Override public void start(@NotNull GridKernalContext ctx) {
- kernalContext = ctx;
+ public PartitionService affinityService() {
+ return partitionService;
+ }
- GridInternalSubscriptionProcessor prc = ctx.internalSubscriptionProcessor();
+ /**
+ * @return Query cache.
+ */
+ public QueryPlanCache queryPlanCache() {
+ return queryPlanCache;
+ }
- if (prc != null) // Stubbed context doesn't have such processor
- prc.registerSchemaChangeListener(schemaHolder);
+ /**
+ * @return Task executor.
+ */
+ public QueryTaskExecutor taskExecutor() {
+ return taskExecutor;
}
- /** {@inheritDoc} */
- @Override public void stop() {
+ /**
+ * @return Schema holder.
+ */
+ public SchemaHolder schemaHolder() {
+ return schemaHolder;
}
- /** {@inheritDoc} */
- @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);
+ /**
+ * @return Message service.
+ */
+ public MessageService messageService() {
+ return messageService;
}
/**
- * Creates a planner.
- *
- * @param traitDefs Trait definitions.
- * @param ctx Planner context.
- * @return Ignite planner.
+ * @return Mapping service.
*/
- public IgnitePlanner planner(RelTraitDef[] traitDefs, PlannerContext ctx) {
- FrameworkConfig cfg = Frameworks.newConfigBuilder(config)
- .defaultSchema(ctx.schema())
- .traitDefs(traitDefs)
- .context(ctx)
- .build();
+ public MappingService mappingService() {
+ return mappingService;
+ }
- return new IgnitePlanner(cfg);
+ /**
+ * @return Exchange service.
+ */
+ public ExchangeService exchangeService() {
+ return exchangeService;
}
/**
- * @param ctx External context.
- * @param query Query string.
- * @param params Query parameters.
- * @return Query execution context.
+ * @return Mailbox registry.
*/
- 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));
+ public MailboxRegistry mailboxRegistry() {
+ return mailboxRegistry;
}
- /** */
- 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();
+ /**
+ * @return Failure processor.
+ */
+ public FailureProcessor failureProcessor() {
+ return failureProcessor;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void start() {
+ onStart(ctx,
+ executionService,
+ mailboxRegistry,
+ partitionService,
+ schemaHolder,
+ messageService,
+ taskExecutor,
+ mappingService,
+ queryPlanCache,
+ exchangeService
+ );
+ }
+
+ /** {@inheritDoc} */
+ @Override public void stop(boolean cancel) {
+ onStop(
+ executionService,
+ mailboxRegistry,
+ partitionService,
+ schemaHolder,
+ messageService,
+ taskExecutor,
+ mappingService,
+ queryPlanCache,
+ exchangeService
+ );
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
+ String query, Object... params) throws IgniteSQLException {
+
+ // TODO multiline query.
+ return Collections.singletonList(executionService.executeQuery(qryCtx, schemaName, query, params));
}
/** */
- private QueryExecution prepare(PlannerContext ctx) {
- return new DistributedExecution(ctx);
+ private void onStart(GridKernalContext ctx, Service... services) {
+ for (Service service : services) {
+ if (service instanceof LifecycleAware)
+ ((LifecycleAware) service).onStart(ctx);
+ }
}
/** */
- private AffinityTopologyVersion readyAffinityVersion() {
- return kernalContext.cache().context().exchange().readyAffinityVersion();
+ private void onStop(Service... services) {
+ for (Service service : services) {
+ if (service instanceof LifecycleAware)
+ ((LifecycleAware) service).onStop();
+ }
}
}
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
deleted file mode 100644
index d420540..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.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;
-
- /**
- * @param ctx Grid kernal context.
- */
- public MappingServiceImpl(GridKernalContext ctx) {
- this.ctx = ctx;
- }
-
- /** {@inheritDoc} */
- @Override public NodesMapping local() {
- return new NodesMapping(Collections.singletonList(ctx.discovery().localNode().id()), null, DEDUPLICATED);
- }
-
- /** {@inheritDoc} */
- @Override public NodesMapping random(AffinityTopologyVersion topVer) {
- List<ClusterNode> nodes = ctx.discovery().discoCache(topVer).serverNodes();
-
- return new NodesMapping(Commons.transform(nodes, ClusterNode::id), null, DEDUPLICATED);
- }
-
- /** {@inheritDoc} */
- @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
- GridCacheContext<?,?> cctx = ctx.cache().context().cacheContext(cacheId);
-
- return cctx.isReplicated() ? replicatedLocation(cctx, topVer) : partitionedLocation(cctx, topVer);
- }
-
- /**
- * @param cctx Cache context.
- * @param topVer Topology version.
- * @return Node mapping, describing location of interested data.
- */
- 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);
- }
-
- /**
- * @param cctx Cache context.
- * @param topVer Topology version.
- * @return Node mapping, describing location of interested data.
- */
- 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);
- }
-
- /**
- * @param nodeId Node ID.
- * @param topology Topology version.
- * @param parts partitions count.
- * @return {@code True} if all partitions are in {@link GridDhtPartitionState#OWNING} state on the given node.
- */
- 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/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
deleted file mode 100644
index fd0cb12..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.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 https://issues.apache.org/jira/browse/IGNITE-12448
- */
-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;
-
- /**
- * @param queryId Query ID.
- * @param exchangeId Exchange ID.
- */
- public Inbox(GridCacheVersion queryId, long exchangeId) {
- this.queryId = queryId;
- this.exchangeId = exchangeId;
- }
-
- /**
- * Binds this Inbox to the given target.
- *
- * @param target Target sink.
- * @param sources Source nodes.
- * @param comparator Optional comparator for merge exchange.
- */
- public void bind(Sink<T> target, Collection<UUID> sources, Comparator<T> comparator) {
- this.target = target;
- this.sources = sources;
- this.comparator = comparator;
- }
-
- /**
- * @param srvc Exchange service.
- */
- void init(ExchangeProcessor srvc) {
- this.srvc = srvc;
- }
-
- /** {@inheritDoc} */
- @Override public void signal() {
- // No-op.
- }
-
- /** {@inheritDoc} */
- @Override public void sources(List<Source> sources) {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override public Sink<T> sink(int idx) {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Pushes a batch into a buffer.
- *
- * @param source Source node.
- * @param batchId Batch ID.
- * @param rows Rows.
- */
- 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
deleted file mode 100644
index b64573ff..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.exchange;
-
-import java.util.ArrayList;
-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;
-
-/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12448
- */
-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 DestinationFunction function;
-
- /** */
- private ExchangeProcessor srvc;
-
- /**
- *
- * @param queryId Query ID.
- * @param exchangeId Exchange ID.
- * @param function Destination function.
- */
- public Outbox(GridCacheVersion queryId, long exchangeId, DestinationFunction function) {
- super(Sink.noOp());
- this.queryId = queryId;
- this.exchangeId = exchangeId;
- 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).onAcknowledge(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 : function.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;
- }
-
- /**
- * Adds a row to current batch.
- *
- * @param row Row.
- */
- 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);
- }
-
- /**
- * Signals data is over.
- */
- public void end() {
- curr.add(EndMarker.INSTANCE);
-
- assert srvc != null;
-
- srvc.send(queryId, exchangeId, nodeId, hwm, curr);
-
- curr = null;
- hwm = Integer.MAX_VALUE;
- }
-
- /**
- * Checks whether there is a place for a new row.
- *
- * @return {@code True} is it possible to add a row to a batch.
- */
- boolean ready() {
- return hwm - lwm < ExchangeProcessor.PER_NODE_BATCH_COUNT || curr.size() < ExchangeProcessor.BATCH_SIZE;
- }
-
- /**
- * Callback method.
- *
- * @param id batch ID.
- */
- void onAcknowledge(int id) {
- if (lwm < id) {
- lwm = id;
-
- if (needSignal) {
- needSignal = false;
-
- signal();
- }
- }
- }
-
- /**
- * Sets "needSignal" flag.
- */
- 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
index 65fb965..8726333 100644
--- 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
@@ -17,40 +17,118 @@
package org.apache.ignite.internal.processors.query.calcite.exec;
-import java.util.Collections;
+import com.google.common.collect.ImmutableList;
+import java.util.Collection;
+import java.util.Comparator;
import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Abstract node of execution tree.
*/
public abstract class AbstractNode<T> implements Node<T> {
+ /** for debug purpose */
+ private volatile Thread thread;
+
/** */
- protected final Sink<T> target;
+ private final ImmutableList<Node<T>> inputs;
/** */
- protected List<Source> sources;
+ private Sink<T> target;
/**
- * @param target Target.
+ * {@link Inbox} node may not have proper context at creation time in case it
+ * creates on first message received from a remote source. This case the context
+ * sets in scope of {@link Inbox#init(ExecutionContext, Collection, Comparator)} method call.
*/
- protected AbstractNode(Sink<T> target) {
- this.target = target;
+ private ExecutionContext ctx;
+
+ /**
+ * @param ctx Execution context.
+ */
+ protected AbstractNode(ExecutionContext ctx) {
+ this(ctx, ImmutableList.of());
}
- /** {@inheritDoc} */
- @Override public void sources(List<Source> sources) {
- this.sources = Collections.unmodifiableList(sources);
+ /**
+ * @param ctx Execution context.
+ */
+ protected AbstractNode(ExecutionContext ctx, @NotNull Node<T> input) {
+ this(ctx, ImmutableList.of(input));
}
/**
- * @param idx Index of a source to signal/
+ * @param ctx Execution context.
*/
- public void signal(int idx) {
- sources.get(idx).signal();
+ protected AbstractNode(ExecutionContext ctx, @NotNull List<Node<T>> inputs) {
+ this.ctx = ctx;
+ this.inputs = ImmutableList.copyOf(inputs);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void target(Sink<T> sink) {
+ target = sink;
}
/** {@inheritDoc} */
- @Override public void signal() {
- sources.forEach(Source::signal);
+ @Override public Sink<T> target() {
+ return target;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Node<T>> inputs() {
+ return inputs;
+ }
+
+ /** */
+ protected void context(ExecutionContext ctx) {
+ this.ctx = ctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override public ExecutionContext context() {
+ return ctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void request() {
+ checkThread();
+
+ inputs().forEach(Node::request);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void cancel() {
+ checkThread();
+
+ context().setCancelled();
+ inputs().forEach(Node::cancel);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void reset() {
+ checkThread();
+
+ inputs().forEach(Node::reset);
+ }
+
+ /**
+ * Links the node inputs to the node sinks.
+ */
+ protected void link() {
+ for (int i = 0; i < inputs.size(); i++)
+ inputs.get(i).target(sink(i));
+ }
+
+ /** */
+ protected void checkThread() {
+ if (!U.assertionsEnabled())
+ return;
+
+ if (thread == null)
+ thread = Thread.currentThread();
+ else
+ assert thread == Thread.currentThread();
}
}
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
index 453dee7..a686866 100644
--- 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
@@ -20,26 +20,95 @@ 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;
+import java.util.UUID;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.util.typedef.internal.U;
/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Client iterator.
*/
-public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]> {
+public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]>, AutoCloseable {
+ /** */
+ public static final Consumer<ConsumerNode> NO_OP = c -> {};
+
/** */
private static final int DEFAULT_BUFFER_SIZE = 1000;
/** */
- private static final Object[] END = new Object[0];
+ private enum State {
+ RUNNING, CANCELLED, END
+ }
+
+ /** */
+ private final ReentrantLock lock;
+
+ /** */
+ private final Condition cond;
+
+ /** */
+ private final int bufferSize;
/** */
- private ArrayDeque<Object[]> buff;
+ private final ArrayDeque<Object> buff;
/** */
- public ConsumerNode() {
- super(Sink.noOp());
+ private final Consumer<ConsumerNode> onClose;
+
+ /** */
+ private Object cur;
+
+ /** */
+ private boolean requested;
+
+ /** */
+ private volatile State state = State.RUNNING;
+
+ /**
+ * @param ctx Execution context.
+ */
+ public ConsumerNode(ExecutionContext ctx, Node<Object[]> input) {
+ this(ctx, input, DEFAULT_BUFFER_SIZE);
+ }
- buff = new ArrayDeque<>(DEFAULT_BUFFER_SIZE);
+ /**
+ * @param ctx Execution context.
+ */
+ public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, Consumer<ConsumerNode> onClose) {
+ this(ctx, input, DEFAULT_BUFFER_SIZE, onClose);
+ }
+
+ /**
+ * @param ctx Execution context.
+ * @param input Input node.
+ * @param bufferSize Buffer size.
+ */
+ public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, int bufferSize) {
+ this(ctx, input, bufferSize, NO_OP);
+ }
+
+ /**
+ * @param ctx Execution context.
+ */
+ public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, int bufferSize, Consumer<ConsumerNode> onClose) {
+ super(ctx, input);
+
+ this.bufferSize = bufferSize;
+ this.onClose = onClose;
+
+ // extra space for possible END marker
+ buff = new ArrayDeque<>(bufferSize + 1);
+ lock = new ReentrantLock();
+ cond = lock.newCondition();
+
+ link();
+ }
+
+ public UUID queryId() {
+ return context().queryId();
}
/** {@inheritDoc} */
@@ -51,36 +120,165 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
}
/** {@inheritDoc} */
+ @Override public void request() {
+ context().execute(this::requestInternal);
+ }
+
+ /** */
+ private void requestInternal() {
+ checkThread();
+
+ if (state == State.RUNNING)
+ input().request();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void cancel() {
+ context().setCancelled();
+
+ if (state != State.RUNNING)
+ return;
+
+ lock.lock();
+ try {
+ if (state != State.RUNNING)
+ return;
+
+ state = State.CANCELLED;
+ buff.clear();
+ cond.signalAll();
+ }
+ finally {
+ lock.unlock();
+ }
+
+ context().execute(input()::cancel);
+ onClose.accept(this);
+ }
+
+ public boolean canceled() {
+ return state == State.CANCELLED;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ cancel();
+ }
+
+ /** {@inheritDoc} */
@Override public boolean push(Object[] row) {
- if (buff.size() == DEFAULT_BUFFER_SIZE)
+ if (state != State.RUNNING)
return false;
- buff.add(row);
+ lock.lock();
+ try {
+ if (state != State.RUNNING)
+ return false;
+
+ if (buff.size() == bufferSize) {
+ requested = false;
+ return false;
+ }
+
+ buff.offer(row);
+ cond.signalAll();
+ }
+ finally {
+ lock.unlock();
+ }
return true;
}
/** {@inheritDoc} */
@Override public void end() {
- buff.add(END);
+ if (state != State.RUNNING)
+ return;
+
+ lock.lock();
+ try {
+ if (state != State.RUNNING)
+ return;
+
+ buff.offer(EndMarker.INSTANCE);
+ cond.signalAll();
+ }
+ finally {
+ lock.unlock();
+ }
}
/** {@inheritDoc} */
@Override public boolean hasNext() {
- if (buff.isEmpty())
- signal();
-
- return buff.peek() != END;
+ if (cur != null)
+ return true;
+ else if (state == State.END)
+ return false;
+ else
+ return (cur = take()) != null;
}
/** {@inheritDoc} */
@Override public Object[] next() {
- if (buff.isEmpty())
- signal();
-
- if(!hasNext())
+ if (!hasNext())
throw new NoSuchElementException();
- return Objects.requireNonNull(buff.poll());
+ Object cur0 = cur;
+ cur = null;
+
+ return (Object[]) cur0;
+ }
+
+ /** */
+ private Object take() {
+ if (state == State.CANCELLED)
+ throw U.convertException(new QueryCancelledException());
+
+ lock.lock();
+ try {
+ if (state == State.CANCELLED)
+ throw U.convertException(new QueryCancelledException());
+
+ assert state == State.RUNNING;
+
+ while (buff.isEmpty()) {
+ requestIfNeeded();
+
+ cond.await();
+
+ if (state == State.CANCELLED)
+ throw U.convertException(new QueryCancelledException());
+
+ assert state == State.RUNNING;
+ }
+
+ Object row = buff.poll();
+
+ if (row != EndMarker.INSTANCE)
+ return row;
+
+ state = State.END;
+ }
+ catch (InterruptedException e) {
+ throw new IgniteInterruptedException(e);
+ }
+ finally {
+ lock.unlock();
+ }
+
+ assert state == State.END;
+
+ onClose.accept(this);
+
+ return null;
+ }
+
+ /** */
+ private void requestIfNeeded() {
+ if (requested)
+ return;
+
+ request();
+ requested = 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/exec/EndMarker.java
similarity index 94%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/EndMarker.java
index d4368c7..377e116 100644
--- 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/exec/EndMarker.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.exchange;
+package org.apache.ignite.internal.processors.query.calcite.exec;
import java.io.ObjectStreamException;
import java.io.Serializable;
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/exec/ExchangeService.java
similarity index 53%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
index 1fb2beb..24b5383 100644
--- 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/exec/ExchangeService.java
@@ -15,16 +15,16 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.exchange;
+package org.apache.ignite.internal.processors.query.calcite.exec;
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.util.Service;
/**
*
*/
-public interface ExchangeProcessor {
+public interface ExchangeService extends Service {
/** A number of rows in a single batch. */
int BATCH_SIZE = 200;
@@ -32,50 +32,39 @@ public interface ExchangeProcessor {
int PER_NODE_BATCH_COUNT = 10;
/**
- * Registers an outbox in Exchange service.
- * Registered outbox will be notified each time a consumer processed a batch of rows.
- * @param outbox Outbox.
- * @return Registered outbox.
- */
- <T> Outbox<T> register(Outbox<T> outbox);
-
- /**
- * Unregisters an outbox in Exchange service.
- * @param outbox Outbox to unregister.
- */
- <T> void unregister(Outbox<T> outbox);
-
- /**
- * Registers an inbox in Exchange service.
- * Registered inbox starts consuming data from remote sources.
- * In case an inbox with the same [queryId, exchangeId] pair is already registered, previously registered inbox is return.
- * @param inbox Inbox.
- * @return Registered inbox.
- */
- <T> Inbox<T> register(Inbox<T> inbox);
-
- /**
- * Unregisters an inbox in Exchange service.
- * @param inbox Inbox to unregister.
- */
- <T> void unregister(Inbox<T> inbox);
-
- /**
* Sends a batch of data to remote node.
+ *
+ * @param caller Caller.
+ * @param nodeId Target node ID.
* @param queryId Query ID.
+ * @param fragmentId Target fragment ID.
* @param exchangeId Exchange ID.
- * @param nodeId Target node ID.
* @param batchId Batch ID.
* @param rows Data rows.
*/
- void send(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId, List<?> rows);
+ void sendBatch(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows);
/**
* Acknowledges a batch with given ID is processed.
+ *
+ * @param caller Caller.
+ * @param nodeId Node ID to notify.
* @param queryId Query ID.
+ * @param fragmentId Target fragment ID.
+ * @param exchangeId Exchange ID.
+ * @param batchId Batch ID.
+ */
+ void acknowledge(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId);
+
+ /**
+ * Sends cancel request.
+ *
+ * @param caller Caller.
+ * @param nodeId Target node ID.
+ * @param queryId Query ID.
+ * @param fragmentId Target fragment ID.
* @param exchangeId Exchange ID.
- * @param nodeId Node ID to notify.
* @param batchId Batch ID.
*/
- void acknowledge(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId);
+ void cancel(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
new file mode 100644
index 0000000..7559f06
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
@@ -0,0 +1,190 @@
+/*
+ * 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.exec;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.message.InboxCancelMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageType;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryBatchAcknowledgeMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryBatchMessage;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class ExchangeServiceImpl extends AbstractService implements ExchangeService {
+ /** */
+ private QueryTaskExecutor taskExecutor;
+
+ /** */
+ private MailboxRegistry mailboxRegistry;
+
+ /** */
+ private MessageService messageService;
+
+ /**
+ * @param ctx Kernal context.
+ */
+ public ExchangeServiceImpl(GridKernalContext ctx) {
+ super(ctx);
+ }
+
+ /**
+ * @param taskExecutor Task executor.
+ */
+ public void taskExecutor(QueryTaskExecutor taskExecutor) {
+ this.taskExecutor = taskExecutor;
+ }
+
+ /**
+ * @return Task executor.
+ */
+ public QueryTaskExecutor taskExecutor() {
+ return taskExecutor;
+ }
+
+ /**
+ * @param mailboxRegistry Mailbox registry.
+ */
+ public void mailboxRegistry(MailboxRegistry mailboxRegistry) {
+ this.mailboxRegistry = mailboxRegistry;
+ }
+
+ /**
+ * @return Mailbox registry.
+ */
+ public MailboxRegistry mailboxRegistry() {
+ return mailboxRegistry;
+ }
+
+ /**
+ * @param messageService Message service.
+ */
+ public void messageService(MessageService messageService) {
+ this.messageService = messageService;
+ }
+
+ /**
+ * @return Message service.
+ */
+ public MessageService messageService() {
+ return messageService;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void sendBatch(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) {
+ messageService().send(nodeId, new QueryBatchMessage(queryId, fragmentId, exchangeId, batchId, rows));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void acknowledge(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
+ messageService().send(nodeId, new QueryBatchAcknowledgeMessage(queryId, fragmentId, exchangeId, batchId));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void cancel(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
+ messageService().send(nodeId, new InboxCancelMessage(queryId, fragmentId, exchangeId, batchId));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ CalciteQueryProcessor proc = Objects.requireNonNull(Commons.lookupComponent(ctx, CalciteQueryProcessor.class));
+
+ taskExecutor(proc.taskExecutor());
+ mailboxRegistry(proc.mailboxRegistry());
+ messageService(proc.messageService());
+
+ init();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void init() {
+ messageService().register((n, m) -> onMessage(n, (InboxCancelMessage) m), MessageType.QUERY_INBOX_CANCEL_MESSAGE);
+ messageService().register((n, m) -> onMessage(n, (QueryBatchAcknowledgeMessage) m), MessageType.QUERY_ACKNOWLEDGE_MESSAGE);
+ messageService().register((n, m) -> onMessage(n, (QueryBatchMessage) m), MessageType.QUERY_BATCH_MESSAGE);
+ }
+
+ /** */
+ protected void onMessage(UUID nodeId, InboxCancelMessage msg) {
+ Inbox<?> inbox = mailboxRegistry().inbox(msg.queryId(), msg.exchangeId());
+
+ if (inbox != null)
+ inbox.cancel();
+ else if (log.isDebugEnabled()) {
+ log.debug("Stale cancel message received: [" +
+ "nodeId=" + nodeId + ", " +
+ "queryId=" + msg.queryId() + ", " +
+ "fragmentId=" + msg.fragmentId() + ", " +
+ "exchangeId=" + msg.exchangeId() + ", " +
+ "batchId=" + msg.batchId() + "]");
+ }
+ }
+
+ /** */
+ protected void onMessage(UUID nodeId, QueryBatchAcknowledgeMessage msg) {
+ Outbox<?> outbox = mailboxRegistry().outbox(msg.queryId(), msg.exchangeId());
+
+ if (outbox != null)
+ outbox.onAcknowledge(nodeId, msg.batchId());
+ else if (log.isDebugEnabled()) {
+ log.debug("Stale acknowledge message received: [" +
+ "nodeId=" + nodeId + ", " +
+ "queryId=" + msg.queryId() + ", " +
+ "fragmentId=" + msg.fragmentId() + ", " +
+ "exchangeId=" + msg.exchangeId() + ", " +
+ "batchId=" + msg.batchId() + "]");
+ }
+ }
+
+ /** */
+ protected void onMessage(UUID nodeId, QueryBatchMessage msg) {
+ Inbox<?> inbox = mailboxRegistry().inbox(msg.queryId(), msg.exchangeId());
+
+ if (inbox == null && msg.batchId() == 0)
+ // first message sent before a fragment is built
+ // note that an inbox source fragment id is also used as an exchange id
+ inbox = mailboxRegistry().register(new Inbox<>(this, mailboxRegistry(), baseInboxContext(msg.queryId(), msg.fragmentId()), msg.exchangeId(), msg.exchangeId()));
+
+ if (inbox != null)
+ inbox.onBatchReceived(nodeId, msg.batchId(), msg.rows());
+ else if (log.isDebugEnabled()){
+ log.debug("Stale batch message received: [" +
+ "nodeId=" + nodeId + ", " +
+ "queryId=" + msg.queryId() + ", " +
+ "fragmentId=" + msg.fragmentId() + ", " +
+ "exchangeId=" + msg.exchangeId() + ", " +
+ "batchId=" + msg.batchId() + "]");
+ }
+ }
+
+ /**
+ * @return Minimal execution context to meet Inbox needs.
+ */
+ private ExecutionContext baseInboxContext(UUID queryId, long fragmentId) {
+ PlanningContext ctx = PlanningContext.builder().logger(log).build();
+ return new ExecutionContext(taskExecutor(), ctx, queryId, fragmentId, null, ImmutableMap.of());
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
new file mode 100644
index 0000000..0b4177a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
@@ -0,0 +1,154 @@
+/*
+ * 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.exec;
+
+import java.util.Map;
+import java.util.UUID;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+
+/**
+ * Runtime context allowing access to the tables in a database.
+ */
+public class ExecutionContext implements DataContext {
+ /** */
+ private final UUID queryId;
+
+ /** */
+ private final long fragmentId;
+
+ /** */
+ private final PlanningContext ctx;
+
+ /** */
+ private final int[] parts;
+
+ /** */
+ private final Map<String, Object> params;
+
+ /** */
+ private final QueryTaskExecutor executor;
+
+ /** */
+ private volatile boolean cancelled;
+
+ /**
+ * @param ctx Parent context.
+ * @param queryId Query ID.
+ * @param fragmentId Fragment ID.
+ * @param parts Partitions.
+ * @param params Parameters.
+ */
+ public ExecutionContext(QueryTaskExecutor executor, PlanningContext ctx, UUID queryId, long fragmentId, int[] parts, Map<String, Object> params) {
+ this.executor = executor;
+ this.queryId = queryId;
+ this.fragmentId = fragmentId;
+ this.parts = parts;
+ this.params = params;
+ this.ctx = ctx;
+ }
+
+ /**
+ * @return Parent context.
+ */
+ public PlanningContext parent() {
+ return ctx;
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return queryId;
+ }
+
+ /**
+ * @return Fragment ID.
+ */
+ public long fragmentId() {
+ return fragmentId;
+ }
+
+ /**
+ * @return Interested partitions.
+ */
+ public int[] partitions() {
+ return parts;
+ }
+
+ /**
+ * @return Keep binary flag.
+ */
+ public boolean keepBinary() {
+ return false; // TODO
+ }
+
+ /**
+ * @return MVCC snapshot.
+ */
+ public MvccSnapshot mvccSnapshot() {
+ return null; // TODO
+ }
+
+ /**
+ * @return Cancelled flag.
+ */
+ public boolean cancelled() {
+ return cancelled;
+ }
+
+ /** {@inheritDoc} */
+ @Override public SchemaPlus getRootSchema() {
+ return ctx.schema();
+ }
+
+ /** {@inheritDoc} */
+ @Override public JavaTypeFactory getTypeFactory() {
+ return ctx.typeFactory();
+ }
+
+ /** {@inheritDoc} */
+ @Override public QueryProvider getQueryProvider() {
+ return null; // TODO
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object get(String name) {
+ return params.get(name);
+ }
+
+ /**
+ * Sets cancelled flag.
+ */
+ public void setCancelled() {
+ cancelled = true;
+ }
+
+ /**
+ * Executes a query task.
+ *
+ * @param task Query task.
+ */
+ public void execute(Runnable task) {
+ executor.execute(queryId, fragmentId, task);
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
similarity index 59%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
index d0e9f04..16afe41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
@@ -15,32 +15,34 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.processors.query.calcite.exec;
import java.util.List;
+import java.util.UUID;
import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
import org.jetbrains.annotations.Nullable;
/**
*
*/
-public interface QueryEngine {
-
+public interface ExecutionService extends Service {
/**
- * @param ctx Kernal context.
+ * Executes a query.
+ *
+ * @param ctx Query external context, contains flags and connection settings like a locale or a timezone.
+ * @param schema Schema name.
+ * @param query Query.
+ * @param params Query parameters.
+ * @return Query cursor.
*/
- void start(GridKernalContext ctx);
-
- /** */
- void stop();
+ FieldsQueryCursor<List<?>> executeQuery(@Nullable QueryContext ctx, String schema, String query, Object[] params);
/**
- * @param ctx Query context, may be null.
- * @param query Query.
- * @param params Optional query parameters.
- * @return Query cursor.
- * @throws IgniteSQLException If failed.
+ * Cancels a running query.
+ *
+ * @param queryId Query ID.
*/
- List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String query, Object... params) throws IgniteSQLException;
+ void cancelQuery(UUID queryId);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
new file mode 100644
index 0000000..e4e1fd5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -0,0 +1,825 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+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.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryCancellable;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageType;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryCancelRequest;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryStartRequest;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryStartResponse;
+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.metadata.PartitionService;
+import org.apache.ignite.internal.processors.query.calcite.prepare.CacheKey;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+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.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCache;
+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.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
+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.serialize.relation.SenderNode;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Fragment;
+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.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.ListFieldsQueryCursor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ *
+ */
+public class ExecutionServiceImpl extends AbstractService implements ExecutionService {
+ /** */
+ private final DiscoveryEventListener discoLsnr;
+
+ /** */
+ private UUID localNodeId;
+
+ /** */
+ private GridEventStorageManager eventManager;
+
+ /** */
+ private GridCachePartitionExchangeManager<?,?> exchangeManager;
+
+ /** */
+ private QueryPlanCache queryPlanCache;
+
+ /** */
+ private SchemaHolder schemaHolder;
+
+ /** */
+ private QueryTaskExecutor taskExecutor;
+
+ /** */
+ private FailureProcessor failureProcessor;
+
+ /** */
+ private PartitionService partitionService;
+
+ /** */
+ private MailboxRegistry mailboxRegistry;
+
+ /** */
+ private MappingService mappingService;
+
+ /** */
+ private MessageService messageService;
+
+ /** */
+ private ExchangeService exchangeService;
+
+ /** */
+ private final Map<UUID, QueryInfo> running;
+
+ /**
+ * @param ctx Kernal.
+ */
+ public ExecutionServiceImpl(GridKernalContext ctx) {
+ super(ctx);
+
+ discoLsnr = (e, c) -> onNodeLeft(e.eventNode().id());
+ running = new ConcurrentHashMap<>();
+ }
+
+ /**
+ * @param localNodeId Local node ID.
+ */
+ public void localNodeId(UUID localNodeId) {
+ this.localNodeId = localNodeId;
+ }
+
+ /**
+ * @return Local node ID.
+ */
+ public UUID localNodeId() {
+ return localNodeId;
+ }
+
+ /**
+ * @param queryPlanCache Query cache.
+ */
+ public void queryPlanCache(QueryPlanCache queryPlanCache) {
+ this.queryPlanCache = queryPlanCache;
+ }
+
+ /**
+ * @return Query cache.
+ */
+ public QueryPlanCache queryPlanCache() {
+ return queryPlanCache;
+ }
+
+ /**
+ * @param schemaHolder Schema holder.
+ */
+ public void schemaHolder(SchemaHolder schemaHolder) {
+ this.schemaHolder = schemaHolder;
+ }
+
+ /**
+ * @return Schema holder.
+ */
+ public SchemaHolder schemaHolder() {
+ return schemaHolder;
+ }
+
+ /**
+ * @param taskExecutor Task executor.
+ */
+ public void taskExecutor(QueryTaskExecutor taskExecutor) {
+ this.taskExecutor = taskExecutor;
+ }
+
+ /**
+ * @return Task executor.
+ */
+ public QueryTaskExecutor taskExecutor() {
+ return taskExecutor;
+ }
+
+ /**
+ * @param failureProcessor Failure processor.
+ */
+ public void failureProcessor(FailureProcessor failureProcessor) {
+ this.failureProcessor = failureProcessor;
+ }
+
+ /**
+ * @return Failure processor.
+ */
+ public FailureProcessor failureProcessor() {
+ return failureProcessor;
+ }
+
+ /**
+ * @param partitionService Partition service.
+ */
+ public void partitionService(PartitionService partitionService) {
+ this.partitionService = partitionService;
+ }
+
+ /**
+ * @return Partition service.
+ */
+ public PartitionService partitionService() {
+ return partitionService;
+ }
+
+ /**
+ * @param mailboxRegistry Mailbox registry.
+ */
+ public void mailboxRegistry(MailboxRegistry mailboxRegistry) {
+ this.mailboxRegistry = mailboxRegistry;
+ }
+
+ /**
+ * @return Mailbox registry.
+ */
+ public MailboxRegistry mailboxRegistry() {
+ return mailboxRegistry;
+ }
+
+ /**
+ * @param mappingService Mapping service.
+ */
+ public void mappingService(MappingService mappingService) {
+ this.mappingService = mappingService;
+ }
+
+ /**
+ * @return Mapping service.
+ */
+ public MappingService mappingService() {
+ return mappingService;
+ }
+
+ /**
+ * @param messageService Message service.
+ */
+ public void messageService(MessageService messageService) {
+ this.messageService = messageService;
+ }
+
+ /**
+ * @return Message service.
+ */
+ public MessageService messageService() {
+ return messageService;
+ }
+
+ /**
+ * @param exchangeService Exchange service.
+ */
+ public void exchangeService(ExchangeService exchangeService) {
+ this.exchangeService = exchangeService;
+ }
+
+ /**
+ * @return Exchange service.
+ */
+ public ExchangeService exchangeService() {
+ return exchangeService;
+ }
+
+ /**
+ * @param eventManager Event manager.
+ */
+ public void eventManager(GridEventStorageManager eventManager) {
+ this.eventManager = eventManager;
+ }
+
+ /**
+ * @return Event manager.
+ */
+ public GridEventStorageManager eventManager() {
+ return eventManager;
+ }
+
+ /**
+ * @param exchangeManager Exchange manager.
+ */
+ public void exchangeManager(GridCachePartitionExchangeManager<?,?> exchangeManager) {
+ this.exchangeManager = exchangeManager;
+ }
+
+ /**
+ * @return Exchange manager.
+ */
+ public GridCachePartitionExchangeManager<?, ?> exchangeManager() {
+ return exchangeManager;
+ }
+
+ /** {@inheritDoc} */
+ @Override public FieldsQueryCursor<List<?>> executeQuery(@Nullable QueryContext ctx, String schema, String query, Object[] params) {
+ UUID queryId = UUID.randomUUID();
+
+ PlanningContext pctx = createContext(ctx, schema, query, params);
+
+ QueryPlan plan = prepare(pctx);
+
+ // Local execution
+ Fragment local = F.first(plan.fragments());
+
+ if (U.assertionsEnabled()) {
+ assert local != null;
+
+ NodesMapping mapping = local.mapping();
+
+ assert mapping != null;
+
+ List<UUID> nodes = mapping.nodes();
+
+ assert nodes != null && nodes.size() == 1 && F.first(nodes).equals(pctx.localNodeId());
+ }
+
+ ExecutionContext ectx = new ExecutionContext(
+ taskExecutor(), pctx, queryId, local.fragmentId(), local.mapping().partitions(pctx.localNodeId()), Commons.parametersMap(params)
+ );
+
+ Node<Object[]> node = new Implementor(partitionService(), mailboxRegistry(), exchangeService(), failureProcessor(), ectx, log).go(local.root());
+
+ assert !(node instanceof SenderNode);
+
+ QueryInfo info = new QueryInfo(ectx, local.root().getRowType(), new ConsumerNode(ectx, node, this::onConsumerClose));
+
+ if (plan.fragments().size() == 1)
+ running.put(queryId, info);
+ else {
+ // remote execution
+ RelOp<IgniteRel, RelGraph> converter = new RelToGraphConverter();
+ List<Pair<UUID, QueryStartRequest>> requests = new ArrayList<>();
+
+ for (Fragment remote : plan.fragments().subList(1, plan.fragments().size())) {
+ long id = remote.fragmentId();
+ NodesMapping mapping = remote.mapping();
+ RelGraph graph = converter.go(remote.root());
+
+ for (UUID nodeId : mapping.nodes()) {
+ info.addFragment(nodeId, id);
+
+ QueryStartRequest req = new QueryStartRequest(
+ queryId,
+ id,
+ schema,
+ graph,
+ pctx.topologyVersion(),
+ mapping.partitions(nodeId),
+ params);
+
+ requests.add(Pair.of(nodeId, req));
+ }
+ }
+
+ running.put(queryId, info);
+
+ // start remote execution
+ for (Pair<UUID, QueryStartRequest> pair : requests)
+ messageService().send(pair.left, pair.right);
+ }
+
+ // start local execution
+ info.consumer.request();
+
+ info.awaitAllReplies();
+
+ // TODO weak map to stop query on cursor collected by GC.
+ return new ListFieldsQueryCursor<>(info.type(), info.<Object[]>iterator(), Arrays::asList);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void cancelQuery(UUID queryId) {
+ mailboxRegistry().outboxes(queryId).forEach(Outbox::cancel);
+
+ QueryInfo info = running.get(queryId);
+
+ if (info != null)
+ info.cancel();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ localNodeId(ctx.localNodeId());
+ exchangeManager(ctx.cache().context().exchange());
+ eventManager(ctx.event());
+
+ CalciteQueryProcessor proc = Objects.requireNonNull(Commons.lookupComponent(ctx, CalciteQueryProcessor.class));
+
+ queryPlanCache(proc.queryPlanCache());
+ schemaHolder(proc.schemaHolder());
+ taskExecutor(proc.taskExecutor());
+ failureProcessor(proc.failureProcessor());
+ partitionService(proc.affinityService());
+ mailboxRegistry(proc.mailboxRegistry());
+ mappingService(proc.mappingService());
+ messageService(proc.messageService());
+ exchangeService(proc.exchangeService());
+
+ init();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void init() {
+ messageService().register((n,m) -> onMessage(n, (QueryStartRequest) m), MessageType.QUERY_START_REQUEST);
+ messageService().register((n,m) -> onMessage(n, (QueryStartResponse) m), MessageType.QUERY_START_RESPONSE);
+ messageService().register((n,m) -> onMessage(n, (QueryCancelRequest) m), MessageType.QUERY_CANCEL_REQUEST);
+
+ eventManager().addDiscoveryEventListener(discoLsnr, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void tearDown() {
+ eventManager().removeDiscoveryEventListener(discoLsnr, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
+
+ running.clear();
+ }
+
+ /** */
+ protected AffinityTopologyVersion topologyVersion() {
+ return exchangeManager().readyAffinityVersion();
+ }
+
+ /** */
+ private QueryPlan prepare(PlanningContext ctx) {
+ CacheKey cacheKey = new CacheKey(ctx.schema().getName(), ctx.query());
+
+ QueryPlan plan = queryPlanCache().queryPlan(ctx, cacheKey, this::prepare0);
+
+ plan.init(mappingService(), ctx);
+
+ return plan;
+ }
+
+ /** */
+ private QueryPlan prepare0(PlanningContext ctx) {
+ IgnitePlanner planner = ctx.planner();
+
+ try {
+ String query = ctx.query();
+
+ assert query != null;
+
+ // Parse
+ SqlNode sqlNode = planner.parse(query);
+
+ // 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();
+
+ IgniteRel igniteRel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+ // Split query plan to multi-step one.
+ return new Splitter().go(igniteRel);
+ }
+ catch (SqlParseException e) {
+ IgniteSQLException ex = new IgniteSQLException("Failed to parse query.", IgniteQueryErrorCode.PARSING, e);
+ Commons.close(planner, ex);
+ throw ex;
+ }
+ catch (ValidationException e) {
+ IgniteSQLException ex = new IgniteSQLException("Failed to validate query.", IgniteQueryErrorCode.UNKNOWN, e);
+ Commons.close(planner, ex);
+ throw ex;
+ }
+ catch (Exception e) {
+ IgniteSQLException ex = new IgniteSQLException("Failed to plan query.", IgniteQueryErrorCode.UNKNOWN, e);
+ Commons.close(planner, ex);
+ throw ex;
+ }
+ }
+
+ /** */
+ private PlanningContext createContext(@Nullable QueryContext qryCtx, @Nullable String schemaName, String query, Object[] params) {
+ RelTraitDef<?>[] traitDefs = {
+ ConventionTraitDef.INSTANCE
+ , DistributionTraitDef.INSTANCE
+ //, RelCollationTraitDef.INSTANCE TODO
+ };
+
+ return PlanningContext.builder()
+ .localNodeId(localNodeId())
+ .parentContext(Commons.convert(qryCtx)) // TODO Connection config on the basis of query context
+ .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+ .defaultSchema(schemaName != null
+ ? schemaHolder().schema().getSubSchema(schemaName)
+ : schemaHolder().schema())
+ .traitDefs(traitDefs)
+ .build())
+ .query(query)
+ .parameters(params)
+ .topologyVersion(topologyVersion())
+ .logger(log)
+ .build();
+ }
+
+ /** */
+ private PlanningContext createContext(@Nullable String schemaName, UUID originatingNodeId, AffinityTopologyVersion topVer) {
+ // TODO pass to context user locale and timezone.
+
+ RelTraitDef<?>[] traitDefs = {
+ ConventionTraitDef.INSTANCE
+ //, RelCollationTraitDef.INSTANCE TODO
+ };
+
+ return PlanningContext.builder()
+ .localNodeId(localNodeId())
+ .originatingNodeId(originatingNodeId)
+ .parentContext(Contexts.empty())
+ .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+ .defaultSchema(schemaName != null
+ ? schemaHolder().schema().getSubSchema(schemaName)
+ : schemaHolder().schema())
+ .traitDefs(traitDefs)
+ .build())
+ .topologyVersion(topVer)
+ .logger(log)
+ .build();
+ }
+
+ /** */
+ protected void onMessage(UUID nodeId, QueryStartRequest msg) {
+ assert nodeId != null && msg != null;
+
+ PlanningContext ctx = createContext(msg.schema(), nodeId, msg.topologyVersion());
+
+ try (IgnitePlanner planner = ctx.planner()) {
+ IgniteRel root = planner.convert(msg.plan());
+
+ assert root instanceof IgniteSender : root;
+
+ // TODO do we need a local optimisation phase here?
+ ExecutionContext execCtx = new ExecutionContext(
+ taskExecutor(),
+ ctx,
+ msg.queryId(),
+ msg.fragmentId(),
+ msg.partitions(),
+ Commons.parametersMap(msg.parameters())
+ );
+
+ Node<Object[]> node = new Implementor(partitionService(), mailboxRegistry(), exchangeService(), failureProcessor(), execCtx, log).go(root);
+
+ assert node instanceof Outbox : node;
+
+ node.request();
+
+ messageService().send(nodeId, new QueryStartResponse(msg.queryId(), msg.fragmentId()));
+ }
+ catch (Exception ex) {
+ messageService().send(nodeId, new QueryStartResponse(msg.queryId(), msg.fragmentId(), ex));
+ }
+ }
+
+ /** */
+ private void onMessage(UUID nodeId, QueryCancelRequest msg) {
+ assert nodeId != null && msg != null;
+
+ cancelQuery(msg.queryId());
+ }
+
+ /** */
+ private void onMessage(UUID nodeId, QueryStartResponse msg) {
+ assert nodeId != null && msg != null;
+
+ QueryInfo info = running.get(msg.queryId());
+
+ if (info != null)
+ info.onResponse(nodeId, msg.fragmentId(), msg.error());
+ }
+
+ /** */
+ private void onConsumerClose(ConsumerNode consumer) {
+ if (consumer.canceled())
+ cancelQuery(consumer.queryId());
+ else
+ running.remove(consumer.queryId());
+ }
+
+ /** */
+ private void onNodeLeft(UUID nodeId) {
+ running.forEach((uuid, queryInfo) -> queryInfo.onNodeLeft(nodeId));
+ }
+
+ /** */
+ private static class RemoteFragmentKey {
+ /** */
+ private final UUID nodeId;
+
+ /** */
+ private final long fragmentId;
+
+ /** */
+ private RemoteFragmentKey(UUID nodeId, long fragmentId) {
+ this.nodeId = nodeId;
+ this.fragmentId = fragmentId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ RemoteFragmentKey that = (RemoteFragmentKey) o;
+
+ if (fragmentId != that.fragmentId)
+ return false;
+ return nodeId.equals(that.nodeId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ int result = nodeId.hashCode();
+ result = 31 * result + (int) (fragmentId ^ (fragmentId >>> 32));
+ return result;
+ }
+ }
+
+ /** */
+ private enum QueryState {
+ RUNNING, CANCELLING, CANCELLED
+ }
+
+ /** */
+ private final class QueryInfo implements QueryCancellable {
+ /** */
+ private final ExecutionContext ctx;
+
+ /** */
+ private final RelDataType type;
+
+ /** */
+ private final ConsumerNode consumer;
+
+ /** remote nodes */
+ private final Set<UUID> remotes;
+
+ /** node to fragment */
+ private final Set<RemoteFragmentKey> waiting;
+
+ /** */
+ private QueryState state;
+
+ /** */
+ private Throwable error;
+
+ /** */
+ private QueryInfo(ExecutionContext ctx, RelDataType type, ConsumerNode consumer) {
+ this.ctx = ctx;
+ this.type = type;
+ this.consumer = consumer;
+
+ remotes = new HashSet<>();
+ waiting = new HashSet<>();
+
+ state = QueryState.RUNNING;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void doCancel() {
+ cancel();
+ }
+
+ /** */
+ private ConsumerNode localNode() {
+ return consumer;
+ }
+
+ /** */
+ private RelDataType type() {
+ return type;
+ }
+
+ /** */
+ private <T> Iterator<T> iterator() {
+ return (Iterator<T>) consumer;
+ }
+
+ /** */
+ private void addFragment(UUID nodeId, long fragmentId) {
+ remotes.add(nodeId);
+ waiting.add(new RemoteFragmentKey(nodeId, fragmentId));
+ }
+
+ /** */
+ private void awaitAllReplies() {
+ Throwable error;
+
+ try {
+ synchronized (this) {
+ while (!waiting.isEmpty())
+ wait();
+
+ error = this.error;
+ }
+ }
+ catch (InterruptedException e) {
+ throw new IgniteInterruptedException(e);
+ }
+
+ if (error != null)
+ throw new IgniteSQLException("Failed to execute query.", error);
+ }
+
+ /** */
+ private void cancel() {
+ boolean cancelLocal = false;
+ boolean cancelRemote = false;
+ QueryState state0 = null;
+
+ synchronized (this) {
+ if (state == QueryState.CANCELLED)
+ return;
+
+ if (state == QueryState.RUNNING) {
+ cancelLocal = true;
+ state0 = state = QueryState.CANCELLING;
+ }
+
+ if (state == QueryState.CANCELLING && waiting.isEmpty()) {
+ cancelRemote = true;
+ state0 = state = QueryState.CANCELLED;
+ }
+ }
+
+ if (cancelLocal)
+ consumer.cancel();
+
+ if (cancelRemote)
+ messageService().send(remotes, new QueryCancelRequest(ctx.queryId()));
+
+ if (state0 == QueryState.CANCELLED)
+ running.remove(ctx.queryId());
+ }
+
+ /** */
+ private void onNodeLeft(UUID nodeId) {
+ List<RemoteFragmentKey> fragments = null;
+
+ synchronized (this) {
+ for (RemoteFragmentKey fragment : waiting) {
+ if (!fragment.nodeId.equals(nodeId))
+ continue;
+
+ if (fragments == null)
+ fragments = new ArrayList<>();
+
+ fragments.add(fragment);
+ }
+ }
+
+ if (!F.isEmpty(fragments)) {
+ ClusterTopologyCheckedException ex = new ClusterTopologyCheckedException("Failed to start query, node left. nodeId=" + nodeId);
+
+ for (RemoteFragmentKey fragment : fragments)
+ onResponse(fragment, ex);
+ }
+ }
+
+ /** */
+ private void onResponse(UUID nodeId, long fragmentId, Throwable error) {
+ onResponse(new RemoteFragmentKey(nodeId, fragmentId), error);
+ }
+
+ /** */
+ private void onResponse(RemoteFragmentKey fragment, Throwable error) {
+ boolean cancel;
+
+ synchronized (this) {
+ if (!waiting.remove(fragment))
+ return;
+
+ if (error != null) {
+ if (this.error != null)
+ this.error.addSuppressed(error);
+ else
+ this.error = error;
+ }
+
+ boolean empty = waiting.isEmpty();
+
+ cancel = empty && this.error != null;
+
+ if (empty)
+ notifyAll();
+ }
+
+ if (cancel)
+ cancel();
+ }
+ }
+}
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
index 722f4a9..5da1db2 100644
--- 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
@@ -20,20 +20,22 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
import java.util.function.Predicate;
/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ *
*/
public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
/** */
private final Predicate<Object[]> predicate;
/**
- * @param target Target.
+ * @param ctx Execution context.
* @param predicate Predicate.
*/
- public FilterNode(Sink<Object[]> target, Predicate<Object[]> predicate) {
- super(target);
+ public FilterNode(ExecutionContext ctx, Node<Object[]> input, Predicate<Object[]> predicate) {
+ super(ctx, input);
this.predicate = predicate;
+
+ link();
}
/** {@inheritDoc} */
@@ -46,11 +48,11 @@ public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Obj
/** {@inheritDoc} */
@Override public boolean push(Object[] row) {
- return !predicate.test(row) || target.push(row);
+ return !predicate.test(row) || target().push(row);
}
/** {@inheritDoc} */
@Override public void end() {
- target.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
index df6d10b..4199f7f 100644
--- 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
@@ -17,17 +17,14 @@
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 org.apache.calcite.DataContext;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.schema.ScannableTable;
-import org.apache.ignite.internal.processors.query.calcite.exchange.Outbox;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
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;
@@ -38,151 +35,116 @@ 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.splitter.RelSource;
import org.apache.ignite.internal.processors.query.calcite.splitter.RelTarget;
-import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
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;
-
/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Implements a query plan.
*/
public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<IgniteRel, Node<Object[]>> {
/** */
- private final PlannerContext ctx;
+ private final ExecutionContext ctx;
+
+ /** */
+ private final PartitionService partitionService;
/** */
- private final DataContext root;
+ private final ExchangeService exchangeService;
/** */
- private final ScalarFactory factory;
+ private final MailboxRegistry mailboxRegistry;
/** */
- private Deque<Sink<Object[]>> stack;
+ private final ScalarFactory scalarFactory;
/**
- * @param root Root context.
+ * @param partitionService Affinity service.
+ * @param mailboxRegistry Mailbox registry.
+ * @param exchangeService Exchange service.
+ * @param failure Failure processor.
+ * @param ctx Root context.
+ * @param log Logger.
*/
- public Implementor(DataContext root) {
- this.root = root;
+ public Implementor(PartitionService partitionService, MailboxRegistry mailboxRegistry, ExchangeService exchangeService, FailureProcessor failure, ExecutionContext ctx, IgniteLogger log) {
+ this.partitionService = partitionService;
+ this.mailboxRegistry = mailboxRegistry;
+ this.exchangeService = exchangeService;
+ this.ctx = ctx;
- ctx = PLANNER_CONTEXT.get(root);
- factory = new ScalarFactory(new RexBuilder(ctx.typeFactory()));
- stack = new ArrayDeque<>();
+ scalarFactory = new ScalarFactory(ctx.getTypeFactory(), failure, log);
}
/** {@inheritDoc} */
@Override public Node<Object[]> visit(IgniteSender rel) {
- assert stack.isEmpty();
-
RelTarget target = rel.target();
+ long targetFragmentId = target.fragmentId();
IgniteDistribution distribution = target.distribution();
- DestinationFunction function = distribution.function().toDestination(ctx, target.mapping(), distribution.getKeys());
-
- Outbox<Object[]> res = new Outbox<>(QUERY_ID.get(root), target.exchangeId(), function);
+ Destination destination = distribution.function().destination(partitionService, target.mapping(), distribution.getKeys());
- stack.push(res.sink());
+ // Outbox fragment ID is used as exchange ID as well.
+ Outbox<Object[]> outbox = new Outbox<>(exchangeService, mailboxRegistry, ctx, targetFragmentId, ctx.fragmentId(), visit(rel.getInput()), destination);
- res.source(source(rel.getInput()));
+ mailboxRegistry.register(outbox);
- return res;
+ return outbox;
}
/** {@inheritDoc} */
@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;
+ Predicate<Object[]> predicate = scalarFactory.filterPredicate(ctx, rel.getCondition(), rel.getRowType());
+ return new FilterNode(ctx, visit(rel.getInput()), predicate);
}
/** {@inheritDoc} */
@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;
+ Function<Object[], Object[]> projection = scalarFactory.projectExpression(ctx, rel.getProjects(), rel.getInput().getRowType());
+ return new ProjectNode(ctx, visit(rel.getInput()), projection);
}
/** {@inheritDoc} */
@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;
+ BiFunction<Object[], Object[], Object[]> expression = scalarFactory.joinExpression(ctx, rel.getCondition(), rel.getLeft().getRowType(), rel.getRight().getRowType());
+ return new JoinNode(ctx, visit(rel.getLeft()), visit(rel.getRight()), expression);
}
/** {@inheritDoc} */
@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);
+ return new ScanNode(ctx, rel.getTable().unwrap(ScannableTable.class).scan(ctx));
}
/** {@inheritDoc} */
@Override public Node<Object[]> visit(IgniteReceiver rel) {
- throw new AssertionError(); // TODO
- }
+ RelSource source = rel.source();
- /** {@inheritDoc} */
- @Override public Node<Object[]> visit(IgniteExchange rel) {
- throw new AssertionError();
+ // Corresponding outbox fragment ID is used as exchange ID as well.
+ Inbox<Object[]> inbox = (Inbox<Object[]>) mailboxRegistry.register(new Inbox<>(exchangeService, mailboxRegistry, ctx, source.fragmentId(), source.fragmentId()));
+
+ // here may be an already created (to consume rows from remote nodes) inbox
+ // without proper context, we need to init it with a right one.
+ inbox.init(ctx, source.mapping().nodes(), scalarFactory.comparator(ctx, rel.collations(), rel.getRowType()));
+
+ return inbox;
}
/** {@inheritDoc} */
@Override public Node<Object[]> visit(IgniteRel rel) {
- throw new AssertionError();
+ return rel.accept(this);
}
- /** */
- private Source source(RelNode rel) {
- if (rel.getConvention() != IgniteConvention.INSTANCE)
- throw new IllegalStateException("INTERPRETABLE is required.");
-
- return ((IgniteRel) rel).accept(this);
+ /** {@inheritDoc} */
+ @Override public Node<Object[]> visit(IgniteExchange rel) {
+ throw new AssertionError();
}
/** */
- private List<Source> sources(List<RelNode> rels) {
- ArrayList<Source> res = new ArrayList<>(rels.size());
-
- for (RelNode rel : rels) {
- res.add(source(rel));
- }
-
- return res;
+ private Node<Object[]> visit(RelNode rel) {
+ return visit((IgniteRel) rel);
}
/** {@inheritDoc} */
@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;
+ return visit(rel);
}
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java
new file mode 100644
index 0000000..de2118f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java
@@ -0,0 +1,442 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.calcite.util.Pair;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * A part of exchange.
+ */
+public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoCloseable {
+ /** */
+ private final ExchangeService exchange;
+
+ /** */
+ private final MailboxRegistry registry;
+
+ /** */
+ private final long exchangeId;
+
+ /** */
+ private final long sourceFragmentId;
+
+ /** */
+ private final Map<UUID, Buffer> perNodeBuffers;
+
+ /** */
+ private Collection<UUID> sources;
+
+ /** */
+ private Comparator<T> comparator;
+
+ /** */
+ private List<Buffer> buffers;
+
+ /** */
+ private boolean end;
+
+ /**
+ * @param exchange Exchange service.
+ * @param registry Mailbox registry.
+ * @param ctx Execution context.
+ * @param sourceFragmentId Source fragment ID.
+ * @param exchangeId Exchange ID.
+ */
+ public Inbox(ExchangeService exchange, MailboxRegistry registry, ExecutionContext ctx, long sourceFragmentId, long exchangeId) {
+ super(ctx);
+ this.exchange = exchange;
+ this.registry = registry;
+
+ this.sourceFragmentId = sourceFragmentId;
+ this.exchangeId = exchangeId;
+
+ perNodeBuffers = new HashMap<>();
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return context().queryId();
+ }
+
+ /**
+ * @return Exchange ID.
+ */
+ public long exchangeId() {
+ return exchangeId;
+ }
+
+ /**
+ * Inits this Inbox.
+ *
+ * @param ctx Execution context.
+ * @param sources Source nodes.
+ * @param comparator Optional comparator for merge exchange.
+ */
+ public void init(ExecutionContext ctx, Collection<UUID> sources, Comparator<T> comparator) {
+ this.comparator = comparator;
+ this.sources = sources;
+
+ context(ctx);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void request() {
+ prepare();
+ pushInternal();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void cancel() {
+ checkThread();
+ context().setCancelled();
+ close();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ registry.unregister(this);
+ }
+
+ /**
+ * Pushes a batch into a buffer.
+ *
+ * @param source Source node.
+ * @param batchId Batch ID.
+ * @param rows Rows.
+ */
+ public void onBatchReceived(UUID source, int batchId, List<?> rows) {
+ checkThread();
+ Buffer buffer = getOrCreateBuffer(source);
+
+ if (buffer.add(batchId, rows))
+ pushInternal();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Sink<T> sink(int idx) {
+ throw new UnsupportedOperationException();
+ }
+
+ /** */
+ private void pushInternal() {
+ checkThread();
+
+ if (context().cancelled())
+ close();
+ else if (!end && ready()) {
+ if (comparator != null)
+ pushOrdered();
+ else
+ pushUnordered();
+ }
+ }
+
+ /** */
+ private void pushOrdered() {
+ PriorityQueue<Pair<T, Buffer>> heap = new PriorityQueue<>(buffers.size(), Map.Entry.comparingByKey(comparator));
+
+ ListIterator<Buffer> it = buffers.listIterator();
+
+ while (it.hasNext()) {
+ Buffer buffer = it.next();
+
+ switch (buffer.check()) {
+ case END:
+ it.remove();
+
+ break;
+ case READY:
+ heap.offer(Pair.of((T)buffer.peek(), buffer));
+
+ break;
+ case WAITING:
+
+ return;
+ }
+ }
+
+ Sink<T> target = target();
+
+ while (!heap.isEmpty()) {
+ Pair<T, Buffer> pair = heap.poll();
+
+ T row = pair.left; Buffer buffer = pair.right;
+
+ if (!target.push(row))
+ return;
+
+ buffer.remove();
+
+ switch (buffer.check()) {
+ case END:
+ buffers.remove(buffer);
+
+ break;
+ case READY:
+ heap.offer(Pair.of((T)buffer.peek(), buffer));
+
+ break;
+ case WAITING:
+
+ return;
+ }
+ }
+
+ end = true;
+ target.end();
+ close();
+ }
+
+ /** */
+ private void pushUnordered() {
+ int size = buffers.size();
+
+ if (size <= 0 && !end)
+ throw new AssertionError("size=" + size + ", end=" + end);
+
+ int idx = ThreadLocalRandom.current().nextInt(size);
+ int noProgress = 0;
+
+ Sink<T> target = target();
+
+ while (size > 0) {
+ Buffer buffer = buffers.get(idx);
+
+ switch (buffer.check()) {
+ case END:
+ buffers.remove(idx);
+
+ if (idx == --size)
+ idx = 0;
+
+ continue;
+ case READY:
+ if (!target.push((T)buffer.peek()))
+ return;
+
+ buffer.remove();
+ noProgress = 0;
+
+ break;
+ case WAITING:
+ if (++noProgress >= size)
+ return;
+
+ break;
+ }
+
+ if (++idx == size)
+ idx = 0;
+ }
+
+ end = true;
+ target.end();
+ close();
+ }
+
+ /** */
+ private void acknowledge(UUID nodeId, int batchId) {
+ exchange.acknowledge(this, nodeId, queryId(), sourceFragmentId, exchangeId, batchId);
+ }
+
+ /** */
+ private void prepare() {
+ if (ready())
+ return;
+
+ assert sources != null;
+
+ sources.forEach(this::getOrCreateBuffer);
+
+ assert perNodeBuffers.size() == sources.size();
+
+ buffers = new ArrayList<>(perNodeBuffers.values());
+ }
+
+ /** */
+ private boolean ready() {
+ return buffers != null;
+ }
+
+ /** */
+ private Buffer getOrCreateBuffer(UUID nodeId) {
+ return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+ }
+
+ /** */
+ private Buffer createBuffer(UUID nodeId) {
+ return new Buffer(nodeId, this);
+ }
+
+ /** */
+ private static final class Batch implements Comparable<Batch> {
+ /** */
+ private final int batchId;
+
+ /** */
+ private final List<?> rows;
+
+ /** */
+ private int idx;
+
+ /** */
+ private Batch(int batchId, List<?> rows) {
+ this.batchId = batchId;
+ this.rows = rows;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ Batch batch = (Batch) o;
+
+ return batchId == batch.batchId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ return batchId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int compareTo(@NotNull Inbox.Batch o) {
+ return Integer.compare(batchId, o.batchId);
+ }
+ }
+
+ /** */
+ private enum State {
+ END, READY, WAITING
+ }
+
+ /** */
+ private static final class Buffer {
+ /** */
+ private static final Batch WAITING = new Batch(0, null);
+
+ /** */
+ private static final Batch END = new Batch(0, null);
+
+ /** */
+ private final Inbox<?> owner;
+
+ /** */
+ private final UUID nodeId;
+
+ /** */
+ private int lastEnqueued = -1;
+
+ /** */
+ private final PriorityQueue<Batch> batches = new PriorityQueue<>(ExchangeService.PER_NODE_BATCH_COUNT);
+
+ /** */
+ private Batch curr = WAITING;
+
+ /** */
+ private Buffer(UUID nodeId, Inbox<?> owner) {
+ this.nodeId = nodeId;
+ this.owner = owner;
+ }
+
+ /** */
+ private boolean add(int id, List<?> rows) {
+ batches.offer(new Batch(id, rows));
+
+ return curr == WAITING && batches.peek().batchId == lastEnqueued + 1;
+ }
+
+ /** */
+ private Batch pollBatch() {
+ if (batches.isEmpty() || batches.peek().batchId != lastEnqueued + 1)
+ return WAITING;
+
+ Batch batch = batches.poll();
+
+ assert batch != null && batch.batchId == lastEnqueued + 1;
+
+ lastEnqueued = batch.batchId;
+
+ return batch;
+ }
+
+ /** */
+ private State check() {
+ if (curr == END)
+ return State.END;
+
+ if (curr == WAITING)
+ curr = pollBatch();
+
+ if (curr == WAITING)
+ return State.WAITING;
+
+ if (curr.rows.get(curr.idx) == EndMarker.INSTANCE) {
+ curr = END;
+
+ return State.END;
+ }
+
+ return State.READY;
+ }
+
+ /** */
+ private Object peek() {
+ assert curr != null;
+ assert curr != WAITING;
+ assert curr != END;
+ assert curr.rows.get(curr.idx) != EndMarker.INSTANCE;
+
+ return curr.rows.get(curr.idx);
+ }
+
+ /** */
+ private Object remove() {
+ assert curr != null;
+ assert curr != WAITING;
+ assert curr != END;
+ assert curr.rows.get(curr.idx) != EndMarker.INSTANCE;
+
+ Object row = curr.rows.set(curr.idx++, null);
+
+ if (curr.idx == curr.rows.size()) {
+ owner.acknowledge(nodeId, curr.batchId);
+
+ curr = pollBatch();
+ }
+
+ return row;
+ }
+ }
+}
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
index 8521f4a..46b5bdb 100644
--- 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
@@ -17,11 +17,12 @@
package org.apache.ignite.internal.processors.query.calcite.exec;
+import com.google.common.collect.ImmutableList;
import java.util.ArrayList;
import java.util.function.BiFunction;
/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * TODO remove buffers.
*/
public class JoinNode extends AbstractNode<Object[]> {
/** */
@@ -43,16 +44,17 @@ public class JoinNode extends AbstractNode<Object[]> {
private boolean end;
/**
- *
- * @param target Target.
+ * @param ctx Execution context.
* @param expression Join expression.
*/
- public JoinNode(Sink<Object[]> target, BiFunction<Object[], Object[], Object[]> expression) {
- super(target);
+ public JoinNode(ExecutionContext ctx, Node<Object[]> left, Node<Object[]> right, BiFunction<Object[], Object[], Object[]> expression) {
+ super(ctx, ImmutableList.of(left, right));
+
this.expression = expression;
+ this.left = new ArraySink<>();
+ this.right = new ArraySink<>();
- left = new ArraySink<>();
- right = new ArraySink<>();
+ link();
}
/** {@inheritDoc} */
@@ -68,29 +70,33 @@ public class JoinNode extends AbstractNode<Object[]> {
}
/** {@inheritDoc} */
- @Override public void signal() {
- if (end)
- return;
+ @Override public void request() {
+ checkThread();
- if (left.end && right.end)
- tryFlush();
-
- assert sources != null && sources.size() == 2;
+ if (context().cancelled() || end)
+ return;
if (!left.end)
- signal(0);
+ input(0).request();
if (!right.end)
- signal(1);
+ input(1).request();
+ if (!end)
+ tryFlush();
}
/** */
public void tryFlush() {
+ assert !end;
+
if (left.end && right.end) {
for (int i = leftIdx; i < left.size(); i++) {
for (int j = rightIdx; j < right.size(); j++) {
+ if (context().cancelled())
+ return;
+
Object[] row = expression.apply(left.get(i), right.get(j));
- if (row != null && !target.push(row)) {
+ if (row != null && !target().push(row)) {
leftIdx = i;
rightIdx = j;
@@ -100,7 +106,7 @@ public class JoinNode extends AbstractNode<Object[]> {
}
end = true;
- target.end();
+ target().end();
}
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java
new file mode 100644
index 0000000..7dd462f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.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.exec;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface MailboxRegistry extends Service {
+ /**
+ * Tries to register and inbox node and returns it if success or returns previously registered inbox otherwise.
+ *
+ * @param inbox Inbox.
+ * @return Registered inbox.
+ */
+ Inbox<?> register(Inbox<?> inbox);
+
+ /**
+ * Unregisters an inbox.
+ *
+ * @param inbox Inbox to unregister.
+ */
+ void unregister(Inbox<?> inbox);
+
+ /**
+ * Registers an outbox.
+ *
+ * @param outbox Outbox to register.
+ */
+ void register(Outbox<?> outbox);
+
+ /**
+ * Unregisters an outbox.
+ *
+ * @param outbox Outbox to unregister.
+ */
+ void unregister(Outbox<?> outbox);
+
+ /**
+ * Returns a registered outbox by provided query ID, exchange ID pair.
+ *
+ * @param queryId Query ID.
+ * @param exchangeId Exchange ID.
+ *
+ * @return Registered outbox. May be {@code null} if execution was cancelled.
+ */
+ Outbox<?> outbox(UUID queryId, long exchangeId);
+
+ /**
+ * Returns a registered inbox by provided query ID, exchange ID pair.
+ *
+ * @param queryId Query ID.
+ * @param exchangeId Exchange ID.
+ *
+ * @return Registered inbox. May be {@code null} if execution was cancelled.
+ */
+ Inbox<?> inbox(UUID queryId, long exchangeId);
+
+ /**
+ * Returns all registered inboxes for provided query ID.
+ *
+ * @param queryId Query ID.
+ * @return Registered inboxes.
+ */
+ List<Inbox<?>> inboxes(UUID queryId);
+
+ /**
+ * Returns all registered outboxes for provided query ID.
+ *
+ * @param queryId Query ID.
+ * @return Registered outboxes.
+ */
+ List<Outbox<?>> outboxes(UUID queryId);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java
new file mode 100644
index 0000000..cc6e0be
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java
@@ -0,0 +1,132 @@
+/*
+ * 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.exec;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+
+/**
+ *
+ */
+public class MailboxRegistryImpl extends AbstractService implements MailboxRegistry {
+ /** */
+ private final Map<MailboxKey, Outbox<?>> locals;
+
+ /** */
+ private final Map<MailboxKey, Inbox<?>> remotes;
+
+ /**
+ * @param ctx Kernal.
+ */
+ public MailboxRegistryImpl(GridKernalContext ctx) {
+ super(ctx);
+
+ locals = new ConcurrentHashMap<>();
+ remotes = new ConcurrentHashMap<>();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Inbox<?> register(Inbox<?> inbox) {
+ Inbox<?> old = remotes.putIfAbsent(new MailboxKey(inbox.queryId(), inbox.exchangeId()), inbox);
+
+ return old != null ? old : inbox;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void unregister(Inbox<?> inbox) {
+ remotes.remove(new MailboxKey(inbox.queryId(), inbox.exchangeId()));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void register(Outbox<?> outbox) {
+ Outbox<?> res = locals.put(new MailboxKey(outbox.queryId(), outbox.exchangeId()), outbox);
+
+ assert res == null : res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void unregister(Outbox<?> outbox) {
+ locals.remove(new MailboxKey(outbox.queryId(), outbox.exchangeId()));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Outbox<?> outbox(UUID queryId, long exchangeId) {
+ return locals.get(new MailboxKey(queryId, exchangeId));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Inbox<?> inbox(UUID queryId, long exchangeId) {
+ return remotes.get(new MailboxKey(queryId, exchangeId));
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Inbox<?>> inboxes(UUID queryId) {
+ return remotes.entrySet().stream()
+ .filter(e -> e.getKey().queryId.equals(queryId))
+ .map(Map.Entry::getValue)
+ .collect(Collectors.toList());
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Outbox<?>> outboxes(UUID queryId) {
+ return locals.entrySet().stream()
+ .filter(e -> e.getKey().queryId.equals(queryId))
+ .map(Map.Entry::getValue)
+ .collect(Collectors.toList()); }
+
+ /** */
+ private static class MailboxKey {
+ /** */
+ private final UUID queryId;
+
+ /** */
+ private final long exchangeId;
+
+ /** */
+ private MailboxKey(UUID queryId, long exchangeId) {
+ this.queryId = queryId;
+ this.exchangeId = exchangeId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ MailboxKey that = (MailboxKey) o;
+
+ if (exchangeId != that.exchangeId)
+ return false;
+ return queryId.equals(that.queryId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ int result = queryId.hashCode();
+ result = 31 * result + (int) (exchangeId ^ (exchangeId >>> 32));
+ return result;
+ }
+ }
+}
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
index 83dc92b..f0e9506 100644
--- 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
@@ -21,10 +21,20 @@ import java.util.List;
/**
* Represents a node of execution tree.
+ *
+ * <p/><b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request()}, {@link Node#cancel()},
+ * {@link Node#reset()}, {@link Sink#push(Object)} and {@link Sink#end()} methods should be used from one single thread.
*/
-public interface Node<T> extends Source {
+public interface Node<T> {
+ /**
+ * Returns runtime context allowing access to the tables in a database.
+ *
+ * @return Execution context.
+ */
+ ExecutionContext context();
+
/**
- * Requests a target sink of the node. The sink is used to push data into the node by its children.
+ * Requests a sink of the node. The sink is used to push data into the node by its children.
*
* @param idx Sink index.
* @return Sink object.
@@ -33,9 +43,42 @@ public interface Node<T> extends Source {
Sink<T> sink(int idx);
/**
- * Registers sources of this node. Sources are used to notify children when the node is ready to consume data.
+ * Registers target sink.
*
- * @param sources Sources.
+ * @param sink Target sink.
+ */
+ void target(Sink<T> sink);
+
+ /**
+ * @return Registered target.
+ */
+ Sink<T> target();
+
+ /**
+ * @return Node inputs collection.
+ */
+ List<Node<T>> inputs();
+
+ /**
+ * @param idx Input index.
+ * @return Node input.
+ */
+ default Node<T> input(int idx) {
+ return inputs().get(idx);
+ }
+
+ /**
+ * Signals that consumer is ready to consume data.
+ */
+ void request();
+
+ /**
+ * Cancels execution.
+ */
+ void cancel();
+
+ /**
+ * Resets execution sub-tree to initial state.
*/
- void sources(List<Source> sources);
+ void reset();
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java
new file mode 100644
index 0000000..3840cb4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java
@@ -0,0 +1,293 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * A part of exchange.
+ */
+public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>, AutoCloseable {
+ /** */
+ private final ExchangeService exchange;
+
+ /** */
+ private final long exchangeId;
+
+ /** */
+ private final MailboxRegistry registry;
+
+ /** */
+ private final long targetFragmentId;
+
+ /** */
+ private final Map<UUID, Buffer> perNodeBuffers = new HashMap<>();
+
+ /** */
+ private final Destination destination;
+
+ /** */
+ private boolean cancelled;
+
+ /**
+ * @param exchange Exchange service.
+ * @param registry Mailbox registry.
+ * @param ctx Execution context.
+ * @param targetFragmentId Target fragment ID.
+ * @param exchangeId Exchange ID.
+ * @param input Input node.
+ * @param destination Destination.
+ */
+ public Outbox(ExchangeService exchange, MailboxRegistry registry, ExecutionContext ctx, long targetFragmentId, long exchangeId, Node<T> input, Destination destination) {
+ super(ctx, input);
+ this.exchange = exchange;
+ this.registry = registry;
+ this.targetFragmentId = targetFragmentId;
+ this.exchangeId = exchangeId;
+ this.destination = destination;
+
+ link();
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return context().queryId();
+ }
+
+ /**
+ * @return Exchange ID.
+ */
+ public long exchangeId() {
+ return exchangeId;
+ }
+
+ /**
+ * callback method.
+ *
+ * @param nodeId Target ID.
+ * @param batchId Batch ID.
+ */
+ public void onAcknowledge(UUID nodeId, int batchId) {
+ perNodeBuffers.get(nodeId).onAcknowledge(batchId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void request() {
+ checkThread();
+
+ if (context().cancelled())
+ cancelInternal();
+ else
+ input().request();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void cancel() {
+ checkThread();
+
+ context().setCancelled();
+ cancelInternal();
+ }
+
+ /** */
+ private void cancelInternal() {
+ if (cancelled)
+ return;
+
+ try {
+ perNodeBuffers.values().forEach(Buffer::cancel);
+ input().cancel();
+ }
+ finally {
+ cancelled = true;
+ close();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void target(Sink<T> sink) {
+ throw new AssertionError();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Sink<T> sink(int idx) {
+ if (idx != 0)
+ throw new IndexOutOfBoundsException();
+
+ return this;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean push(T row) {
+ List<UUID> nodes = destination.targets(row);
+
+ assert !F.isEmpty(nodes);
+
+ List<Buffer> buffers = new ArrayList<>(nodes.size());
+
+ for (UUID node : nodes) {
+ Buffer dest = getOrCreateBuffer(node);
+
+ if (!dest.ready())
+ return false;
+
+ buffers.add(dest);
+ }
+
+ for (Buffer dest : buffers)
+ dest.add(row);
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void end() {
+ for (UUID node : destination.targets())
+ getOrCreateBuffer(node).end();
+
+ close();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ registry.unregister(this);
+ }
+
+ /** */
+ private void sendBatch(UUID nodeId, int batchId, List<?> rows) {
+ exchange.sendBatch(this, nodeId, queryId(), targetFragmentId, exchangeId, batchId, rows);
+ }
+
+ /** */
+ private void sendCancel(UUID nodeId, int batchId) {
+ exchange.cancel(this, nodeId, queryId(), targetFragmentId, exchangeId, batchId);
+ }
+
+ /** */
+ private Buffer getOrCreateBuffer(UUID nodeId) {
+ return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+ }
+
+ /** */
+ private Buffer createBuffer(UUID nodeId) {
+ return new Buffer(nodeId, this);
+ }
+
+ /** */
+ private static final class Buffer {
+ /** */
+ private final Outbox<?> owner;
+
+ /** */
+ private final UUID nodeId;
+
+ /** */
+ private int hwm = -1;
+
+ /** */
+ private int lwm = -1;
+
+ /** */
+ private List<Object> curr;
+
+ /** */
+ private Buffer(UUID nodeId, Outbox<?> owner) {
+ this.nodeId = nodeId;
+ this.owner = owner;
+
+ curr = new ArrayList<>(ExchangeService.BATCH_SIZE + 1); // extra space for end marker;
+ }
+
+ /**
+ * Adds a row to current batch.
+ *
+ * @param row Row.
+ */
+ public void add(Object row) {
+ assert ready();
+
+ if (curr.size() == ExchangeService.BATCH_SIZE) {
+ owner.sendBatch(nodeId, ++hwm, curr);
+ curr = new ArrayList<>(ExchangeService.BATCH_SIZE + 1); // extra space for end marker;
+ }
+
+ curr.add(row);
+ }
+
+ /**
+ * Signals data is over.
+ */
+ public void end() {
+ int batchId = hwm + 1;
+ hwm = Integer.MAX_VALUE;
+
+ List<Object> tmp = curr;
+ curr = null;
+
+ tmp.add(EndMarker.INSTANCE);
+ owner.sendBatch(nodeId, batchId, tmp);
+ }
+
+ public void cancel() {
+ int batchId = hwm + 1;
+ hwm = Integer.MAX_VALUE;
+
+ curr = null;
+ owner.sendCancel(nodeId, batchId);
+ }
+
+ /**
+ * Checks whether there is a place for a new row.
+ *
+ * @return {@code True} is it possible to add a row to a batch.
+ */
+ private boolean ready() {
+ if (curr == null)
+ throw new AssertionError();
+
+ return hwm != Integer.MAX_VALUE
+ && hwm - lwm < ExchangeService.PER_NODE_BATCH_COUNT
+ || curr.size() < ExchangeService.BATCH_SIZE;
+ }
+
+ /**
+ * Callback method.
+ *
+ * @param id batch ID.
+ */
+ private void onAcknowledge(int id) {
+ if (lwm > id)
+ return;
+
+ boolean request = hwm - lwm == ExchangeService.PER_NODE_BATCH_COUNT;
+
+ lwm = id;
+
+ if (request)
+ owner.request();
+ }
+ }
+}
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
index 210bea69..796bacd 100644
--- 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
@@ -27,13 +27,15 @@ public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Ob
private final Function<Object[], Object[]> projection;
/**
- * @param target Target.
+ * @param ctx Execution context.
* @param projection Projection.
*/
- public ProjectNode(Sink<Object[]> target, Function<Object[], Object[]> projection) {
- super(target);
+ public ProjectNode(ExecutionContext ctx, Node<Object[]> input, Function<Object[], Object[]> projection) {
+ super(ctx, input);
this.projection = projection;
+
+ link();
}
/** {@inheritDoc} */
@@ -46,11 +48,11 @@ public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Ob
/** {@inheritDoc} */
@Override public boolean push(Object[] row) {
- return target.push(projection.apply(row));
+ return target().push(projection.apply(row));
}
/** {@inheritDoc} */
@Override public void end() {
- target.end();
+ target().end();
}
}
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/QueryTaskExecutor.java
similarity index 68%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
index dd5226a..1ff87f5 100644
--- 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/QueryTaskExecutor.java
@@ -17,12 +17,19 @@
package org.apache.ignite.internal.processors.query.calcite.exec;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
/**
- * Represents an execution node source. Sources are used to notify children when an execution node is ready to consume data.
+ *
*/
-public interface Source {
+public interface QueryTaskExecutor extends Service {
/**
- * Signals that consumer is ready to consume data. Called by consumer node.
+ * Executes a query task in a thread, responsible for particular query fragment.
+ *
+ * @param queryId Query ID.
+ * @param fragmentId Fragment ID.
+ * @param queryTask Query task.
*/
- void signal();
+ void execute(UUID queryId, long fragmentId, Runnable queryTask);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
new file mode 100644
index 0000000..c23cee1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
@@ -0,0 +1,74 @@
+/*
+ * 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.exec;
+
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME;
+
+/**
+ * TODO use {@link org.apache.ignite.internal.util.StripedExecutor}, registered in core pols.
+ */
+public class QueryTaskExecutorImpl extends AbstractService implements QueryTaskExecutor {
+ /** */
+ private IgniteStripedThreadPoolExecutor stripedThreadPoolExecutor;
+
+ /** */
+ public QueryTaskExecutorImpl(GridKernalContext ctx) {
+ super(ctx);
+ }
+
+ /**
+ * @param stripedThreadPoolExecutor Executor.
+ */
+ public void stripedThreadPoolExecutor(IgniteStripedThreadPoolExecutor stripedThreadPoolExecutor) {
+ this.stripedThreadPoolExecutor = stripedThreadPoolExecutor;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute(UUID queryId, long fragmentId, Runnable queryTask) {
+ stripedThreadPoolExecutor.execute(queryTask, hash(queryId, fragmentId));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ stripedThreadPoolExecutor(new IgniteStripedThreadPoolExecutor(
+ ctx.config().getQueryThreadPoolSize(),
+ ctx.igniteInstanceName(),
+ "calciteQry",
+ ctx.uncaughtExceptionHandler(),
+ true,
+ DFLT_THREAD_KEEP_ALIVE_TIME
+ ));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void tearDown() {
+ U.shutdownNow(getClass(), stripedThreadPoolExecutor, log);
+ }
+
+ /** */
+ private static int hash(UUID queryId, long fragmentId) {
+ // inlined Objects.hash(...)
+ return U.safeAbs(31 * (31 + (queryId != null ? queryId.hashCode() : 0)) + Long.hashCode(fragmentId));
+ }
+}
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
index e0adce7..d11f29a5 100644
--- 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
@@ -19,19 +19,32 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
import com.google.common.collect.ImmutableList;
import java.util.Arrays;
+import java.util.Comparator;
import java.util.List;
+import java.util.Map;
import java.util.function.BiFunction;
import java.util.function.Function;
import java.util.function.Predicate;
import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
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.RelCollation;
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;
+import org.apache.commons.lang.text.StrBuilder;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
/**
* Implements rex expression into a function object. Uses JaninoRexCompiler under the hood.
@@ -39,18 +52,43 @@ import org.apache.calcite.rex.RexNode;
*/
public class ScalarFactory {
/** */
+ private static final int CACHE_SIZE = 1024;
+
+ /** */
+ private static final Map<String, Scalar> CACHE = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+
+ /** */
+ private final JavaTypeFactory typeFactory;
+
+ /** */
private final JaninoRexCompiler rexCompiler;
/** */
- private final RexBuilder builder;
+ private final ExceptionHandler handler;
/**
- * @param builder RexBuilder.
+ * @param typeFactory Type factory.
+ * @param failure Failure processor.
+ * @param log Logger.
*/
- public ScalarFactory(RexBuilder builder) {
- this.builder = builder;
+ public ScalarFactory(JavaTypeFactory typeFactory, FailureProcessor failure, IgniteLogger log) {
+ this.typeFactory = typeFactory;
- rexCompiler = new JaninoRexCompiler(builder);
+ rexCompiler = new JaninoRexCompiler(new RexBuilder(typeFactory));
+ handler = new ExceptionHandler(failure, log.getLogger(ScalarFactory.class));
+ }
+
+ /**
+ * Creates a comparator for given data type and collations. Mainly used for sorted exchange.
+ *
+ * @param root Execution context, holds a planner context, query and its parameters,
+ * execution specific variables (like queryId, current user, session, etc).
+ * @param collations Collations.
+ * @param rowType Input row type.
+ * @return Row comparator.
+ */
+ public <T> Comparator<T> comparator(DataContext root, List<RelCollation> collations, RelDataType rowType) {
+ return null; // TODO
}
/**
@@ -62,10 +100,10 @@ public class ScalarFactory {
* @return Filter predicate.
*/
public <T> Predicate<T> filterPredicate(DataContext root, RexNode filter, RelDataType rowType) {
- Scalar scalar = rexCompiler.compile(ImmutableList.of(filter), rowType);
+ Scalar scalar = scalar(ImmutableList.of(filter), rowType);
Context ctx = InterpreterUtils.createContext(root);
- return new FilterPredicate<>(ctx, scalar);
+ return new FilterPredicate<>(ctx, scalar, handler);
}
/**
@@ -78,11 +116,11 @@ public class ScalarFactory {
* @return Project function.
*/
public <T> Function<T, T> projectExpression(DataContext root, List<RexNode> projects, RelDataType rowType) {
- Scalar scalar = rexCompiler.compile(projects, rowType);
+ Scalar scalar = scalar(projects, rowType);
Context ctx = InterpreterUtils.createContext(root);
int count = projects.size();
- return new ProjectExpression<>(ctx, scalar, count);
+ return new ProjectExpression<>(ctx, scalar, count, handler);
}
/**
@@ -96,17 +134,16 @@ public class ScalarFactory {
*/
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);
+ Scalar scalar = scalar(ImmutableList.of(expression), rowType);
Context ctx = InterpreterUtils.createContext(root);
ctx.values = new Object[rowType.getFieldCount()];
- return new JoinExpression<>(ctx, scalar);
+ return new JoinExpression<>(ctx, scalar, handler);
}
/** */
private RelDataType combinedType(RelDataType... types) {
- RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(typeFactory());
+ RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(typeFactory);
for (RelDataType type : types)
typeBuilder.addAll(type.getFieldList());
@@ -115,8 +152,20 @@ public class ScalarFactory {
}
/** */
- private RelDataTypeFactory typeFactory() {
- return builder.getTypeFactory();
+ private Scalar scalar(List<RexNode> nodes, RelDataType type) {
+ assert !F.isEmpty(nodes);
+
+ return CACHE.computeIfAbsent(cacheKey(nodes, type), k -> rexCompiler.compile(nodes, type));
+ }
+
+ /** */
+ private String cacheKey(List<RexNode> nodes, RelDataType type) {
+ StrBuilder b = new StrBuilder("[").append(F.first(nodes));
+ for (int i = 1; i < nodes.size(); i++)
+ b.append(";").append(nodes.get(i));
+ b.append("]:").append(type.getFullTypeString());
+
+ return b.toString();
}
/** */
@@ -128,24 +177,35 @@ public class ScalarFactory {
private final Scalar scalar;
/** */
+ private final ExceptionHandler handler;
+
+ /** */
private final Object[] vals;
/**
* @param ctx Interpreter context.
* @param scalar Scalar.
*/
- private FilterPredicate(Context ctx, Scalar scalar) {
+ private FilterPredicate(Context ctx, Scalar scalar, ExceptionHandler handler) {
this.ctx = ctx;
this.scalar = scalar;
+ this.handler = handler;
vals = new Object[1];
}
/** {@inheritDoc} */
@Override public boolean test(T r) {
- ctx.values = (Object[]) r;
- scalar.execute(ctx, vals);
- return (Boolean) vals[0];
+ try {
+ ctx.values = (Object[]) r;
+ scalar.execute(ctx, vals);
+ return (Boolean) vals[0];
+ }
+ catch (Exception e) {
+ handler.onException(e);
+
+ throw e;
+ }
}
}
@@ -161,35 +221,46 @@ public class ScalarFactory {
private final Scalar scalar;
/** */
+ private final ExceptionHandler handler;
+
+ /** */
private Object[] left0;
/**
* @param ctx Interpreter context.
* @param scalar Scalar.
*/
- private JoinExpression(Context ctx, Scalar scalar) {
+ private JoinExpression(Context ctx, Scalar scalar, ExceptionHandler handler) {
this.ctx = ctx;
this.scalar = scalar;
+ this.handler = handler;
vals = new Object[1];
}
/** {@inheritDoc} */
@Override public T apply(T left, T right) {
- if (left0 != left) {
- left0 = (Object[]) left;
- System.arraycopy(left0, 0, ctx.values, 0, left0.length);
- }
+ try {
+ 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);
+ Object[] right0 = (Object[]) right;
+ System.arraycopy(right0, 0, ctx.values, left0.length, right0.length);
- scalar.execute(ctx, vals);
+ scalar.execute(ctx, vals);
- if ((Boolean) vals[0])
- return (T) Arrays.copyOf(ctx.values, ctx.values.length);
+ if ((Boolean) vals[0])
+ return (T) Arrays.copyOf(ctx.values, ctx.values.length);
- return null;
+ return null;
+ }
+ catch (Exception e) {
+ handler.onException(e);
+
+ throw e;
+ }
}
}
@@ -204,24 +275,58 @@ public class ScalarFactory {
/** */
private final int count;
+ /** */
+ private final ExceptionHandler handler;
+
/**
* @param ctx Interpreter context.
* @param scalar Scalar.
* @param count Resulting columns count.
*/
- private ProjectExpression(Context ctx, Scalar scalar, int count) {
+ private ProjectExpression(Context ctx, Scalar scalar, int count, ExceptionHandler handler) {
this.ctx = ctx;
this.scalar = scalar;
this.count = count;
+ this.handler = handler;
}
/** {@inheritDoc} */
@Override public T apply(T r) {
- ctx.values = (Object[]) r;
- Object[] res = new Object[count];
- scalar.execute(ctx, res);
+ try {
+ ctx.values = (Object[]) r;
+ Object[] res = new Object[count];
+ scalar.execute(ctx, res);
+
+ return (T) res;
+ }
+ catch (Exception e) {
+ handler.onException(e);
+
+ throw e;
+ }
+ }
+ }
+
+ /** */
+ private static class ExceptionHandler {
+ /** */
+ private final FailureProcessor failure;
+
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private ExceptionHandler(@Nullable FailureProcessor failure, IgniteLogger log) {
+ this.failure = failure;
+ this.log = log;
+ }
+
+ /** */
+ void onException(Throwable ex) {
+ U.error(log, ex, ex);
- return (T) res;
+ if (failure != null)
+ failure.process(new FailureContext(FailureType.CRITICAL_ERROR, ex));
}
}
}
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
index 3fabc97..393d26d 100644
--- 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
@@ -18,18 +18,12 @@
package org.apache.ignite.internal.processors.query.calcite.exec;
import java.util.Iterator;
-import java.util.List;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Scan node.
*/
-public class ScanNode implements SingleNode<Object[]> {
- /** */
- private static final Object[] END = new Object[0];
-
- /** */
- private final Sink<Object[]> target;
-
+public class ScanNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, AutoCloseable {
/** */
private final Iterable<Object[]> source;
@@ -37,48 +31,83 @@ public class ScanNode implements SingleNode<Object[]> {
private Iterator<Object[]> it;
/** */
- private Object[] row;
+ private Object row;
/**
- * @param target Target.
+ * @param ctx Execution context.
* @param source Source.
*/
- public ScanNode(Sink<Object[]> target, Iterable<Object[]> source) {
- this.target = target;
+ public ScanNode(ExecutionContext ctx, Iterable<Object[]> source) {
+ super(ctx);
+
this.source = source;
}
/** {@inheritDoc} */
- @Override public void signal() {
- if (row == END)
- return;
+ @Override public void request() {
+ try {
+ requestInternal();
+ }
+ catch (Exception e) {
+ Commons.close(it, e);
- if (row != null && !target.push(row))
- return;
+ throw e;
+ }
+ }
+
+ /** */
+ private void requestInternal() {
+ checkThread();
- row = null;
+ if (context().cancelled()
+ || row == EndMarker.INSTANCE
+ || row != null && !target().push((Object[]) row))
+ return;
if (it == null)
it = source.iterator();
+ Thread thread = Thread.currentThread();
+
while (it.hasNext()) {
+ if (context().cancelled() || thread.isInterrupted()) {
+ row = null;
+
+ close();
+
+ return;
+ }
+
row = it.next();
- if (!target.push(row))
+ // TODO load balancing - resubmit this::request() in case of long execution
+
+ if (!target().push((Object[]) row))
return;
}
- row = END;
- target.end();
+ try {
+ row = EndMarker.INSTANCE;
+
+ target().end();
+ }
+ finally {
+ close();
+ }
}
/** {@inheritDoc} */
- @Override public void sources(List<Source> sources) {
- throw new UnsupportedOperationException();
+ @Override public void close() {
+ Commons.close(it);
+
+ it = null;
+
+ if (row != EndMarker.INSTANCE)
+ row = null;
}
/** {@inheritDoc} */
@Override public Sink<Object[]> sink(int idx) {
- throw new UnsupportedOperationException();
+ throw new AssertionError();
}
}
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
index 8dd7e42..cebb0d2 100644
--- 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
@@ -17,14 +17,12 @@
package org.apache.ignite.internal.processors.query.calcite.exec;
-import java.util.Collections;
import java.util.Objects;
/**
* A node with a single input
*/
public interface SingleNode<T> extends Node<T> {
-
/**
* @return Single sink object.
*/
@@ -33,10 +31,9 @@ public interface SingleNode<T> extends Node<T> {
}
/**
- * Registers a single source.
- * @param source Source.
+ * @return Single input.
*/
- default void source(Source source) {
- sources(Collections.singletonList(source));
+ default Node<T> input() {
+ return Objects.requireNonNull(input(0));
}
}
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
index 5023a63..8d09c5f 100644
--- 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
@@ -19,20 +19,11 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
/**
* Represents an abstract data consumer.
+ *
+ * <p/><b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request()}, {@link Node#cancel()},
+ * {@link Node#reset()}, {@link Sink#push(Object)} and {@link Sink#end()} methods should be used from one single thread.
*/
public interface Sink<T> {
- /** */
- @SuppressWarnings("rawtypes")
- Sink NO_OP = new Sink() {
- /** {@inheritDoc} */
- @Override public boolean push(Object row) {
- return true;
- }
-
- /** {@inheritDoc} */
- @Override public void end() {}
- };
-
/**
* Pushes a row to consumer.
* @param row Data row.
@@ -45,11 +36,4 @@ public interface Sink<T> {
* Signals that data is over.
*/
void end();
-
- /**
- * @return No-op sink object.
- */
- static <T> Sink<T> noOp() {
- return (Sink<T>) NO_OP;
- }
}
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/message/CalciteMessage.java
similarity index 65%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessage.java
index d4368c7..811f6f9 100644
--- 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/message/CalciteMessage.java
@@ -15,25 +15,26 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.exchange;
+package org.apache.ignite.internal.processors.query.calcite.message;
-import java.io.ObjectStreamException;
-import java.io.Serializable;
+import org.apache.ignite.plugin.extensions.communication.Message;
/**
- * Poison pill object.
+ *
*/
-public final class EndMarker implements Serializable {
- /** */
- public static final EndMarker INSTANCE = new EndMarker();
-
+public interface CalciteMessage extends Message {
/**
- * Default constructor.
+ * @return Message type.
*/
- private EndMarker(){}
+ MessageType type();
+
+ /** {@inheritDoc} */
+ @Override default short directType() {
+ return type().directType();
+ }
- /** */
- private Object readResolve() throws ObjectStreamException {
- return INSTANCE;
+ /** {@inheritDoc} */
+ @Override default void onAckReceived() {
+ // No-op.
}
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessageFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessageFactory.java
new file mode 100644
index 0000000..258e03c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessageFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.message;
+
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageFactory;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Message factory.
+ */
+public class CalciteMessageFactory implements MessageFactory {
+ /** {@inheritDoc} */
+ @Override public @Nullable Message create(short type) {
+ return MessageType.newMessage(type);
+ }
+
+ /**
+ * Produces a row message.
+ *
+ * TODO In future a row is expected to implement Message interface.
+ */
+ public static Message asMessage(Object row) {
+ return new GenericRowMessage(row);
+ }
+
+ /**
+ * Produces a row from a message.
+ *
+ * TODO In future a row is expected to implement Message interface.
+ */
+ public static Object asRow(Message mRow) {
+ if (mRow instanceof GenericRowMessage)
+ return ((GenericRowMessage) mRow).row();
+
+ throw new AssertionError("Unexpected message type. [message=" + mRow + "]");
+ }
+}
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/message/ExecutionContextAware.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAware.java
index c995ef2..779dc91 100644
--- 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/message/ExecutionContextAware.java
@@ -15,23 +15,29 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
-import java.util.List;
import java.util.UUID;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
/**
- * Determines where to send a row to.
+ * Execution context is used to determine a stripe where to process a message.
*/
-public interface DestinationFunction {
+public interface ExecutionContextAware extends CalciteMessage {
/**
- * @param row Row.
- * @return Target nodes list.
+ * @return Query ID.
*/
- List<UUID> destination(Object row);
+ UUID queryId();
/**
- * @return All target nodes.
+ * @return Fragment ID.
*/
- List<UUID> targets();
+ long fragmentId();
+
+ /**
+ * @return IO policy.
+ */
+ default byte ioPolicy() {
+ return GridIoPolicy.CALLER_THREAD;
+ }
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/GenericRowMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/GenericRowMessage.java
new file mode 100644
index 0000000..d9e7590
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/GenericRowMessage.java
@@ -0,0 +1,118 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class GenericRowMessage implements MarshalableMessage {
+ /** */
+ @GridDirectTransient
+ private Object row;
+
+ /** */
+ private byte[] serRow;
+
+ /** */
+ public GenericRowMessage() {
+
+ }
+
+ /** */
+ public GenericRowMessage(Object row) {
+ this.row = row;
+ }
+
+ /** */
+ public Object row() {
+ return row;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+ if (row != null && serRow == null)
+ serRow = marshaller.marshal(row);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+ if (serRow != null && row == null)
+ row = marshaller.unmarshal(serRow, loader);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeByteArray("serRow", serRow))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ serRow = reader.readByteArray("serRow");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(GenericRowMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.GENERIC_ROW_MESSAGE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 1;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCancelMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCancelMessage.java
new file mode 100644
index 0000000..9368052
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCancelMessage.java
@@ -0,0 +1,171 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class InboxCancelMessage implements ExecutionContextAware {
+ /** */
+ private UUID queryId;
+
+ /** */
+ private long fragmentId;
+
+ /** */
+ private long exchangeId;
+
+ /** */
+ private int batchId;
+
+ /** */
+ public InboxCancelMessage(){}
+
+ /** */
+ public InboxCancelMessage(UUID queryId, long fragmentId, long exchangeId, int batchId) {
+ this.queryId = queryId;
+ this.fragmentId = fragmentId;
+ this.exchangeId = exchangeId;
+ this.batchId = batchId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public UUID queryId() {
+ return queryId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long fragmentId() {
+ return fragmentId;
+ }
+
+ /**
+ * @return Exchange ID.
+ */
+ public long exchangeId() {
+ return exchangeId;
+ }
+
+ /**
+ * @return Batch ID.
+ */
+ public int batchId() {
+ return batchId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeInt("batchId", batchId))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeLong("exchangeId", exchangeId))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeLong("fragmentId", fragmentId))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeUuid("queryId", queryId))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ batchId = reader.readInt("batchId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ exchangeId = reader.readLong("exchangeId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ fragmentId = reader.readLong("fragmentId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ queryId = reader.readUuid("queryId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(InboxCancelMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.QUERY_INBOX_CANCEL_MESSAGE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 4;
+ }
+}
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/message/MarshalableMessage.java
similarity index 57%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MarshalableMessage.java
index c995ef2..b555157 100644
--- 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/message/MarshalableMessage.java
@@ -15,23 +15,27 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
-import java.util.List;
-import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.marshaller.Marshaller;
/**
- * Determines where to send a row to.
+ *
*/
-public interface DestinationFunction {
+public interface MarshalableMessage extends CalciteMessage {
/**
- * @param row Row.
- * @return Target nodes list.
+ * Prepares the message before sending.
+ *
+ * @param marshaller Marchaller.
*/
- List<UUID> destination(Object row);
+ void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException;
/**
- * @return All target nodes.
+ * Prepares the message before processing.
+ *
+ * @param marshaller Marchaller.
+ * @param loader Class loader.
*/
- List<UUID> targets();
+ void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException;
}
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/message/MessageListener.java
similarity index 75%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
index 112aad9..61fc897 100644
--- 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/message/MessageListener.java
@@ -15,12 +15,17 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.serialize;
+package org.apache.ignite.internal.processors.query.calcite.message;
-import java.io.Serializable;
+import java.util.UUID;
/**
- * Graph node.
+ *
*/
-public interface GraphNode extends Serializable {
+public interface MessageListener {
+ /**
+ * @param nodeId Sender node ID.
+ * @param msg Message.
+ */
+ void onMessage(UUID nodeId, CalciteMessage msg);
}
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/message/MessageService.java
similarity index 53%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
index c995ef2..02bbe46 100644
--- 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/message/MessageService.java
@@ -15,23 +15,37 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
-import java.util.List;
+import java.util.Collection;
import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
/**
- * Determines where to send a row to.
+ *
*/
-public interface DestinationFunction {
+public interface MessageService extends Service {
+ /**
+ * Sends a message to given nodes.
+ *
+ * @param nodeIds Nodes IDs.
+ * @param msg Message.
+ */
+ void send(Collection<UUID> nodeIds, CalciteMessage msg);
+
/**
- * @param row Row.
- * @return Target nodes list.
+ * Sends a message to given node.
+ *
+ * @param nodeId Node ID.
+ * @param msg Message.
*/
- List<UUID> destination(Object row);
+ void send(UUID nodeId, CalciteMessage msg);
/**
- * @return All target nodes.
+ * Registers a listener for messages of a given type.
+ *
+ * @param lsnr Listener.
+ * @param type Message type.
*/
- List<UUID> targets();
+ void register(MessageListener lsnr, MessageType type);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java
new file mode 100644
index 0000000..c6e3c2b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java
@@ -0,0 +1,287 @@
+/*
+ * 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.message;
+
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.GridTopic;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.managers.communication.GridMessageListener;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.marshaller.MarshallerUtils;
+import org.apache.ignite.plugin.extensions.communication.Message;
+
+/**
+ *
+ */
+public class MessageServiceImpl extends AbstractService implements MessageService {
+ /** */
+ private final GridMessageListener msgLsnr;
+
+ private UUID localNodeId;
+
+ private GridIoManager ioManager;
+
+ private ClassLoader classLoader;
+
+ private QueryTaskExecutor taskExecutor;
+
+ private FailureProcessor failureProcessor;
+
+ private Marshaller marsh;
+
+ /** */
+ private EnumMap<MessageType, MessageListener> lsnrs;
+
+ /** */
+ public MessageServiceImpl(GridKernalContext ctx) {
+ super(ctx);
+
+ msgLsnr = this::onMessage;
+ }
+
+ /**
+ * @param localNodeId Local node ID.
+ */
+ public void localNodeId(UUID localNodeId) {
+ this.localNodeId = localNodeId;
+ }
+
+ /**
+ * @return Local node ID.
+ */
+ public UUID localNodeId() {
+ return localNodeId;
+ }
+
+ /**
+ * @param ioManager IO manager.
+ */
+ public void ioManager(GridIoManager ioManager) {
+ this.ioManager = ioManager;
+ }
+
+ /**
+ * @return IO manager.
+ */
+ public GridIoManager ioManager() {
+ return ioManager;
+ }
+
+ /**
+ * @param classLoader Class loader.
+ */
+ public void classLoader(ClassLoader classLoader) {
+ this.classLoader = classLoader;
+ }
+
+ /**
+ * @return Class loader.
+ */
+ public ClassLoader classLoader() {
+ return classLoader;
+ }
+
+ /**
+ * @param taskExecutor Task executor.
+ */
+ public void taskExecutor(QueryTaskExecutor taskExecutor) {
+ this.taskExecutor = taskExecutor;
+ }
+
+ /**
+ * @return Task executor.
+ */
+ public QueryTaskExecutor taskExecutor() {
+ return taskExecutor;
+ }
+
+ /**
+ * @param marsh Marshaller.
+ */
+ public void marshaller(Marshaller marsh) {
+ this.marsh = marsh;
+ }
+
+ /**
+ * @return Marshaller.
+ */
+ public Marshaller marshaller() {
+ return marsh;
+ }
+
+ /**
+ * @param failureProcessor Failure processor.
+ */
+ public void failureProcessor(FailureProcessor failureProcessor) {
+ this.failureProcessor = failureProcessor;
+ }
+
+ /**
+ * @return Failure processor.
+ */
+ public FailureProcessor failureProcessor() {
+ return failureProcessor;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ localNodeId(ctx.localNodeId());
+ classLoader(U.resolveClassLoader(ctx.config()));
+ ioManager(ctx.io());
+
+ @SuppressWarnings("deprecation")
+ Marshaller marsh0 = ctx.config().getMarshaller();
+
+ if (marsh0 == null) // Stubbed context doesn't have a marshaller
+ marsh0 = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName());
+
+ marshaller(marsh0);
+
+ CalciteQueryProcessor proc = Objects.requireNonNull(Commons.lookupComponent(ctx, CalciteQueryProcessor.class));
+
+ taskExecutor(proc.taskExecutor());
+ failureProcessor(proc.failureProcessor());
+
+ init();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void init() {
+ ioManager().addMessageListener(GridTopic.TOPIC_QUERY, msgLsnr);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void tearDown() {
+ ioManager().removeMessageListener(GridTopic.TOPIC_QUERY, msgLsnr);
+ lsnrs = null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void send(Collection<UUID> nodeIds, CalciteMessage msg) {
+ for (UUID nodeId : nodeIds)
+ send(nodeId, msg);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void send(UUID nodeId, CalciteMessage msg) {
+ if (localNodeId().equals(nodeId))
+ onMessage(nodeId, msg, true);
+ else {
+ byte plc = msg instanceof ExecutionContextAware ?
+ ((ExecutionContextAware) msg).ioPolicy() : GridIoPolicy.QUERY_POOL;
+
+ if (!prepareMarshal(msg))
+ return;
+
+ try {
+ ioManager().sendToGridTopic(nodeId, GridTopic.TOPIC_QUERY, msg, plc);
+ }
+ catch (ClusterTopologyCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send message, node failed: " + nodeId);
+ }
+ catch (IgniteCheckedException e) {
+ failureProcessor().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void register(MessageListener lsnr, MessageType type) {
+ if (lsnrs == null)
+ lsnrs = new EnumMap<>(MessageType.class);
+
+ MessageListener old = lsnrs.put(type, lsnr);
+
+ assert old == null : old;
+ }
+
+ /** */
+ protected void onMessage(UUID nodeId, CalciteMessage msg, boolean async) {
+ if (msg instanceof ExecutionContextAware) {
+ ExecutionContextAware msg0 = (ExecutionContextAware) msg;
+ taskExecutor().execute(msg0.queryId(), msg0.fragmentId(), () -> onMessageInternal(nodeId, msg));
+ }
+ else if (async)
+ taskExecutor().execute(IgniteUuid.VM_ID, ThreadLocalRandom.current().nextLong(1024), () -> onMessageInternal(nodeId, msg));
+ else
+ onMessageInternal(nodeId, msg);
+ }
+
+ /** */
+ protected boolean prepareMarshal(Message msg) {
+ try {
+ if (msg instanceof MarshalableMessage)
+ ((MarshalableMessage) msg).prepareMarshal(marshaller());
+
+ return true;
+ }
+ catch (IgniteCheckedException e) {
+ failureProcessor().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+ }
+
+ return false;
+ }
+
+ /** */
+ protected boolean prepareUnmarshal(Message msg) {
+ try {
+ if (msg instanceof MarshalableMessage)
+ ((MarshalableMessage) msg).prepareUnmarshal(marshaller(), classLoader());
+
+ return true;
+ }
+ catch (IgniteCheckedException e) {
+ failureProcessor().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+ }
+
+ return false;
+ }
+
+ /** */
+ private void onMessage(UUID nodeId, Object msg, byte plc) {
+ if (msg instanceof CalciteMessage)
+ onMessage(nodeId, (CalciteMessage) msg, false);
+ }
+
+ /** */
+ private void onMessageInternal(UUID nodeId, CalciteMessage msg) {
+ if (!prepareUnmarshal(msg))
+ return;
+
+ MessageListener lsnr = Objects.requireNonNull(lsnrs.get(msg.type()));
+ lsnr.onMessage(nodeId, msg);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java
new file mode 100644
index 0000000..24ee972
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java
@@ -0,0 +1,90 @@
+/*
+ * 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.message;
+
+import java.util.function.Supplier;
+
+/**
+ *
+ */
+public enum MessageType {
+ QUERY_START_REQUEST(300, QueryStartRequest::new),
+ QUERY_START_RESPONSE(301, QueryStartResponse::new),
+ QUERY_CANCEL_REQUEST(302, QueryCancelRequest::new),
+ QUERY_BATCH_MESSAGE(303, QueryBatchMessage::new),
+ QUERY_ACKNOWLEDGE_MESSAGE(304, QueryBatchAcknowledgeMessage::new),
+ QUERY_INBOX_CANCEL_MESSAGE(305, InboxCancelMessage::new),
+ GENERIC_ROW_MESSAGE(306, GenericRowMessage::new);
+
+ /** */
+ private final int directType;
+
+ /** */
+ private final Supplier<CalciteMessage> factory;
+
+ /**
+ * @param directType Message direct type.
+ */
+ MessageType(int directType, Supplier<CalciteMessage> factory) {
+ this.directType = directType;
+ this.factory = factory;
+ }
+
+ /**
+ * @return Message direct type;
+ */
+ public short directType() {
+ return (short) directType;
+ }
+
+ /** */
+ private CalciteMessage newMessage() {
+ CalciteMessage msg = factory.get();
+
+ assert msg.type() == this;
+
+ return msg;
+ }
+
+ /**
+ * Message factory method.
+ *
+ * @param directType Message direct type.
+ * @return new message or {@code null} in case of unknown message direct type.
+ */
+ public static CalciteMessage newMessage(short directType) {
+ switch (directType) {
+ case 300:
+ return QUERY_START_REQUEST.newMessage();
+ case 301:
+ return QUERY_START_RESPONSE.newMessage();
+ case 302:
+ return QUERY_CANCEL_REQUEST.newMessage();
+ case 303:
+ return QUERY_BATCH_MESSAGE.newMessage();
+ case 304:
+ return QUERY_ACKNOWLEDGE_MESSAGE.newMessage();
+ case 305:
+ return QUERY_INBOX_CANCEL_MESSAGE.newMessage();
+ case 306:
+ return GENERIC_ROW_MESSAGE.newMessage();
+ default:
+ return null;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
new file mode 100644
index 0000000..7258272
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
@@ -0,0 +1,173 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryBatchAcknowledgeMessage implements ExecutionContextAware {
+ /** */
+ private UUID queryId;
+
+ /** */
+ private long fragmentId;
+
+ /** */
+ private long exchangeId;
+
+ /** */
+ private int batchId;
+
+ /** */
+ public QueryBatchAcknowledgeMessage() {
+
+ }
+
+ /** */
+ public QueryBatchAcknowledgeMessage(UUID queryId, long fragmentId, long exchangeId, int batchId) {
+ this.queryId = queryId;
+ this.fragmentId = fragmentId;
+ this.exchangeId = exchangeId;
+ this.batchId = batchId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public UUID queryId() {
+ return queryId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long fragmentId() {
+ return fragmentId;
+ }
+
+ /**
+ * @return Exchange ID.
+ */
+ public long exchangeId() {
+ return exchangeId;
+ }
+
+ /**
+ * @return Batch ID.
+ */
+ public int batchId() {
+ return batchId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeInt("batchId", batchId))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeLong("exchangeId", exchangeId))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeLong("fragmentId", fragmentId))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeUuid("queryId", queryId))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ batchId = reader.readInt("batchId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ exchangeId = reader.readLong("exchangeId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ fragmentId = reader.readLong("fragmentId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ queryId = reader.readUuid("queryId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(QueryBatchAcknowledgeMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.QUERY_ACKNOWLEDGE_MESSAGE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 4;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java
new file mode 100644
index 0000000..bc15b80
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java
@@ -0,0 +1,246 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryBatchMessage implements MarshalableMessage, ExecutionContextAware {
+ /** */
+ private UUID queryId;
+
+ /** */
+ private long fragmentId;
+
+ /** */
+ private long exchangeId;
+
+ /** */
+ private int batchId;
+
+ /** */
+ @GridDirectTransient
+ private List<Object> rows;
+
+ /** */
+ @GridDirectCollection(Message.class)
+ private List<Message> mRows;
+
+ /** */
+ public QueryBatchMessage() {
+
+ }
+
+ /** */
+ public QueryBatchMessage(UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) {
+ this.queryId = queryId;
+ this.fragmentId = fragmentId;
+ this.exchangeId = exchangeId;
+ this.batchId = batchId;
+ this.rows = Commons.cast(rows);
+ }
+
+ /** {@inheritDoc} */
+ @Override public UUID queryId() {
+ return queryId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long fragmentId() {
+ return fragmentId;
+ }
+
+ /**
+ * @return Exchange ID.
+ */
+ public long exchangeId() {
+ return exchangeId;
+ }
+
+ /**
+ * @return Batch ID.
+ */
+ public int batchId() {
+ return batchId;
+ }
+
+ /**
+ * @return Rows.
+ */
+ public List<Object> rows() {
+ return rows;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+ if (mRows != null || rows == null)
+ return;
+
+ mRows = new ArrayList<>(rows.size());
+
+ for (Object row : rows) {
+ Message mRow = CalciteMessageFactory.asMessage(row);
+
+ if (mRow instanceof MarshalableMessage)
+ ((MarshalableMessage) mRow).prepareMarshal(marshaller);
+
+ mRows.add(mRow);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+ if (rows != null || mRows == null)
+ return;
+
+ rows = new ArrayList<>(mRows.size());
+
+ for (Message mRow : mRows) {
+ if (mRow instanceof MarshalableMessage)
+ ((MarshalableMessage) mRow).prepareUnmarshal(marshaller, loader);
+
+ Object row = CalciteMessageFactory.asRow(mRow);
+
+ rows.add(row);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeInt("batchId", batchId))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeLong("exchangeId", exchangeId))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeLong("fragmentId", fragmentId))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeCollection("mRows", mRows, MessageCollectionItemType.MSG))
+ return false;
+
+ writer.incrementState();
+
+ case 4:
+ if (!writer.writeUuid("queryId", queryId))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ batchId = reader.readInt("batchId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ exchangeId = reader.readLong("exchangeId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ fragmentId = reader.readLong("fragmentId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ mRows = reader.readCollection("mRows", MessageCollectionItemType.MSG);
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 4:
+ queryId = reader.readUuid("queryId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(QueryBatchMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.QUERY_BATCH_MESSAGE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 5;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryCancelRequest.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryCancelRequest.java
new file mode 100644
index 0000000..fe0e2ac
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryCancelRequest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryCancelRequest implements CalciteMessage {
+ /** */
+ private UUID queryId;
+
+ /** */
+ QueryCancelRequest(){}
+
+ /** */
+ public QueryCancelRequest(UUID queryId) {
+ this.queryId = queryId;
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return queryId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeUuid("queryId", queryId))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ queryId = reader.readUuid("queryId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(QueryCancelRequest.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.QUERY_CANCEL_REQUEST;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 1;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
new file mode 100644
index 0000000..c5c386d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
@@ -0,0 +1,283 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryStartRequest implements MarshalableMessage {
+ /** */
+ private String schema;
+
+ /** */
+ private UUID queryId;
+
+ /** */
+ private long fragmentId;
+
+ /** */
+ private int[] partitions;
+
+ /** */
+ private AffinityTopologyVersion version;
+
+ /** */
+ @GridDirectTransient
+ private RelGraph plan;
+
+ /** */
+ private byte[] planBytes;
+
+ /** */
+ @GridDirectTransient
+ private Object[] params;
+
+ /** */
+ private byte[] paramsBytes;
+
+ /** */
+ public QueryStartRequest(UUID queryId, long fragmentId, String schema, RelGraph plan, AffinityTopologyVersion version, int[] partitions, Object[] params) {
+ this.schema = schema;
+ this.queryId = queryId;
+ this.fragmentId = fragmentId;
+ this.partitions = partitions;
+ this.version = version;
+ this.plan = plan;
+ this.params = params;
+ }
+
+ /** */
+ QueryStartRequest() {
+
+ }
+
+ /**
+ * @return Schema name.
+ */
+ public String schema() {
+ return schema;
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return queryId;
+ }
+
+ /**
+ * @return Fragment ID.
+ */
+ public long fragmentId() {
+ return fragmentId;
+ }
+
+ /**
+ * @return Partitions.
+ */
+ public int[] partitions() {
+ return partitions;
+ }
+
+ /**
+ * @return Topology version.
+ */
+ public AffinityTopologyVersion topologyVersion() {
+ return version;
+ }
+
+ /**
+ * @return Fragment plan.
+ */
+ public RelGraph plan() {
+ return plan;
+ }
+
+ /**
+ * @return Query parameters.
+ */
+ public Object[] parameters() {
+ return params;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+ if (planBytes == null && plan != null)
+ planBytes = marshaller.marshal(plan);
+
+ if (paramsBytes == null && params != null)
+ paramsBytes = marshaller.marshal(params);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+ if (plan == null && planBytes != null)
+ plan = marshaller.unmarshal(planBytes, loader);
+
+ if (params == null && paramsBytes != null)
+ params = marshaller.unmarshal(paramsBytes, loader);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeLong("fragmentId", fragmentId))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeByteArray("paramsBytes", paramsBytes))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeIntArray("partitions", partitions))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeByteArray("planBytes", planBytes))
+ return false;
+
+ writer.incrementState();
+
+ case 4:
+ if (!writer.writeUuid("queryId", queryId))
+ return false;
+
+ writer.incrementState();
+
+ case 5:
+ if (!writer.writeString("schema", schema))
+ return false;
+
+ writer.incrementState();
+
+ case 6:
+ if (!writer.writeAffinityTopologyVersion("version", version))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ fragmentId = reader.readLong("fragmentId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ paramsBytes = reader.readByteArray("paramsBytes");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ partitions = reader.readIntArray("partitions");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ planBytes = reader.readByteArray("planBytes");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 4:
+ queryId = reader.readUuid("queryId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 5:
+ schema = reader.readString("schema");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 6:
+ version = reader.readAffinityTopologyVersion("version");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(QueryStartRequest.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.QUERY_START_REQUEST;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 7;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java
new file mode 100644
index 0000000..05140b5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java
@@ -0,0 +1,174 @@
+/*
+ * 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.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryStartResponse implements MarshalableMessage {
+ /** */
+ private UUID queryId;
+
+ /** */
+ private long fragmentId;
+
+ /** */
+ @GridDirectTransient
+ private Throwable error;
+
+ /** */
+ private byte[] errBytes;
+
+ /** */
+ public QueryStartResponse() {}
+
+ /** */
+ public QueryStartResponse(UUID queryId, long fragmentId) {
+ this(queryId, fragmentId, null);
+ }
+
+ /** */
+ public QueryStartResponse(UUID queryId, long fragmentId, Throwable error) {
+ this.queryId = queryId;
+ this.fragmentId = fragmentId;
+ this.error = error;
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return queryId;
+ }
+
+ /**
+ * @return Fragment ID.
+ */
+ public long fragmentId() {
+ return fragmentId;
+ }
+
+ /**
+ * @return Error.
+ */
+ public Throwable error() {
+ return error;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+ if (error != null)
+ errBytes = marshaller.marshal(error);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+ if (errBytes != null)
+ error = marshaller.unmarshal(errBytes, loader);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeByteArray("errBytes", errBytes))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeLong("fragmentId", fragmentId))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeUuid("queryId", queryId))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ errBytes = reader.readByteArray("errBytes");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ fragmentId = reader.readLong("fragmentId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ queryId = reader.readUuid("queryId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(QueryStartResponse.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public MessageType type() {
+ return MessageType.QUERY_START_RESPONSE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 3;
+ }
+}
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
index be4c0e8..b3c5720 100644
--- 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
@@ -71,6 +71,13 @@ public class FragmentInfo {
}
/**
+ * @return {@code True} if the fragment has nodes mapping.
+ */
+ public boolean mapped() {
+ return mapping != null;
+ }
+
+ /**
* @return Pairs of underlying Receiver relational nodes and theirs data source information.
*/
public ImmutableList<Pair<IgniteReceiver, RelSource>> sources() {
@@ -99,9 +106,8 @@ public class FragmentInfo {
if (mapping != null) {
NodesMapping newMapping = mapping.prune(filter);
- if (newMapping != mapping) {
+ if (newMapping != mapping)
return new FragmentInfo(sources, newMapping);
- }
}
return this;
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
index 8271fc0..1c4c940 100644
--- 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
@@ -20,6 +20,7 @@ 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.RelDistribution;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.JoinInfo;
import org.apache.calcite.rel.core.JoinRelType;
@@ -105,7 +106,11 @@ public class IgniteMdDistribution implements MetadataHandler<BuiltInMetadata.Dis
* See {@link IgniteMdDistribution#distribution(RelNode, RelMetadataQuery)}
*/
public IgniteDistribution distribution(LogicalTableScan rel, RelMetadataQuery mq) {
- return rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+ RelDistribution distr = rel.getTable().getDistribution();
+
+ assert distr instanceof IgniteDistribution : distr;
+
+ return (IgniteDistribution) distr;
}
/**
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
index e2ae675..7b82c33 100644
--- 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
@@ -34,13 +34,13 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetada
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
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.schema.DistributedTable;
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;
/**
- * Implementation class for {@link RelMetadataQueryEx#getFragmentLocation(RelNode)} method call.
+ * Implementation class for {@link RelMetadataQueryEx#getFragmentInfo(RelNode)} method call.
*/
public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
/**
@@ -136,7 +136,7 @@ public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
* See {@link IgniteMdFragmentInfo#getFragmentInfo(RelNode, RelMetadataQuery)}
*/
public FragmentInfo getFragmentInfo(IgniteTableScan rel, RelMetadataQuery mq) {
- return rel.getTable().unwrap(IgniteTable.class).fragmentInfo(Commons.plannerContext(rel));
+ return new FragmentInfo(rel.getTable().unwrap(DistributedTable.class).mapping(Commons.context(rel)));
}
/**
@@ -146,7 +146,7 @@ public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
* @return Fragment meta information.
*/
public static FragmentInfo fragmentInfo(RelNode rel, RelMetadataQuery mq) {
- return RelMetadataQueryEx.wrap(mq).getFragmentLocation(rel);
+ return RelMetadataQueryEx.wrap(mq).getFragmentInfo(rel);
}
/** */
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
index c46f733..ba44d60 100644
--- 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
@@ -17,30 +17,25 @@
package org.apache.ignite.internal.processors.query.calcite.metadata;
+import java.util.function.Predicate;
+import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
/**
* Service is responsible for nodes mapping calculation.
*/
-public interface MappingService {
- /**
- * @return Local node mapping that consists of local node only, uses for root query fragment.
- */
- NodesMapping local();
-
+public interface MappingService extends Service {
/**
* Returns Nodes mapping for intermediate fragments, without Scan nodes leafs. Such fragments may be executed
- * on any cluster node, actual list of nodes is chosen on the basis of adopted selection strategy.
+ * on any cluster node, actual list of nodes is chosen on the basis of adopted selection strategy (using node filter).
*
* @param topVer Topology version.
+ * @param desiredCnt desired nodes count, {@code 0} means all possible nodes.
+ * @param nodeFilter Node filter.
* @return Nodes mapping for intermediate fragments.
*/
- NodesMapping random(AffinityTopologyVersion topVer);
-
- /**
- * @param cacheId Cache ID.
- * @param topVer Topology version.
- * @return Nodes mapping for particular table, depends on underlying cache distribution.
- */
- NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer);
+ NodesMapping mapBalanced(@NotNull AffinityTopologyVersion topVer, int desiredCnt, @Nullable Predicate<ClusterNode> nodeFilter);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java
new file mode 100644
index 0000000..f790c63
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java
@@ -0,0 +1,78 @@
+/*
+ * 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.metadata;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping.DEDUPLICATED;
+
+/**
+ *
+ */
+public class MappingServiceImpl extends AbstractService implements MappingService {
+ /** */
+ private GridDiscoveryManager discoveryManager;
+
+ /**
+ * @param ctx Kernal.
+ */
+ public MappingServiceImpl(GridKernalContext ctx) {
+ super(ctx);
+ }
+
+ /**
+ * @param discoveryManager Discovery manager.
+ */
+ public void discoveryManager(GridDiscoveryManager discoveryManager) {
+ this.discoveryManager = discoveryManager;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ discoveryManager(ctx.discovery());
+ }
+
+ /** {@inheritDoc} */
+ @Override public NodesMapping mapBalanced(@NotNull AffinityTopologyVersion topVer, int desiredCnt, @Nullable Predicate<ClusterNode> nodeFilter) {
+ assert desiredCnt >= 0;
+
+ List<ClusterNode> nodes = discoveryManager.discoCache(topVer).serverNodes();
+
+ if (nodeFilter != null)
+ nodes = nodes.stream().filter(nodeFilter).collect(Collectors.toList());
+
+ if (desiredCnt != 0 && desiredCnt < nodes.size()) {
+ Collections.shuffle(nodes);
+
+ nodes = nodes.subList(0, desiredCnt);
+ }
+
+ return new NodesMapping(Commons.transform(nodes, ClusterNode::id), null, DEDUPLICATED);
+ }
+}
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
index 1feecfb..cd28a28 100644
--- 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
@@ -53,6 +53,9 @@ public class NodesMapping implements Serializable {
public static final byte DEDUPLICATED = 1 << 4;
/** */
+ public static final byte CLIENT = 1 << 5;
+
+ /** */
private final List<UUID> nodes;
/** */
@@ -102,7 +105,7 @@ public class NodesMapping implements Serializable {
public NodesMapping mergeWith(NodesMapping other) throws LocationMappingException {
byte flags = (byte) (this.flags | other.flags);
- if ((flags & PARTIALLY_REPLICATED) == 0)
+ if ((flags & (PARTIALLY_REPLICATED | CLIENT)) == 0)
return new NodesMapping(U.firstNotNull(nodes, other.nodes), mergeAssignments(other, null), flags);
List<UUID> nodes;
@@ -216,7 +219,9 @@ public class NodesMapping implements Serializable {
/** */
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;
+ byte flags = (byte) (this.flags | other.flags);
+ List<List<UUID>> left = assignments;
+ List<List<UUID>> right = other.assignments;
if (left == null && right == null)
return null; // nothing to intersect;
@@ -224,7 +229,7 @@ public class NodesMapping implements Serializable {
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)
+ if (nodes == null)
return assignments;
List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
@@ -238,7 +243,7 @@ public class NodesMapping implements Serializable {
partNodes0.add(partNode);
}
- if (partNodes0.isEmpty())
+ if (partNodes0.isEmpty()) // TODO check with partition filters
throw new LocationMappingException("Failed to map fragment to location.");
assignments0.add(partNodes0);
@@ -251,7 +256,8 @@ public class NodesMapping implements Serializable {
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());
+ List<UUID> leftNodes = left.get(i);
+ List<UUID> partNodes = new ArrayList<>(leftNodes.size());
HashSet<UUID> rightNodesSet = new HashSet<>(right.get(i));
for (UUID partNode : leftNodes) {
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
index 9744264..fd4c33b 100644
--- 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
@@ -38,6 +38,16 @@ public class OptimisticPlanningException extends RuntimeException{
}
/**
+ *
+ * @param message Message.
+ * @param edge Edge of query plan, where the exception was thrown.
+ */
+ public OptimisticPlanningException(String message, Edge edge) {
+ super(message);
+ this.edge = edge;
+ }
+
+ /**
* @return Edge of query plan, where the exception was thrown.
*/
public Edge edge() {
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/metadata/PartitionService.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionService.java
index 4079743..54638d1 100644
--- 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/metadata/PartitionService.java
@@ -15,19 +15,20 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.prepare;
+package org.apache.ignite.internal.processors.query.calcite.metadata;
-import java.util.List;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
/**
- * Operation object, encapsulates query execution logic.
+ *
*/
-public interface QueryExecution {
+public interface PartitionService extends Service {
/**
- * Executes a query.
+ * Creates a partition mapping function on the basis of affinity function of cache with given ID.
*
- * @return Query cursor.
+ * @param cacheId Cache ID.
+ * @return Partition mapping function.
*/
- FieldsQueryCursor<List<?>> execute();
+ ToIntFunction<Object> partitionFunction(int cacheId);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionServiceImpl.java
new file mode 100644
index 0000000..2b8e35d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionServiceImpl.java
@@ -0,0 +1,63 @@
+/*
+ * 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.metadata;
+
+import java.util.function.ToIntFunction;
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+
+/**
+ *
+ */
+public class PartitionServiceImpl extends AbstractService implements PartitionService {
+ /** */
+ private GridCacheSharedContext<?,?> cacheSharedContext;
+
+ /**
+ * @param ctx Kernal.
+ */
+ public PartitionServiceImpl(GridKernalContext ctx) {
+ super(ctx);
+ }
+
+ /**
+ * @param cacheSharedContext Cache shared context.
+ */
+ public void cacheSharedContext(GridCacheSharedContext<?,?> cacheSharedContext) {
+ this.cacheSharedContext = cacheSharedContext;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ cacheSharedContext(ctx.cache().context());
+ }
+
+ /** {@inheritDoc} */
+ @Override public ToIntFunction<Object> partitionFunction(int cacheId) {
+ if (cacheId == CU.UNDEFINED_CACHE_ID)
+ return k -> k == null ? 0 : k.hashCode();
+
+ AffinityFunction affinity = cacheSharedContext.cacheContext(cacheId).group().affinityFunction();
+
+ return affinity::partition;
+ }
+}
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
index f63753a..28be9ef 100644
--- 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
@@ -118,7 +118,7 @@ public class RelMetadataQueryEx extends RelMetadataQuery {
* @param rel Relational node.
* @return Fragment meta information.
*/
- public FragmentInfo getFragmentLocation(RelNode rel) {
+ public FragmentInfo getFragmentInfo(RelNode rel) {
for (;;) {
try {
return sourceDistributionHandler.getFragmentInfo(rel, this);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
new file mode 100644
index 0000000..a1c5bf9
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
@@ -0,0 +1,78 @@
+/*
+ * 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.Objects;
+
+/**
+ *
+ */
+public class CacheKey {
+ /** */
+ private final String schemaName;
+
+ /** */
+ private final String query;
+
+ /** */
+ private final Object contextKey;
+
+ /**
+ * @param schemaName Schema name.
+ * @param query Query string.
+ * @param contextKey Optional context key to differ queries with and without/different flags, having an impact
+ * on result plan (like LOCAL flag)
+ */
+ public CacheKey(String schemaName, String query, Object contextKey) {
+ this.schemaName = schemaName;
+ this.query = query;
+ this.contextKey = contextKey;
+ }
+
+ /**
+ * @param schemaName Schema name.
+ * @param query Query string.
+ */
+ public CacheKey(String schemaName, String query) {
+ this(schemaName, query, null);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ CacheKey cacheKey = (CacheKey) o;
+
+ if (!schemaName.equals(cacheKey.schemaName))
+ return false;
+ if (!query.equals(cacheKey.query))
+ return false;
+ return Objects.equals(contextKey, cacheKey.contextKey);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ int result = schemaName.hashCode();
+ result = 31 * result + query.hashCode();
+ result = 31 * result + (contextKey != null ? contextKey.hashCode() : 0);
+ return result;
+ }
+}
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
deleted file mode 100644
index eb1c36a..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import org.apache.calcite.DataContext;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-
-/**
- * Helpers to get named parameters from {@link DataContext}
- */
-public enum ContextValue {
- QUERY_ID("_query_id", GridCacheVersion.class),
- PLANNER_CONTEXT("_planner_context", PlannerContext.class);
-
- /** */
- private final String valueName;
-
- /** */
- private final Class<?> type;
-
- /**
- * @param valueName Value name.
- * @param type value type.
- */
- ContextValue(String valueName, Class<?> type) {
- this.valueName = valueName;
- this.type = type;
- }
-
- /**
- * @return valueName.
- */
- public String valueName() {
- return valueName;
- }
-
- /**
- * @param ctx Data context.
- * @return Parameter value.
- */
- 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
deleted file mode 100644
index 62b4be5..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.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;
-
-/**
- * Runtime context allowing access to the tables in a database.
- */
-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
deleted file mode 100644
index 253f10d..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.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;
-
-/**
- * Operation object, encapsulates query execution logic.
- */
-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.";
-
- ctx.logger().error(msg, e);
-
- throw new IgniteSQLException(msg, IgniteQueryErrorCode.PARSING, e);
- } catch (Exception e) {
- String msg = "Failed to create query execution graph.";
-
- ctx.logger().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
index d237c5d..40dcf0e 100644
--- 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
@@ -20,13 +20,9 @@ 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;
@@ -47,7 +43,6 @@ 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;
@@ -73,10 +68,6 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
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;
/**
@@ -93,7 +84,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
private final FrameworkConfig frameworkConfig;
/** */
- private final Context context;
+ private final PlanningContext ctx;
/** */
private final CalciteConnectionConfig connectionConfig;
@@ -112,7 +103,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
private final SqlRexConvertletTable convertletTable;
/** */
- private final RexExecutor executor;
+ private final RexExecutor rexExecutor;
/** */
private final SchemaPlus defaultSchema;
@@ -133,42 +124,26 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
private SqlValidator validator;
/**
- * @param config Framework config.
+ * @param ctx Planner context.
*/
- 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);
+ IgnitePlanner(PlanningContext ctx) {
+ this.ctx = ctx;
+
+ frameworkConfig = ctx.frameworkConfig();
+ connectionConfig = ctx.connectionConfig();
+ typeFactory = ctx.typeFactory();
+
+ defaultSchema = frameworkConfig.getDefaultSchema();
+ operatorTable = frameworkConfig.getOperatorTable();
+ programs = frameworkConfig.getPrograms();
+ parserConfig = frameworkConfig.getParserConfig();
+ sqlToRelConverterConfig = frameworkConfig.getSqlToRelConverterConfig();
+ convertletTable = frameworkConfig.getConvertletTable();
+ rexExecutor = frameworkConfig.getExecutor();
+ traitDefs = frameworkConfig.getTraitDefs();
}
- /** */
- 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() {
@@ -191,11 +166,18 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
open = false;
}
+ /**
+ * @return Planner context.
+ */
+ public PlanningContext context() {
+ return ctx;
+ }
+
/** */
private void ready() {
if (!open) {
- planner = VolcanoUtils.impatient(new VolcanoPlanner(frameworkConfig.getCostFactory(), context));
- planner.setExecutor(executor);
+ planner = VolcanoUtils.impatient(new VolcanoPlanner(frameworkConfig.getCostFactory(), ctx));
+ planner.setExecutor(rexExecutor);
metadataProvider = new CachingRelMetadataProvider(IgniteMetadata.METADATA_PROVIDER, planner);
validator = new IgniteSqlValidator(operatorTable(), createCatalogReader(), typeFactory, conformance());
@@ -246,21 +228,27 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
* @param graph Relational nodes tree representation.
* @return Root node of relational tree.
*/
- public RelNode convert(RelGraph graph) {
+ public IgniteRel convert(RelGraph graph) {
ready();
- RelOptCluster cluster = createCluster(createRexBuilder());
+ RelOptCluster cluster = createCluster();
RelBuilder relBuilder = createRelBuilder(cluster, createCatalogReader());
return new GraphToRelConverter(this, relBuilder, operatorTable).convert(graph);
}
+ /** Creates a cluster. */
+ RelOptCluster createCluster() {
+ ready();
+
+ return createCluster(createRexBuilder());
+ }
+
/** {@inheritDoc} */
@Override public RelRoot rel(SqlNode sql) {
ready();
- RexBuilder rexBuilder = createRexBuilder();
- RelOptCluster cluster = createCluster(rexBuilder);
+ RelOptCluster cluster = createCluster();
SqlToRelConverter.Config config = SqlToRelConverter.configBuilder()
.withConfig(sqlToRelConverterConfig)
.withTrimUnusedFields(false)
@@ -276,20 +264,6 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
}
/**
- * Splits relational nodes tree into fragments and returns a query plan.
- * @param rel Root node of relational tree.
- * @return Query plan.
- */
- public QueryPlan plan(RelNode rel) {
- ready();
-
- if (rel.getConvention() != IgniteConvention.INSTANCE)
- throw new IllegalArgumentException("Physical node is required.");
-
- return new Splitter().go((IgniteRel) rel);
- }
-
- /**
* Creates an intermediate relational nodes tree representation for a given relational nodes tree.
*
* @param rel Root node of relational tree.
@@ -323,8 +297,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
SqlValidator validator = new IgniteSqlValidator(operatorTable(), catalogReader, typeFactory, conformance);
validator.setIdentifierExpansion(true);
- RexBuilder rexBuilder = createRexBuilder();
- RelOptCluster cluster = createCluster(rexBuilder);
+ RelOptCluster cluster = createCluster();
SqlToRelConverter.Config config = SqlToRelConverter
.configBuilder()
.withConfig(sqlToRelConverterConfig)
@@ -369,7 +342,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
* @param targetTraits Target traits.
* @return The root of the new RelNode tree.
*/
- public RelNode transform(PlannerType plannerType, PlannerPhase plannerPhase, RelNode input, RelTraitSet targetTraits) {
+ public <T extends RelNode> T transform(PlannerType plannerType, PlannerPhase plannerPhase, RelNode input, RelTraitSet targetTraits) {
ready();
RelTraitSet toTraits = targetTraits.simplify();
@@ -380,12 +353,12 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
case HEP:
final HepProgramBuilder programBuilder = new HepProgramBuilder();
- for (RelOptRule rule : plannerPhase.getRules(Commons.plannerContext(context))) {
+ for (RelOptRule rule : plannerPhase.getRules(Commons.context(ctx))) {
programBuilder.addRuleInstance(rule);
}
final HepPlanner hepPlanner =
- new HepPlanner(programBuilder.build(), context, true, null, RelOptCostImpl.FACTORY);
+ new HepPlanner(programBuilder.build(), ctx, true, null, RelOptCostImpl.FACTORY);
hepPlanner.setRoot(input);
@@ -396,7 +369,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
break;
case VOLCANO:
- Program program = Programs.of(plannerPhase.getRules(Commons.plannerContext(context)));
+ Program program = Programs.of(plannerPhase.getRules(Commons.context(ctx)));
output = program.run(planner, input, toTraits,
ImmutableList.of(), ImmutableList.of());
@@ -406,7 +379,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
throw new AssertionError("Unknown planner type: " + plannerType);
}
- return output;
+ return (T) output;
}
/** {@inheritDoc} */
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
index 6fba164..aaa7ad2 100644
--- 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
@@ -17,13 +17,20 @@
package org.apache.ignite.internal.processors.query.calcite.prepare;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SelectScope;
import org.apache.calcite.sql.validate.SqlConformance;
import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.ignite.internal.processors.query.calcite.type.SystemType;
/** Validator. */
public class IgniteSqlValidator extends SqlValidatorImpl {
@@ -44,16 +51,34 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
/** {@inheritDoc} */
@Override protected RelDataType getLogicalSourceRowType(
RelDataType sourceRowType, SqlInsert insert) {
- final RelDataType superType =
- super.getLogicalSourceRowType(sourceRowType, insert);
- return ((JavaTypeFactory) typeFactory).toSql(superType);
+ return typeFactory().toSql(super.getLogicalSourceRowType(sourceRowType, insert));
}
/** {@inheritDoc} */
@Override protected RelDataType getLogicalTargetRowType(
RelDataType targetRowType, SqlInsert insert) {
- final RelDataType superType =
- super.getLogicalTargetRowType(targetRowType, insert);
- return ((JavaTypeFactory) typeFactory).toSql(superType);
+ return typeFactory().toSql(super.getLogicalTargetRowType(targetRowType, insert));
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void addToSelectList(List<SqlNode> list, Set<String> aliases,
+ List<Map.Entry<String, RelDataType>> fieldList, SqlNode exp, SelectScope scope, boolean includeSystemVars) {
+ if (includeSystemVars || !isSystemType(deriveType(scope, exp)))
+ super.addToSelectList(list, aliases, fieldList, exp, scope, includeSystemVars);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isSystemField(RelDataTypeField field) {
+ return isSystemType(field.getType());
+ }
+
+ /** */
+ private boolean isSystemType(RelDataType type) {
+ return type instanceof SystemType;
+ }
+
+ /** */
+ private JavaTypeFactory typeFactory() {
+ return (JavaTypeFactory) typeFactory;
}
}
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
deleted file mode 100644
index 14343a7..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import 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.cache.affinity.AffinityFunction;
-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;
-
-/**
- * Planner context, encapsulates services, kernal context, query string and its flags and parameters and helper methods
- * to work with them.
- */
-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 constructor, used by a builder.
- */
- 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;
- }
-
- /**
- * @return Query and its parameters.
- */
- public Query query() {
- return query;
- }
-
- /**
- * @return Topology version.
- */
- public AffinityTopologyVersion topologyVersion() {
- return topologyVersion;
- }
-
- /**
- * @return Schema.
- */
- public SchemaPlus schema() {
- return schema;
- }
-
- /**
- * @return Logger.
- */
- public IgniteLogger logger() {
- return logger;
- }
-
- /**
- * @return Kernal context.
- */
- public GridKernalContext kernalContext() {
- return kernalContext;
- }
-
- /**
- * Returns an affinityFunction for a given cache ID.
- *
- * @param cacheId Cache ID.
- * @return Affinity function.
- */
- public AffinityFunction affinityFunction(int cacheId) {
- return kernalContext.cache().context().cacheContext(cacheId).group().affinityFunction();
- }
-
- /**
- * @return Query processor.
- */
- public CalciteQueryProcessor queryProcessor() {
- return queryProcessor;
- }
-
- /**
- * Package private method to set a planner after it creates using provided PlannerContext.
- *
- * @param planner Planner.
- */
- void planner(IgnitePlanner planner) {
- this.planner = planner;
- }
-
- /**
- * @return Planner.
- */
- public IgnitePlanner planner() {
- return planner;
- }
-
- /**
- * @return Mapping service.
- */
- public MappingService mappingService() {
- return mappingService;
- }
-
- /**
- * @return Exchange processor.
- */
- public ExchangeProcessor exchangeProcessor() {
- return exchangeProcessor;
- }
-
- // Helper methods
-
- /**
- * @return Type factory.
- */
- public JavaTypeFactory typeFactory() {
- return planner.getTypeFactory();
- }
-
- /**
- * @return Local node mapping that consists of local node only, uses for root query fragment.
- */
- public NodesMapping mapForLocal() {
- return mappingService.local();
- }
-
- /**
- * Returns Nodes mapping for intermediate fragments, without Scan nodes leafs. Such fragments may be executed
- * on any cluster node, actual list of nodes is chosen on the basis of adopted selection strategy.
- *
- * @return Nodes mapping for intermediate fragments.
- */
- public NodesMapping mapForRandom() {
- return mappingService.random(topologyVersion);
- }
-
- /**
- * @param cacheId Cache ID.
- * @return Nodes mapping for particular table, depends on underlying cache distribution.
- */
- public NodesMapping mapForCache(int cacheId) {
- return mappingService.distributed(cacheId, topologyVersion);
- }
-
- /**
- * @return Query provider. Used to execute a query to remote database (federated database case)
- * or to execute a correlated query.
- */
- public QueryProvider queryProvider() {
- return null; // TODO
- }
-
- /** {@inheritDoc} */
- @Override public <C> C unwrap(Class<C> aClass) {
- if (aClass == getClass())
- return aClass.cast(this);
-
- return parentContext.unwrap(aClass);
- }
-
- /**
- * @return Context builder.
- */
- public static Builder builder() {
- return new Builder();
- }
-
- /**
- * Planner context 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;
-
- /**
- * @param parentContext Parent context.
- * @return Builder for chaining.
- */
- public Builder parentContext(Context parentContext) {
- this.parentContext = parentContext;
- return this;
- }
-
- /**
- * @param query Query.
- * @return Builder for chaining.
- */
- public Builder query(Query query) {
- this.query = query;
- return this;
- }
-
- /**
- * @param topologyVersion Topology version.
- * @return Builder for chaining.
- */
- public Builder topologyVersion(AffinityTopologyVersion topologyVersion) {
- this.topologyVersion = topologyVersion;
- return this;
- }
-
- /**
- * @param schema Schema.
- * @return Builder for chaining.
- */
- public Builder schema(SchemaPlus schema) {
- this.schema = schema;
- return this;
- }
-
- /**
- * @param logger Logger.
- * @return Builder for chaining.
- */
- public Builder logger(IgniteLogger logger) {
- this.logger = logger;
- return this;
- }
-
- /**
- * @param kernalContext Kernal context.
- * @return Builder for chaining.
- */
- public Builder kernalContext(GridKernalContext kernalContext) {
- this.kernalContext = kernalContext;
- return this;
- }
-
- /**
- * @param queryProcessor Query processor.
- * @return Builder for chaining.
- */
- public Builder queryProcessor(CalciteQueryProcessor queryProcessor) {
- this.queryProcessor = queryProcessor;
- return this;
- }
-
- /**
- * @param mappingService Mapping service.
- * @return Builder for chaining.
- */
- public Builder mappingService(MappingService mappingService) {
- this.mappingService = mappingService;
- return this;
- }
-
- /**
- * @param exchangeProcessor Exchange processor.
- * @return Builder for chaining.
- */
- public Builder exchangeProcessor(ExchangeProcessor exchangeProcessor) {
- this.exchangeProcessor = exchangeProcessor;
- return this;
- }
-
- /**
- * Builds planner context.
- *
- * @return Planner context.
- */
- 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
index 526ca08..9417af2 100644
--- 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
@@ -31,7 +31,7 @@ public enum PlannerPhase {
/** */
SUBQUERY_REWRITE("Sub-queries rewrites") {
/** {@inheritDoc} */
- @Override public RuleSet getRules(PlannerContext ctx) {
+ @Override public RuleSet getRules(PlanningContext ctx) {
return RuleSets.ofList(
SubQueryRemoveRule.FILTER,
SubQueryRemoveRule.PROJECT,
@@ -42,7 +42,7 @@ public enum PlannerPhase {
/** */
OPTIMIZATION("Main optimization phase") {
/** {@inheritDoc} */
- @Override public RuleSet getRules(PlannerContext ctx) {
+ @Override public RuleSet getRules(PlanningContext ctx) {
return RuleSets.ofList(
JoinConverter.INSTANCE,
ProjectConverter.INSTANCE,
@@ -65,5 +65,5 @@ public enum PlannerPhase {
* @param ctx Planner context.
* @return Rule set.
*/
- public abstract RuleSet getRules(PlannerContext ctx);
+ public abstract RuleSet getRules(PlanningContext ctx);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
new file mode 100644
index 0000000..fabe1d6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
@@ -0,0 +1,342 @@
+/*
+ * 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.Properties;
+import java.util.UUID;
+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.plan.Context;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/**
+ * Planning context.
+ */
+public final class PlanningContext implements Context {
+ /** */
+ private final FrameworkConfig frameworkConfig;
+
+ /** */
+ private final Context parentContext;
+
+ /** */
+ private final UUID localNodeId;
+
+ /** */
+ private final UUID originatingNodeId;
+
+ /** */
+ private final String query;
+
+ /** */
+ private final Object[] parameters;
+
+ /** */
+ private final AffinityTopologyVersion topologyVersion;
+
+ /** */
+ private final IgniteLogger logger;
+
+ /** */
+ private IgnitePlanner planner;
+
+ /** */
+ private CalciteConnectionConfig connectionConfig;
+
+ /** */
+ private JavaTypeFactory typeFactory;
+
+ /**
+ * Private constructor, used by a builder.
+ */
+ private PlanningContext(FrameworkConfig config, Context parentContext, UUID localNodeId, UUID originatingNodeId,
+ String query, Object[] parameters, AffinityTopologyVersion topologyVersion, IgniteLogger logger) {
+ this.parentContext = parentContext;
+ this.localNodeId = localNodeId;
+ this.parameters = parameters;
+ this.originatingNodeId = originatingNodeId == null ? localNodeId : originatingNodeId;
+ this.query = query;
+ this.topologyVersion = topologyVersion;
+ this.logger = logger;
+
+ // link frameworkConfig#context() to this.
+ Frameworks.ConfigBuilder b = config == null ? Frameworks.newConfigBuilder() :
+ Frameworks.newConfigBuilder(config);
+
+ frameworkConfig = b.context(this).build();
+ }
+
+ /**
+ * @return Local node ID.
+ */
+ public UUID localNodeId() {
+ return localNodeId;
+ }
+
+ /**
+ * @return Originating node ID (the node, who started the execution).
+ */
+ public UUID originatingNodeId() {
+ return originatingNodeId;
+ }
+
+ /**
+ * @return Framework config.
+ */
+ public FrameworkConfig frameworkConfig() {
+ return frameworkConfig;
+ }
+
+ /**
+ * @return Query.
+ */
+ public String query() {
+ return query;
+ }
+
+ /**
+ * @return Query parameters.
+ */
+ public Object[] parameters() {
+ return parameters;
+ }
+
+ /**
+ * @return Topology version.
+ */
+ public AffinityTopologyVersion topologyVersion() {
+ return topologyVersion;
+ }
+
+ /**
+ * @return Logger.
+ */
+ public IgniteLogger logger() {
+ return logger;
+ }
+
+ // Helper methods
+
+ /**
+ * @return Planner.
+ */
+ public IgnitePlanner planner() {
+ if (planner == null)
+ planner = new IgnitePlanner(this);
+
+ return planner;
+ }
+
+ /**
+ * @return Schema.
+ */
+ public SchemaPlus schema() {
+ return frameworkConfig.getDefaultSchema();
+ }
+
+ /**
+ * @return Type factory.
+ */
+ public JavaTypeFactory typeFactory() {
+ if (typeFactory != null)
+ return typeFactory;
+
+ RelDataTypeSystem typeSystem = connectionConfig().typeSystem(RelDataTypeSystem.class, frameworkConfig.getTypeSystem());
+
+ return typeFactory = new IgniteTypeFactory(typeSystem);
+ }
+
+ /**
+ * @return Connection config. Defines connected user parameters like TimeZone or Locale.
+ */
+ public CalciteConnectionConfig connectionConfig() {
+ if (connectionConfig != null)
+ return connectionConfig;
+
+ CalciteConnectionConfig connConfig = unwrap(CalciteConnectionConfig.class);
+
+ if (connConfig != null)
+ return connectionConfig = connConfig;
+
+ Properties properties = new Properties();
+
+ properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
+ String.valueOf(frameworkConfig.getParserConfig().caseSensitive()));
+ properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
+ String.valueOf(frameworkConfig.getParserConfig().conformance()));
+
+ return connectionConfig = new CalciteConnectionConfigImpl(properties);
+ }
+
+ /**
+ * @return New cluster based on a planner and its configuration.
+ */
+ public RelOptCluster createCluster() {
+ return planner().createCluster();
+ }
+
+ /** {@inheritDoc} */
+ @Override public <C> C unwrap(Class<C> aClass) {
+ if (aClass == getClass())
+ return aClass.cast(this);
+
+ return parentContext.unwrap(aClass);
+ }
+
+ /**
+ * @return Context builder.
+ */
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ /**
+ * @return Context builder.
+ */
+ public static Builder builder(PlanningContext template) {
+ return new Builder()
+ .logger(template.logger)
+ .topologyVersion(template.topologyVersion)
+ .query(template.query)
+ .parameters(template.parameters)
+ .parentContext(template.parentContext)
+ .frameworkConfig(template.frameworkConfig)
+ .originatingNodeId(template.originatingNodeId)
+ .localNodeId(template.localNodeId);
+ }
+
+ /**
+ * Planner context builder.
+ */
+ public static class Builder {
+ /** */
+ private UUID localNodeId;
+
+ /** */
+ private UUID originatingNodeId;
+
+ /** */
+ private FrameworkConfig frameworkConfig;
+
+ /** */
+ private Context parentContext;
+
+ /** */
+ private String query;
+
+ /** */
+ private Object[] parameters;
+
+ /** */
+ private AffinityTopologyVersion topologyVersion;
+
+ /** */
+ private IgniteLogger logger;
+
+ /**
+ * @param localNodeId Local node ID.
+ * @return Builder for chaining.
+ */
+ public Builder localNodeId(UUID localNodeId) {
+ this.localNodeId = localNodeId;
+ return this;
+ }
+
+ /**
+ * @param originatingNodeId Originating node ID (the node, who started the execution).
+ * @return Builder for chaining.
+ */
+ public Builder originatingNodeId(UUID originatingNodeId) {
+ this.originatingNodeId = originatingNodeId;
+ return this;
+ }
+
+ /**
+ * @param frameworkConfig Framework config.
+ * @return Builder for chaining.
+ */
+ public Builder frameworkConfig(FrameworkConfig frameworkConfig) {
+ this.frameworkConfig = frameworkConfig;
+ return this;
+ }
+
+ /**
+ * @param parentContext Parent context.
+ * @return Builder for chaining.
+ */
+ public Builder parentContext(Context parentContext) {
+ this.parentContext = parentContext;
+ return this;
+ }
+
+ /**
+ * @param query Query.
+ * @return Builder for chaining.
+ */
+ public Builder query(String query) {
+ this.query = query;
+ return this;
+ }
+
+ /**
+ * @param parameters Query parameters.
+ * @return Builder for chaining.
+ */
+ public Builder parameters(Object[] parameters) {
+ this.parameters = parameters;
+ return this;
+ }
+
+ /**
+ * @param topologyVersion Topology version.
+ * @return Builder for chaining.
+ */
+ public Builder topologyVersion(AffinityTopologyVersion topologyVersion) {
+ this.topologyVersion = topologyVersion;
+ return this;
+ }
+
+ /**
+ * @param logger Logger.
+ * @return Builder for chaining.
+ */
+ public Builder logger(IgniteLogger logger) {
+ this.logger = logger;
+ return this;
+ }
+
+ /**
+ * Builds planner context.
+ *
+ * @return Planner context.
+ */
+ public PlanningContext build() {
+ return new PlanningContext(frameworkConfig, parentContext, localNodeId, originatingNodeId, query,
+ parameters, topologyVersion, logger);
+ }
+ }
+}
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
deleted file mode 100644
index 651da36..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Represents a query and its parameters.
- */
-public class Query {
- /** */
- private final String sql;
-
- /** */
- private final Object[] params;
-
- /**
- * @param sql Query text.
- * @param params Query parameters.
- */
- public Query(String sql, Object[] params) {
- this.sql = sql;
- this.params = params;
- }
-
- /**
- * @return Query text.
- */
- public String sql() {
- return sql;
- }
-
- /**
- * @return Query parameters.
- */
- public Object[] params() {
- return params;
- }
-
- /**
- * Populates a given parameters map with named parameters (using theirs ordinal)
- * @param stashed Stashed parameters map.
- * @return Populated parameters map.
- */
- 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;
- }
-
- /** {@inheritDoc} */
- @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);
- }
-
- /** {@inheritDoc} */
- @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/QueryPlanCache.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
index 4079743..0f2586e 100644
--- 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/QueryPlanCache.java
@@ -17,17 +17,18 @@
package org.apache.ignite.internal.processors.query.calcite.prepare;
-import java.util.List;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
/**
- * Operation object, encapsulates query execution logic.
+ *
*/
-public interface QueryExecution {
+public interface QueryPlanCache extends Service {
/**
- * Executes a query.
- *
- * @return Query cursor.
+ * @param ctx Context.
+ * @param key Cache key.
+ * @param factory Factory method to generate a plan on cache miss.
+ * @return Query plan.
*/
- FieldsQueryCursor<List<?>> execute();
+ QueryPlan queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
new file mode 100644
index 0000000..9605be4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
@@ -0,0 +1,115 @@
+/*
+ * 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.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+
+/**
+ *
+ */
+public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCache, SchemaChangeListener {
+ /** */
+ private static final int CACHE_SIZE = 1024;
+
+ /** */
+ private GridInternalSubscriptionProcessor subscriptionProcessor;
+
+ /** */
+ private volatile Map<CacheKey, QueryPlan> cache;
+
+ /**
+ * @param ctx Kernal context.
+ */
+ public QueryPlanCacheImpl(GridKernalContext ctx) {
+ super(ctx);
+
+ cache = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+ subscriptionProcessor(ctx.internalSubscriptionProcessor());
+
+ init();
+ }
+
+ /**
+ * @param subscriptionProcessor Subscription processor.
+ */
+ public void subscriptionProcessor(GridInternalSubscriptionProcessor subscriptionProcessor) {
+ this.subscriptionProcessor = subscriptionProcessor;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void init() {
+ subscriptionProcessor.registerSchemaChangeListener(this);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public QueryPlan queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory) {
+ Map<CacheKey, QueryPlan> cache = this.cache;
+
+ QueryPlan template = cache.get(key);
+
+ if (template != null)
+ return template.clone(ctx.createCluster());
+
+ QueryPlan plan = factory.create(ctx);
+
+ cache.putIfAbsent(key, plan.clone(Commons.EMPTY_CLUSTER));
+
+ return plan;
+ }
+
+ /**
+ * Clear cached plans.
+ */
+ public void clear() {
+ cache = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onSchemaDrop(String schemaName) {
+ clear();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+ clear();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onSchemaCreate(String schemaName) {
+ // No-op
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+ // No-op
+ }
+}
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/QueryPlanFactory.java
similarity index 76%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
index 4079743..da9ad4d 100644
--- 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/QueryPlanFactory.java
@@ -17,17 +17,15 @@
package org.apache.ignite.internal.processors.query.calcite.prepare;
-import java.util.List;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
/**
- * Operation object, encapsulates query execution logic.
+ *
*/
-public interface QueryExecution {
+public interface QueryPlanFactory {
/**
- * Executes a query.
- *
- * @return Query cursor.
+ * @param ctx Planning context.
+ * @return Query plan.
*/
- FieldsQueryCursor<List<?>> execute();
+ QueryPlan create(PlanningContext ctx);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
index c3c869b..5289c90 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
@@ -21,6 +21,8 @@ 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.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
@@ -72,4 +74,11 @@ public class IgniteReceiver extends AbstractRelNode implements IgniteRel {
public IgniteDistribution distribution() {
return getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
}
+
+ /**
+ * @return Node collations.
+ */
+ public List<RelCollation> collations() {
+ return getTraitSet().getTraits(RelCollationTraitDef.INSTANCE);
+ }
}
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/schema/DistributedTable.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/DistributedTable.java
index e10fdb4..e03b73d 100644
--- 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/schema/DistributedTable.java
@@ -15,33 +15,26 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.splitter;
+package org.apache.ignite.internal.processors.query.calcite.schema;
-import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.schema.Table;
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.DistributionFunction;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
/**
*
*/
-public interface RelTarget {
+public interface DistributedTable extends Table {
/**
- * @return Exchange id, has to be unique in scope of query.
+ * Returns locations mapping of the table.
+ * @param ctx Context.
+ * @return Nodes mapping.
*/
- long exchangeId();
+ NodesMapping mapping(PlanningContext ctx);
/**
- * Returns target mapping. It's used in calculation where to send a particular row to.
- * See {@link DistributionFunction#toDestination(PlannerContext, NodesMapping, ImmutableIntList)}
- *
- * @return Target mapping.
- */
- NodesMapping mapping();
-
- /**
- * @return Target distribution.
+ * @return Distribution of the table.
*/
IgniteDistribution distribution();
}
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
index eee1d62..7af76e2 100644
--- 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
@@ -22,10 +22,6 @@ 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;
/**
* Ignite schema.
@@ -59,33 +55,10 @@ public class IgniteSchema extends AbstractSchema {
}
/**
- * 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);
+ public void addTable(String tableName, Table table) {
+ tableMap.put(tableName, table);
}
/**
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
index 6932a17..a5f9184 100644
--- 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
@@ -18,14 +18,18 @@
package org.apache.ignite.internal.processors.query.calcite.schema;
import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
+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.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.RelCollationTraitDef;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelReferentialConstraint;
import org.apache.calcite.rel.type.RelDataType;
@@ -35,111 +39,186 @@ 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.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheStoppedException;
+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.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
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.processors.query.calcite.trait.DistributionFunction;
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;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.TableScan;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
/** */
-public class IgniteTable extends AbstractTable implements TranslatableTable, ScannableTable {
+public class IgniteTable extends AbstractTable implements TranslatableTable, ScannableTable, DistributedTable, SortedTable {
/** */
- private final String tableName;
+ private final String name;
/** */
- private final String cacheName;
+ private final TableDescriptor desc;
/** */
- private final RowType rowType;
-
- /** */
- private final Object identityKey;
+ private final Statistic statistic;
/**
- * @param tableName Table name.
- * @param cacheName Cache name.
- * @param rowType Row type.
- * @param identityKey Affinity identity key.
+ * @param name Table full name.
*/
- public IgniteTable(String tableName, String cacheName, RowType rowType, Object identityKey) {
- this.tableName = tableName;
- this.cacheName = cacheName;
- this.rowType = rowType;
- this.identityKey = identityKey;
- }
+ public IgniteTable(String name, TableDescriptor desc) {
+ this.name = name;
+ this.desc = desc;
- /**
- * @return Table name;
- */
- public String tableName() {
- return tableName;
+ statistic = new StatisticsImpl();
}
/**
- * @return Cache name.
+ * @return Table name.
*/
- public String cacheName() {
- return cacheName;
+ public String name() {
+ return name;
}
/** {@inheritDoc} */
@Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
- return rowType.asRelDataType(typeFactory);
+ return desc.apply(typeFactory);
}
/** {@inheritDoc} */
@Override public Statistic getStatistic() {
- return new TableStatistics();
+ return statistic;
}
/** {@inheritDoc} */
@Override public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
RelOptCluster cluster = context.getCluster();
RelTraitSet traitSet = cluster.traitSetOf(IgniteConvention.INSTANCE)
- .replaceIf(DistributionTraitDef.INSTANCE, this::distribution);
+ .replaceIfs(RelCollationTraitDef.INSTANCE, this::collations)
+ .replaceIf(DistributionTraitDef.INSTANCE, this::distribution);
return new IgniteTableScan(cluster, traitSet, relOptTable);
}
- /**
- * @return Table distribution trait.
- */
- public IgniteDistribution distribution() {
- Object key = identityKey();
+ /** {@inheritDoc} */
+ @Override public NodesMapping mapping(PlanningContext ctx) {
+ GridCacheContext<?, ?> cctx = desc.cacheContext();
+
+ assert cctx != null;
- if (key == null)
- return IgniteDistributions.broadcast();
+ if (!cctx.gate().enterIfNotStopped())
+ throw U.convertException(new CacheStoppedException(cctx.name()));
- return IgniteDistributions.hash(rowType.distributionKeys(), new DistributionFunction.AffinityDistribution(CU.cacheId(cacheName), key));
+ try {
+ if (cctx.isReplicated())
+ return replicatedMapping(cctx, ctx.topologyVersion());
+
+ return partitionedMapping(cctx, ctx.topologyVersion());
+ }
+ finally {
+ cctx.gate().leave();
+ }
}
- /**
- * @return Affinity identity key.
- */
- protected Object identityKey() {
- return identityKey;
+ /** {@inheritDoc} */
+ @Override public IgniteDistribution distribution() {
+ return desc.distribution();
}
- /**
- * @param ctx Planner context.
- * @return Fragment meta information.
- */
- public FragmentInfo fragmentInfo(PlannerContext ctx) {
- return new FragmentInfo(ctx.mapForCache(CU.cacheId(cacheName)));
+ /** {@inheritDoc} */
+ @Override public List<RelCollation> collations() {
+ return desc.collations();
}
/** {@inheritDoc} */
@Override public Enumerable<Object[]> scan(DataContext root) {
- throw new AssertionError(); // TODO
+ return Linq4j.asEnumerable(new TableScan((ExecutionContext) root, desc));
+ }
+
+ /** */
+ private NodesMapping partitionedMapping(@NotNull GridCacheContext<?,?> cctx, @NotNull 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 replicatedMapping(@NotNull GridCacheContext<?,?> cctx, @NotNull 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;
}
/** */
- private class TableStatistics implements Statistic {
+ private class StatisticsImpl implements Statistic {
/** {@inheritDoc} */
@Override public Double getRowCount() {
return null;
@@ -157,11 +236,11 @@ public class IgniteTable extends AbstractTable implements TranslatableTable, Sca
/** {@inheritDoc} */
@Override public List<RelCollation> getCollations() {
- return ImmutableList.of();
+ return collations();
}
/** {@inheritDoc} */
- @Override public RelDistribution getDistribution() {
+ @Override public IgniteDistribution getDistribution() {
return distribution();
}
}
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/schema/SchemaHolder.java
similarity index 73%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
index 112aad9..311ef65 100644
--- 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/schema/SchemaHolder.java
@@ -15,12 +15,17 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.serialize;
+package org.apache.ignite.internal.processors.query.calcite.schema;
-import java.io.Serializable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
/**
- * Graph node.
+ *
*/
-public interface GraphNode extends Serializable {
+public interface SchemaHolder extends Service {
+ /**
+ * @return Schema.
+ */
+ SchemaPlus schema();
}
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/SchemaHolderImpl.java
similarity index 55%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
index 3c3433e..83f2453 100644
--- 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/SchemaHolderImpl.java
@@ -21,22 +21,46 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.GridKernalContext;
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.calcite.util.AbstractService;
import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
/**
* Holds actual schema and mutates it on schema change, requested by Ignite.
*/
-public class CalciteSchemaHolder implements SchemaChangeListener {
+public class SchemaHolderImpl extends AbstractService implements SchemaHolder, SchemaChangeListener {
/** */
private final Map<String, IgniteSchema> schemas = new HashMap<>();
/** */
+ private GridInternalSubscriptionProcessor subscriptionProcessor;
+
+ /** */
private volatile SchemaPlus schema;
/**
+ * @param ctx Kernal context.
+ */
+ public SchemaHolderImpl(GridKernalContext ctx) {
+ super(ctx);
+
+ subscriptionProcessor(ctx.internalSubscriptionProcessor());
+
+ init();
+ }
+
+ /**
+ * @param subscriptionProcessor Subscription processor.
+ */
+ public void subscriptionProcessor(GridInternalSubscriptionProcessor subscriptionProcessor) {
+ this.subscriptionProcessor = subscriptionProcessor;
+ }
+
+ /**
* Sets updated schema.
* @param schema New schema.
*/
@@ -44,10 +68,18 @@ public class CalciteSchemaHolder implements SchemaChangeListener {
this.schema = schema;
}
- /**
- * @return Actual schema.
- */
- public SchemaPlus schema() {
+ /** {@inheritDoc} */
+ @Override public void init() {
+ subscriptionProcessor.registerSchemaChangeListener(this);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onStart(GridKernalContext ctx) {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public SchemaPlus schema() {
return schema;
}
@@ -64,21 +96,36 @@ public class CalciteSchemaHolder implements SchemaChangeListener {
}
/** {@inheritDoc} */
- @Override public synchronized void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
- schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeCreate(typeDescriptor, cacheInfo);
+ @Override public synchronized void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+ IgniteSchema schema = schemas.computeIfAbsent(schemaName, IgniteSchema::new);
+
+ String tableName = typeDescriptor.tableName();
+ TableDescriptorImpl desc = new TableDescriptorImpl(cacheInfo.cacheContext(), typeDescriptor, affinityIdentity(cacheInfo));
+
+ schema.addTable(tableName, new IgniteTable(tableName, desc));
+
rebuild();
}
/** {@inheritDoc} */
- @Override public synchronized void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
- schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeDrop(typeDescriptor, cacheInfo);
+ @Override public synchronized void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+ IgniteSchema schema = schemas.computeIfAbsent(schemaName, IgniteSchema::new);
+
+ schema.removeTable(typeDescriptor.tableName());
+
rebuild();
}
/** */
+ private Object affinityIdentity(GridCacheContextInfo<?, ?> cacheInfo) {
+ return cacheInfo.config().getCacheMode() == CacheMode.PARTITIONED ?
+ cacheInfo.cacheContext().group().affinity().similarAffinityKey() : null;
+ }
+
+ /** */
private void rebuild() {
SchemaPlus schema = Frameworks.createRootSchema(false);
schemas.forEach(schema::add);
- schema(schema.getSubSchema(QueryUtils.DFLT_SCHEMA));
+ schema(schema);
}
}
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/schema/SortedTable.java
similarity index 75%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SortedTable.java
index 112aad9..09bc18a 100644
--- 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/schema/SortedTable.java
@@ -15,12 +15,17 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite.serialize;
+package org.apache.ignite.internal.processors.query.calcite.schema;
-import java.io.Serializable;
+import java.util.List;
+import org.apache.calcite.rel.RelCollation;
/**
- * Graph node.
+ *
*/
-public interface GraphNode extends Serializable {
+public interface SortedTable {
+ /**
+ * @return The table collations.
+ */
+ List<RelCollation> collations();
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
new file mode 100644
index 0000000..a70809f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.List;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public interface TableDescriptor extends RelProtoDataType {
+ /**
+ * @return Underlying cache context.
+ */
+ GridCacheContext<?,?> cacheContext();
+
+ /**
+ * @return Distribution.
+ */
+ IgniteDistribution distribution();
+
+ /**
+ * @return Collations.
+ */
+ List<RelCollation> collations();
+
+ /**
+ * Checks whether a provided cache row belongs to described table.
+ *
+ * @param row Cache row.
+ * @return {@code True} If a provided cache row matches a defined query type.
+ */
+ boolean match(CacheDataRow row);
+
+ /**
+ * Converts a cache row to relational node row.
+ *
+ * @param ectx Execution context.
+ * @param row Cache row.
+ * @return Relational node row.
+ * @throws IgniteCheckedException If failed.
+ */
+ <T> T toRow(ExecutionContext ectx, CacheDataRow row) throws IgniteCheckedException;
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
new file mode 100644
index 0000000..31c607f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
@@ -0,0 +1,131 @@
+/*
+ * 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 java.util.Map;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+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.IgniteTypeFactory;
+
+/**
+ *
+ */
+public class TableDescriptorImpl implements TableDescriptor {
+ /** */
+ private final GridCacheContext<?, ?> cctx;
+
+ /** */
+ private final GridQueryTypeDescriptor typeDesc;
+
+ /** */
+ private final Object affinityIdentity;
+
+ /** */
+ private final int affinityFieldIdx;
+
+ /** */
+ public TableDescriptorImpl(GridCacheContext<?,?> cctx, GridQueryTypeDescriptor typeDesc, Object affinityIdentity) {
+ this.cctx = cctx;
+ this.typeDesc = typeDesc;
+ this.affinityIdentity = affinityIdentity;
+
+ affinityFieldIdx = lookupAffinityIndex(typeDesc);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType apply(RelDataTypeFactory factory) {
+ IgniteTypeFactory f = (IgniteTypeFactory) factory;
+
+ RelDataTypeFactory.Builder b = new RelDataTypeFactory.Builder(f);
+
+ b.add(QueryUtils.KEY_FIELD_NAME, f.createSystemType(typeDesc.keyClass()));
+ b.add(QueryUtils.VAL_FIELD_NAME, f.createSystemType(typeDesc.valueClass()));
+
+ for (Map.Entry<String, Class<?>> field : typeDesc.fields().entrySet())
+ b.add(field.getKey(), f.createJavaType(field.getValue()));
+
+ return b.build();
+ }
+
+ /** {@inheritDoc} */
+ @Override public GridCacheContext<?, ?> cacheContext() {
+ return cctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteDistribution distribution() {
+ if (affinityIdentity == null)
+ return IgniteDistributions.broadcast();
+
+ return IgniteDistributions.affinity(affinityFieldIdx, cctx.cacheId(), affinityIdentity);
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelCollation> collations() {
+ return ImmutableList.of();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean match(CacheDataRow row) {
+ return typeDesc.matchType(row.value());
+ }
+
+ /** {@inheritDoc} */
+ @Override public <T> T toRow(ExecutionContext ectx, CacheDataRow row) throws IgniteCheckedException {
+ Object[] res = new Object[typeDesc.fields().size() + 2];
+
+ int i = 0;
+
+ res[i++] = cctx.unwrapBinaryIfNeeded(row.key(), ectx.keepBinary());
+ res[i++] = cctx.unwrapBinaryIfNeeded(row.value(), ectx.keepBinary());
+
+ for (String field : typeDesc.fields().keySet())
+ res[i++] = cctx.unwrapBinaryIfNeeded(typeDesc.value(field, row.key(), row.value()), ectx.keepBinary());
+
+ return (T) res;
+ }
+
+ /** */
+ private static int lookupAffinityIndex(GridQueryTypeDescriptor queryTypeDesc) {
+ if (queryTypeDesc.affinityKey() != null) {
+ int idx = 2;
+
+ String affField = queryTypeDesc.affinityKey();
+
+ for (String s : queryTypeDesc.fields().keySet()) {
+ if (affField.equals(s))
+ return idx;
+
+ idx++;
+ }
+ }
+
+ return 0;
+ }
+}
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
deleted file mode 100644
index 3230b3a..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.serialize;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.GridIntList;
-
-/**
- * A graph, represented by adjacency list.
- */
-public class Graph<T extends GraphNode> implements Serializable {
- /** */
- private final List<T> nodes = new ArrayList<>();
-
- /** */
- private final List<GridIntList> edges = new ArrayList<>();
-
- /**
- * @return Pairs of node and its ID.
- */
- public List<Ord<T>> nodes() {
- return Ord.zip(nodes);
- }
-
- /**
- * @return Edges.
- */
- public List<ImmutableIntList> edges() {
- return Commons.transform(edges, l -> ImmutableIntList.of(l.array()));
- }
-
- /**
- * Adds a new node and link it as a child of node with given ID.
- *
- * @param parentId Parent node ID.
- * @param node Node.
- * @return New node ID.
- */
- public int addNode(int parentId, T node) {
- int id = addNode(node);
-
- addEdge(parentId, id);
-
- return id;
- }
-
- /**
- * Adds a new node.
- *
- * @param node Node.
- * @return New node ID.
- */
- public int addNode(T node) {
- assert nodes.size() == edges.size();
-
- int id = nodes.size();
-
- nodes.add(node);
- edges.add(new GridIntList());
-
- return id;
- }
-
- /**
- * Adds a new edge.
- *
- * @param parentId Parent node ID.
- * @param childId Child node 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);
- }
-
- /**
- * Returns children of a node with given ID.
- *
- * @param parentId Parent node ID.
- * @return Pairs of child node and its ID.
- */
- 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/expression/CallExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
index 8087e53..124df13 100644
--- 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
@@ -18,8 +18,11 @@
package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.SqlSyntax;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
/**
* Describes {@link org.apache.calcite.rex.RexCall}.
@@ -32,15 +35,20 @@ public class CallExpression implements Expression {
private final SqlSyntax opSyntax;
/** */
+ private DataType type;
+
+ /** */
private final List<Expression> operands;
/**
* @param op Sql operation.
+ * @param type Result type.
* @param operands Operands.
*/
- public CallExpression(SqlOperator op, List<Expression> operands) {
- this.operands = operands;
+ public CallExpression(SqlOperator op, RelDataType type, List<Expression> operands) {
opName = op.getName();
+ this.type = Types.fromType(type);
+ this.operands = operands;
opSyntax = op.getSyntax();
}
@@ -59,6 +67,13 @@ public class CallExpression implements Expression {
}
/**
+ * @return Data type.
+ */
+ public DataType dataType() {
+ return type;
+ }
+
+ /**
* @return Operands.
*/
public List<Expression> operands() {
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
index 74fe3c7..09e5290 100644
--- 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
@@ -19,6 +19,7 @@ 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;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
/**
* Describes {@link org.apache.calcite.rex.RexDynamicParam}.
@@ -35,7 +36,7 @@ public class DynamicParamExpression implements Expression {
* @param index Index.
*/
public DynamicParamExpression(RelDataType type, int index) {
- this.type = DataType.fromType(type);
+ this.type = Types.fromType(type);
this.index = index;
}
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
index 176a1a0..3ee1784 100644
--- 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
@@ -81,7 +81,7 @@ public class ExpToRexTranslator implements ExpImplementor<RexNode> {
/** {@inheritDoc} */
@Override public RexNode implement(CallExpression exp) {
- return builder.makeCall(op(exp.name(), exp.syntax()), translate(exp.operands()));
+ return builder.makeCall(exp.dataType().toRelDataType(builder.getTypeFactory()), op(exp.name(), exp.syntax()), translate(exp.operands()));
}
/** {@inheritDoc} */
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
index 1facc80..840b198 100644
--- 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
@@ -19,6 +19,7 @@ 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;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
/**
* Describes {@link org.apache.calcite.rex.RexInputRef}.
@@ -35,7 +36,7 @@ public class InputRefExpression implements Expression {
* @param index Index.
*/
public InputRefExpression(RelDataType type, int index) {
- this.type = DataType.fromType(type);
+ this.type = Types.fromType(type);
this.index = index;
}
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
index 7ba50b2..ccac4ea 100644
--- 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
@@ -19,24 +19,24 @@ 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;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
/**
* Describes {@link org.apache.calcite.rex.RexLiteral}.
*/
-@SuppressWarnings("rawtypes")
public class LiteralExpression implements Expression {
/** */
private final DataType type;
/** */
- private final Comparable value;
+ private final Comparable<?> value;
/**
* @param type Data type.
* @param value Value.
*/
- public LiteralExpression(RelDataType type, Comparable value) {
- this.type = DataType.fromType(type);
+ public LiteralExpression(RelDataType type, Comparable<?> value) {
+ this.type = Types.fromType(type);
this.value = value;
}
@@ -50,7 +50,7 @@ public class LiteralExpression implements Expression {
/**
* @return Value.
*/
- public Comparable value() {
+ public Comparable<?> value() {
return value;
}
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
index a152180..521a3229 100644
--- 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
@@ -19,6 +19,7 @@ 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;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
/**
* Describes {@link org.apache.calcite.rex.RexLocalRef}.
@@ -35,7 +36,7 @@ public class LocalRefExpression implements Expression {
* @param index Index.
*/
... 7670 lines suppressed ...