You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2021/07/22 09:09:36 UTC
[ignite-3] branch main updated: Revert "IGNITE-14836 Integrate
execution of SELECT queries into Ignite 3.0 (#213)"
This is an automated email from the ASF dual-hosted git repository.
sk0x50 pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new 44e162a Revert "IGNITE-14836 Integrate execution of SELECT queries into Ignite 3.0 (#213)"
44e162a is described below
commit 44e162a6c89cfdfa187b1e5cb85347cd4d756eb8
Author: Slava Koptilin <sl...@gmail.com>
AuthorDate: Thu Jul 22 12:07:26 2021 +0300
Revert "IGNITE-14836 Integrate execution of SELECT queries into Ignite 3.0 (#213)"
This reverts commit 50ea1ed9d420d2e525d9935bfb02f75ee0938f1f.
---
modules/calcite/pom.xml | 11 -
.../query/calcite/SqlQueryProcessor.java | 160 ----
.../internal/processors/query/calcite/Stubs.java | 54 --
.../calcite/exec/ClosableIteratorsHolder.java | 168 ----
.../query/calcite/exec/ExchangeService.java | 84 --
.../query/calcite/exec/ExchangeServiceImpl.java | 262 -------
.../query/calcite/exec/ExecutionContext.java | 4 +-
.../query/calcite/exec/ExecutionService.java | 45 --
.../query/calcite/exec/ExecutionServiceImpl.java | 865 ---------------------
.../query/calcite/exec/LogicalRelImplementor.java | 654 ----------------
.../query/calcite/exec/MailboxRegistry.java | 99 ---
.../query/calcite/exec/MailboxRegistryImpl.java | 169 ----
.../query/calcite/exec/QueryTaskExecutorImpl.java | 4 +-
.../processors/query/calcite/exec/rel/Inbox.java | 519 -------------
.../processors/query/calcite/exec/rel/Outbox.java | 361 ---------
.../query/calcite/exec/rel/RootNode.java | 16 +-
.../query/calcite/message/ErrorMessage.java | 45 --
.../message/ExecutionContextAwareMessage.java | 38 -
.../query/calcite/message/InboxCloseMessage.java | 45 --
.../query/calcite/message/MessageListener.java | 31 -
.../query/calcite/message/MessageService.java | 50 --
.../query/calcite/message/MessageServiceImpl.java | 149 ----
.../query/calcite/message/OutboxCloseMessage.java | 45 --
.../message/QueryBatchAcknowledgeMessage.java | 36 -
.../query/calcite/message/QueryBatchMessage.java | 48 --
.../query/calcite/message/QueryStartRequest.java | 52 --
.../query/calcite/message/QueryStartResponse.java | 45 --
.../calcite/message/SqlQueryMessageGroup.java | 40 -
.../query/calcite/metadata/AffinityService.java | 33 -
.../query/calcite/metadata/ColocationGroup.java | 50 +-
.../calcite/metadata/FragmentDescription.java | 12 +-
.../query/calcite/metadata/FragmentMapping.java | 12 +-
.../query/calcite/metadata/IgniteMdRowCount.java | 6 +-
.../query/calcite/metadata/MappingService.java | 3 +-
.../query/calcite/metadata/MappingServiceImpl.java | 60 --
.../query/calcite/metadata/RemoteException.java | 68 --
.../calcite/prepare/AbstractMultiStepPlan.java | 14 +-
.../processors/query/calcite/prepare/CacheKey.java | 78 --
.../processors/query/calcite/prepare/Cloner.java | 16 +-
.../processors/query/calcite/prepare/DdlPlan.java | 46 --
.../query/calcite/prepare/DummyPlanCache.java | 31 -
.../query/calcite/prepare/ExplainPlan.java | 58 --
.../query/calcite/prepare/FieldsMetadata.java | 37 -
.../query/calcite/prepare/FieldsMetadataImpl.java | 47 --
.../processors/query/calcite/prepare/Fragment.java | 5 +-
.../query/calcite/prepare/FragmentPlan.java | 49 --
.../query/calcite/prepare/IgniteRelShuttle.java | 22 +-
.../query/calcite/prepare/MultiStepDmlPlan.java | 40 -
.../query/calcite/prepare/MultiStepPlan.java | 8 +-
.../query/calcite/prepare/MultiStepQueryPlan.java | 2 +-
.../query/calcite/prepare/PlannerPhase.java | 9 +-
.../query/calcite/prepare/PlanningContext.java | 21 +-
.../query/calcite/prepare/QueryPlanCache.java | 38 -
.../query/calcite/prepare/QueryPlanFactory.java | 31 -
.../query/calcite/rel/IgniteRelVisitor.java | 16 +-
.../query/calcite/rel/set/IgniteIntersect.java | 68 --
.../query/calcite/rel/set/IgniteMapIntersect.java | 74 --
.../query/calcite/rel/set/IgniteMapMinus.java | 73 +-
.../query/calcite/rel/set/IgniteMapSetOp.java | 108 ---
.../query/calcite/rel/set/IgniteMinus.java | 71 --
.../calcite/rel/set/IgniteReduceIntersect.java | 90 ---
.../query/calcite/rel/set/IgniteReduceMinus.java | 51 +-
.../query/calcite/rel/set/IgniteReduceSetOp.java | 75 --
.../query/calcite/rel/set/IgniteSetOp.java | 78 --
.../calcite/rel/set/IgniteSingleIntersect.java | 68 --
.../query/calcite/rel/set/IgniteSingleMinus.java | 72 +-
.../query/calcite/rel/set/IgniteSingleSetOp.java | 97 ---
.../query/calcite/rule/SetOpConverterRule.java | 186 -----
.../query/calcite/schema/ColumnDescriptor.java | 3 +
.../query/calcite/schema/ColumnDescriptorImpl.java | 93 ---
.../query/calcite/schema/IgniteTable.java | 21 -
.../query/calcite/schema/IgniteTableImpl.java | 207 -----
.../query/calcite/schema/SchemaHolder.java | 30 -
.../query/calcite/schema/SchemaHolderImpl.java | 124 ---
.../query/calcite/schema/TableDescriptor.java | 10 -
.../query/calcite/schema/TableDescriptorImpl.java | 149 ----
.../query/calcite/trait/AffinityAdapter.java | 51 --
.../processors/query/calcite/trait/AllNodes.java | 41 -
.../query/calcite/trait/Destination.java | 36 -
.../query/calcite/trait/DistributionFunction.java | 87 ---
.../query/calcite/trait/DistributionTrait.java | 11 +-
.../query/calcite/trait/IgniteDistribution.java | 14 -
.../query/calcite/trait/Partitioned.java | 50 --
.../processors/query/calcite/trait/RandomNode.java | 48 --
.../processors/query/calcite/util/Commons.java | 101 +--
.../query/calcite/util/NodeLeaveHandler.java | 40 -
.../calcite/util/StripedThreadPoolExecutor.java | 2 +
.../calcite/exec/rel/AbstractExecutionTest.java | 23 +-
.../query/calcite/planner/AbstractPlannerTest.java | 28 +-
.../query/calcite/planner/ExceptPlannerTest.java | 329 ++++++++
.../query/calcite/planner/PlannerTest.java | 6 +-
.../query/calcite/planner/SetOpPlannerTest.java | 421 ----------
...tionPlannerTest.java => TableFunctionTest.java} | 2 +-
.../apache/ignite/internal/util/Cancellable.java | 29 -
modules/runner/pom.xml | 5 -
.../org/apache/ignite/internal/app/IgniteImpl.java | 11 +-
.../apache/ignite/internal/app/IgnitionImpl.java | 12 +-
modules/schema/README.md | 2 +-
.../internal/table/event/TableEventParameters.java | 4 +-
parent/pom.xml | 6 -
100 files changed, 691 insertions(+), 7327 deletions(-)
diff --git a/modules/calcite/pom.xml b/modules/calcite/pom.xml
index 9362b1c..833930a 100644
--- a/modules/calcite/pom.xml
+++ b/modules/calcite/pom.xml
@@ -51,17 +51,6 @@
</dependency>
<dependency>
- <groupId>org.apache.ignite</groupId>
- <artifactId>ignite-table</artifactId>
- </dependency>
-
- <dependency>
- <groupId>org.apache.ignite</groupId>
- <artifactId>ignite-network-annotation-processor</artifactId>
- <optional>true</optional>
- </dependency>
-
- <dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
</dependency>
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlQueryProcessor.java
deleted file mode 100644
index f3a8955..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlQueryProcessor.java
+++ /dev/null
@@ -1,160 +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;
-
-import java.util.List;
-
-import org.apache.ignite.internal.manager.EventListener;
-import org.apache.ignite.internal.processors.query.calcite.exec.ArrayRowHandler;
-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.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.prepare.DummyPlanCache;
-import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolderImpl;
-import org.apache.ignite.internal.processors.query.calcite.util.StripedThreadPoolExecutor;
-import org.apache.ignite.internal.table.distributed.TableManager;
-import org.apache.ignite.internal.table.event.TableEvent;
-import org.apache.ignite.internal.table.event.TableEventParameters;
-import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.network.ClusterService;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-public class SqlQueryProcessor {
- /** Default Ignite thread keep alive time. */
- public static final long DFLT_THREAD_KEEP_ALIVE_TIME = 60_000L;
-
- private final ExecutionService executionSrvc;
-
- private final MessageService msgSrvc;
-
- private final QueryTaskExecutor taskExecutor;
-
- public SqlQueryProcessor(
- ClusterService clusterSrvc,
- TableManager tableManager
- ) {
- taskExecutor = new QueryTaskExecutorImpl(
- new StripedThreadPoolExecutor(
- 4,
- "calciteQry",
- null,
- true,
- DFLT_THREAD_KEEP_ALIVE_TIME
- )
- );
-
- msgSrvc = new MessageServiceImpl(
- clusterSrvc.topologyService(),
- clusterSrvc.messagingService(),
- taskExecutor
- );
-
- SchemaHolderImpl schemaHolder = new SchemaHolderImpl(clusterSrvc.topologyService());
-
- executionSrvc = new ExecutionServiceImpl<>(
- clusterSrvc.topologyService(),
- msgSrvc,
- new DummyPlanCache(),
- schemaHolder,
- taskExecutor,
- ArrayRowHandler.INSTANCE
- );
-
- tableManager.listen(TableEvent.CREATE, new TableCreatedListener(schemaHolder));
- tableManager.listen(TableEvent.ALTER, new TableUpdatedListener(schemaHolder));
- tableManager.listen(TableEvent.DROP, new TableDroppedListener(schemaHolder));
- }
-
- public List<Cursor<List<?>>> query(String schemaName, String qry, Object... params) {
- return executionSrvc.executeQuery(schemaName, qry, params);
- }
-
- private abstract static class AbstractTableEventListener implements EventListener<TableEventParameters> {
- protected final SchemaHolderImpl schemaHolder;
-
- private AbstractTableEventListener(
- SchemaHolderImpl schemaHolder
- ) {
- this.schemaHolder = schemaHolder;
- }
-
- /** {@inheritDoc} */
- @Override public void remove(@NotNull Throwable exception) {
- throw new IllegalStateException();
- }
- }
-
- private static class TableCreatedListener extends AbstractTableEventListener {
- private TableCreatedListener(
- SchemaHolderImpl schemaHolder
- ) {
- super(schemaHolder);
- }
-
- /** {@inheritDoc} */
- @Override public boolean notify(@NotNull TableEventParameters parameters, @Nullable Throwable exception) {
- schemaHolder.onSqlTypeCreated(
- "PUBLIC",
- parameters.tableName(),
- parameters.table().schemaView().schema()
- );
-
- return false;
- }
- }
-
- private static class TableUpdatedListener extends AbstractTableEventListener {
- private TableUpdatedListener(
- SchemaHolderImpl schemaHolder
- ) {
- super(schemaHolder);
- }
-
- /** {@inheritDoc} */
- @Override public boolean notify(@NotNull TableEventParameters parameters, @Nullable Throwable exception) {
- schemaHolder.onSqlTypeUpdated(
- "PUBLIC",
- parameters.tableName(),
- parameters.table().schemaView().schema()
- );
-
- return false;
- }
- }
-
- private static class TableDroppedListener extends AbstractTableEventListener {
- private TableDroppedListener(
- SchemaHolderImpl schemaHolder
- ) {
- super(schemaHolder);
- }
-
- /** {@inheritDoc} */
- @Override public boolean notify(@NotNull TableEventParameters parameters, @Nullable Throwable exception) {
- schemaHolder.onSqlTypeDropped(
- "PUBLIC",
- parameters.tableName()
- );
-
- return false;
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
index 6ebc28c..093146c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
@@ -16,17 +16,6 @@
*/
package org.apache.ignite.internal.processors.query.calcite;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-
/** Stubs */
public class Stubs {
/** */
@@ -42,47 +31,4 @@ public class Stubs {
public static String stringFoo(Object... args) {
return args == null ? "null" : "not null";
}
-
- public static Collection<Object[]> resultSetGenerator(int cnt, IgniteTypeFactory factory, RelDataType type) {
- List<Object[]> res = new ArrayList<>(cnt);
-
- for (int row = 0; row < cnt; row++) {
- Object[] tmp = new Object[type.getFieldCount()];
-
- res.add(tmp);
-
- for (RelDataTypeField field : type.getFieldList())
- tmp[field.getIndex()] = rndValueOfType(factory.getJavaClass(field.getType()));
- }
-
- return res;
- }
-
- private static Object rndValueOfType(Type type) {
- if (type == byte.class || type == Byte.class)
- return (byte)ThreadLocalRandom.current().nextInt(100);
-
- if (type == short.class || type == Short.class)
- return (short)ThreadLocalRandom.current().nextInt(100);
-
- if (type == int.class || type == Integer.class)
- return ThreadLocalRandom.current().nextInt(100);
-
- if (type == long.class || type == Long.class)
- return (long)ThreadLocalRandom.current().nextInt(100);
-
- if (type == float.class || type == Float.class)
- return ThreadLocalRandom.current().nextFloat();
-
- if (type == double.class || type == Double.class)
- return ThreadLocalRandom.current().nextDouble();
-
- if (type == UUID.class)
- return UUID.randomUUID();
-
- if (type == String.class)
- return UUID.randomUUID().toString();
-
- throw new IllegalStateException("Can't generate value of type " + type.getTypeName());
- }
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolder.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolder.java
deleted file mode 100644
index 56bb0e4..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolder.java
+++ /dev/null
@@ -1,168 +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.exec;
-
-import java.lang.ref.Reference;
-import java.lang.ref.ReferenceQueue;
-import java.lang.ref.WeakReference;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.function.Consumer;
-
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.lang.IgniteLogger;
-
-/**
- */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class ClosableIteratorsHolder {
- /** */
- private final ReferenceQueue refQueue;
-
- /** */
- private final Map<Reference, Object> refMap;
-
- /** */
- private final IgniteLogger log;
-
- /** */
- private volatile boolean stopped;
-
- /** */
- private Thread cleanWorker;
-
- /** */
- public ClosableIteratorsHolder(IgniteLogger log) {
- this.log = log;
-
- refQueue = new ReferenceQueue<>();
- refMap = new ConcurrentHashMap<>();
- }
-
- /**
- * @param src Closeable iterator.
- * @return Weak closable iterator wrapper.
- */
- public <T> Iterator<T> iterator(final Iterator<T> src) {
- cleanUp(false);
-
- return new DelegatingIterator<>(src);
- }
-
- /** */
- public void init() {
- cleanWorker = new Thread(null, () -> cleanUp(true), "calciteIteratorsCleanWorker");
- cleanWorker.setDaemon(true);
- cleanWorker.start();
- }
-
- /** */
- public void tearDown() {
- stopped = true;
- refMap.clear();
-
- Thread t = cleanWorker;
-
- if (t != null)
- t.interrupt();
- }
-
- /** */
- private void cleanUp(boolean blocking) {
- for (Reference<?> ref = nextRef(blocking); !stopped && ref != null; ref = nextRef(blocking))
- Commons.close(refMap.remove(ref), log);
- }
-
- /** */
- private Reference nextRef(boolean blocking) {
- try {
- return !blocking ? refQueue.poll() : refQueue.remove();
- }
- catch (InterruptedException ignored) {
- return null;
- }
- }
-
- /** */
- private AutoCloseable closeable(Object referent, Object resource) {
- if (!(resource instanceof AutoCloseable))
- return null;
-
- return new CloseableReference(referent, resource);
- }
-
- /** */
- private final class DelegatingIterator<T> implements Iterator<T>, AutoCloseable {
- /** */
- private final Iterator<T> delegate;
-
- /** */
- private final AutoCloseable closeable;
-
- /** */
- private DelegatingIterator(Iterator<T> delegate) {
- closeable = closeable(this, this.delegate = delegate);
- }
-
- /** {@inheritDoc} */
- @Override public boolean hasNext() {
- return delegate.hasNext();
- }
-
- /** {@inheritDoc} */
- @Override public T next() {
- return delegate.next();
- }
-
- /** {@inheritDoc} */
- @Override public void remove() {
- delegate.remove();
- }
-
- /** {@inheritDoc} */
- @Override public void forEachRemaining(Consumer<? super T> action) {
- delegate.forEachRemaining(action);
- }
-
- /** {@inheritDoc} */
- @Override public void close() throws Exception {
- Commons.close(closeable);
- }
- }
-
- /** */
- private final class CloseableReference extends WeakReference implements AutoCloseable {
- /** */
- private CloseableReference(Object referent, Object resource) {
- super(referent, refQueue);
-
- refMap.put(this, resource);
- }
-
- /** {@inheritDoc} */
- @Override public void close() throws Exception {
- try {
- Commons.close(refMap.remove(this));
- }
- finally {
- clear();
- }
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
deleted file mode 100644
index 112c6c4..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
+++ /dev/null
@@ -1,84 +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.exec;
-
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-
-/**
- *
- */
-public interface ExchangeService {
- /**
- * Sends a batch of data to remote node.
- * @param nodeId Target node ID.
- * @param qryId Query ID.
- * @param fragmentId Target fragment ID.
- * @param exchangeId Exchange ID.
- * @param batchId Batch ID.
- * @param last Last batch flag.
- * @param rows Data rows.
- */
- <Row> void sendBatch(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId, boolean last,
- List<Row> rows) throws IgniteInternalCheckedException;
-
- /**
- * Acknowledges a batch with given ID is processed.
- * @param nodeId Node ID to notify.
- * @param qryId Query ID.
- * @param fragmentId Target fragment ID.
- * @param exchangeId Exchange ID.
- * @param batchId Batch ID.
- */
- void acknowledge(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId) throws IgniteInternalCheckedException;
-
- /**
- * Sends cancel request.
- * @param nodeId Target node ID.
- * @param qryId Query ID.
- * @param fragmentId Target fragment ID.
- * @param exchangeId Exchange ID.
- */
- void closeInbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException;
-
- /**
- * Sends cancel request.
- * @param nodeId Target node ID.
- * @param qryId Query ID.
- * @param fragmentId Target fragment ID.
- * @param exchangeId Exchange ID.
- */
- void closeOutbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException;
-
- /**
- * @param nodeId Target node ID.
- * @param qryId Query ID.
- * @param fragmentId Source fragment ID.
- * @param err Exception to send.
- * @throws IgniteInternalCheckedException On error marshaling or send ErrorMessage.
- */
- void sendError(String nodeId, UUID qryId, long fragmentId, Throwable err) throws IgniteInternalCheckedException;
-
- /**
- * @param nodeId Node ID.
- * @return {@code true} if node is alive, {@code false} otherwise.
- */
- boolean alive(String nodeId);
-}
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
deleted file mode 100644
index c3bc745..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
+++ /dev/null
@@ -1,262 +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.exec;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.UUID;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
-import org.apache.ignite.internal.processors.query.calcite.message.InboxCloseMessage;
-import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
-import org.apache.ignite.internal.processors.query.calcite.message.OutboxCloseMessage;
-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.message.SqlQueryMessageGroup;
-import org.apache.ignite.internal.processors.query.calcite.message.SqlQueryMessagesFactory;
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteLogger;
-
-import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
-
-/**
- *
- */
-public class ExchangeServiceImpl implements ExchangeService {
- private static final IgniteLogger LOG = IgniteLogger.forClass(ExchangeServiceImpl.class);
-
- private static final SqlQueryMessagesFactory FACTORY = new SqlQueryMessagesFactory();
-
- /** */
- private final QueryTaskExecutor taskExecutor;
-
- /** */
- private final MailboxRegistry mailboxRegistry;
-
- /** */
- private final MessageService msgSrvc;
-
- public ExchangeServiceImpl(
- QueryTaskExecutor taskExecutor,
- MailboxRegistry mailboxRegistry,
- MessageService msgSrvc
- ) {
- this.taskExecutor = taskExecutor;
- this.mailboxRegistry = mailboxRegistry;
- this.msgSrvc = msgSrvc;
-
- init();
- }
-
- /** {@inheritDoc} */
- @Override public <Row> void sendBatch(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId,
- boolean last, List<Row> rows) throws IgniteInternalCheckedException {
- msgSrvc.send(
- nodeId,
- FACTORY.queryBatchMessage()
- .queryId(qryId)
- .fragmentId(fragmentId)
- .exchangeId(exchangeId)
- .batchId(batchId)
- .last(last)
- .rows(Commons.cast(rows))
- .build()
- );
- }
-
- /** {@inheritDoc} */
- @Override public void acknowledge(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId)
- throws IgniteInternalCheckedException {
- msgSrvc.send(
- nodeId,
- FACTORY.queryBatchAcknowledgeMessage()
- .queryId(qryId)
- .fragmentId(fragmentId)
- .exchangeId(exchangeId)
- .batchId(batchId)
- .build()
- );
- }
-
- /** {@inheritDoc} */
- @Override public void closeOutbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException {
- msgSrvc.send(
- nodeId,
- FACTORY.outboxCloseMessage()
- .queryId(qryId)
- .fragmentId(fragmentId)
- .exchangeId(exchangeId)
- .build()
- );
- }
-
- /** {@inheritDoc} */
- @Override public void closeInbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException {
- msgSrvc.send(
- nodeId,
- FACTORY.inboxCloseMessage()
- .queryId(qryId)
- .fragmentId(fragmentId)
- .exchangeId(exchangeId)
- .build()
- );
- }
-
- /** {@inheritDoc} */
- @Override public void sendError(String nodeId, UUID qryId, long fragmentId, Throwable err) throws IgniteInternalCheckedException {
- msgSrvc.send(
- nodeId,
- FACTORY.errorMessage()
- .queryId(qryId)
- .fragmentId(fragmentId)
- .error(err)
- .build()
- );
- }
-
- private void init() {
- msgSrvc.register((n, m) -> onMessage(n, (InboxCloseMessage) m), SqlQueryMessageGroup.INBOX_CLOSE_MESSAGE);
- msgSrvc.register((n, m) -> onMessage(n, (OutboxCloseMessage) m), SqlQueryMessageGroup.OUTBOX_CLOSE_MESSAGE);
- msgSrvc.register((n, m) -> onMessage(n, (QueryBatchAcknowledgeMessage) m), SqlQueryMessageGroup.QUERY_BATCH_ACK);
- msgSrvc.register((n, m) -> onMessage(n, (QueryBatchMessage) m), SqlQueryMessageGroup.QUERY_BATCH_MESSAGE);
- }
-
- /** {@inheritDoc} */
- @Override public boolean alive(String nodeId) {
- return msgSrvc.alive(nodeId);
- }
-
- /** */
- protected void onMessage(String nodeId, InboxCloseMessage msg) {
- Collection<Inbox<?>> inboxes = mailboxRegistry.inboxes(msg.queryId(), msg.fragmentId(), msg.exchangeId());
-
- if (!nullOrEmpty(inboxes)) {
- for (Inbox<?> inbox : inboxes)
- inbox.context().execute(inbox::close, inbox::onError);
- }
- else if (LOG.isDebugEnabled()) {
- LOG.debug("Stale inbox cancel message received: [" +
- "nodeId=" + nodeId +
- ", queryId=" + msg.queryId() +
- ", fragmentId=" + msg.fragmentId() +
- ", exchangeId=" + msg.exchangeId() + "]");
- }
- }
-
- /** */
- protected void onMessage(String nodeId, OutboxCloseMessage msg) {
- Collection<Outbox<?>> outboxes = mailboxRegistry.outboxes(msg.queryId(), msg.fragmentId(), msg.exchangeId());
-
- if (!nullOrEmpty(outboxes)) {
- for (Outbox<?> outbox : outboxes)
- outbox.context().execute(outbox::close, outbox::onError);
-
- for (Outbox<?> outbox : outboxes)
- outbox.context().execute(outbox.context()::cancel, outbox::onError);
- }
- else if (LOG.isDebugEnabled()) {
- LOG.debug("Stale oubox cancel message received: [" +
- "nodeId=" + nodeId +
- ", queryId=" + msg.queryId() +
- ", fragmentId=" + msg.fragmentId() +
- ", exchangeId=" + msg.exchangeId() + "]");
- }
- }
-
- /** */
- protected void onMessage(String nodeId, QueryBatchAcknowledgeMessage msg) {
- Outbox<?> outbox = mailboxRegistry.outbox(msg.queryId(), msg.exchangeId());
-
- if (outbox != null) {
- try {
- outbox.onAcknowledge(nodeId, msg.batchId());
- }
- catch (Throwable e) {
- outbox.onError(e);
-
- throw new IgniteInternalException("Unexpected exception", e);
- }
- }
- 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(String 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<?> newInbox = new Inbox<>(baseInboxContext(nodeId, msg.queryId(), msg.fragmentId()),
- this, mailboxRegistry, msg.exchangeId(), msg.exchangeId());
-
- inbox = mailboxRegistry.register(newInbox);
- }
-
- if (inbox != null) {
- try {
- inbox.onBatchReceived(nodeId, msg.batchId(), msg.last(), Commons.cast(msg.rows()));
- }
- catch (Throwable e) {
- inbox.onError(e);
-
- throw new IgniteInternalException("Unexpected exception", e);
- }
- }
- 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(String nodeId, UUID qryId, long fragmentId) {
- return new ExecutionContext<>(
- taskExecutor,
- PlanningContext.builder()
- .originatingNodeId(nodeId)
- .build(),
- qryId,
- new FragmentDescription(
- fragmentId,
- null,
- null,
- null),
- 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
index 12570ca..96ad599 100644
--- 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
@@ -134,7 +134,7 @@ public class ExecutionContext<Row> implements DataContext {
* @param exchangeId ExchangeId to find remote nodes for.
* @return Remote nodes for given exchangeId.
*/
- public List<String> remotes(long exchangeId) {
+ public List<UUID> remotes(long exchangeId) {
return fragmentDesc.remotes().get(exchangeId);
}
@@ -170,7 +170,7 @@ public class ExecutionContext<Row> implements DataContext {
/**
* @return Originating node ID.
*/
- public String originatingNodeId() {
+ public UUID originatingNodeId() {
return planningContext().originatingNodeId();
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
deleted file mode 100644
index d98fa19..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
+++ /dev/null
@@ -1,45 +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.exec;
-
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.ignite.internal.util.Cursor;
-
-/**
- *
- */
-public interface ExecutionService {
- /**
- * Executes a query.
- *
- * @param schema Schema name.
- * @param query Query.
- * @param params Query parameters.
- * @return Query cursor.
- */
- List<Cursor<List<?>>> executeQuery(String schema, String query, Object[] params);
-
- /**
- * Cancels a running query.
- *
- * @param queryId Query ID.
- */
- 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
deleted file mode 100644
index 15e0b1c..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ /dev/null
@@ -1,865 +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.exec;
-
-import java.util.ArrayList;
-import java.util.Collection;
-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 com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlDdl;
-import org.apache.calcite.sql.SqlExplain;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Node;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
-import org.apache.ignite.internal.processors.query.calcite.message.ErrorMessage;
-import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
-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.message.SqlQueryMessageGroup;
-import org.apache.ignite.internal.processors.query.calcite.message.SqlQueryMessagesFactory;
-import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
-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.RemoteException;
-import org.apache.ignite.internal.processors.query.calcite.prepare.CacheKey;
-import org.apache.ignite.internal.processors.query.calcite.prepare.DdlPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.ExplainPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadata;
-import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadataImpl;
-import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
-import org.apache.ignite.internal.processors.query.calcite.prepare.FragmentPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
-import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepDmlPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepQueryPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlan;
-import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCache;
-import org.apache.ignite.internal.processors.query.calcite.prepare.QueryTemplate;
-import org.apache.ignite.internal.processors.query.calcite.prepare.Splitter;
-import org.apache.ignite.internal.processors.query.calcite.prepare.ValidationResult;
-import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlSqlToCommandConverter;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
-import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTraitDef;
-import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTraitDef;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.processors.query.calcite.util.NodeLeaveHandler;
-import org.apache.ignite.internal.processors.query.calcite.util.TransformingIterator;
-import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
-import org.apache.ignite.internal.util.Cancellable;
-import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteLogger;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.TopologyService;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-import static java.util.Collections.singletonList;
-import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
-import static org.apache.ignite.internal.processors.query.calcite.exec.PlannerHelper.optimize;
-import static org.apache.ignite.internal.processors.query.calcite.externalize.RelJsonReader.fromJson;
-import static org.apache.ignite.internal.processors.query.calcite.util.Commons.FRAMEWORK_CONFIG;
-import static org.apache.ignite.internal.util.CollectionUtils.first;
-import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
-
-/**
- *
- */
-public class ExecutionServiceImpl<Row> implements ExecutionService {
- private static final IgniteLogger LOG = IgniteLogger.forClass(ExecutionServiceImpl.class);
-
- private static final SqlQueryMessagesFactory FACTORY = new SqlQueryMessagesFactory();
-
- /** */
- private final MessageService msgSrvc;
-
- /** */
- private final String locNodeId;
-
- /** */
- private final QueryPlanCache qryPlanCache;
-
- /** */
- private final SchemaHolder schemaHolder;
-
- /** */
- private final QueryTaskExecutor taskExecutor;
-
- /** */
- private final AffinityService affSrvc;
-
- /** */
- private final MailboxRegistry mailboxRegistry;
-
- /** */
- private final MappingService mappingSrvc;
-
- /** */
- private final ExchangeService exchangeSrvc;
-
- /** */
- private final ClosableIteratorsHolder iteratorsHolder;
-
- /** */
- private final Map<UUID, QueryInfo> running;
-
- /** */
- private final RowHandler<Row> handler;
-
- /** */
- private final DdlSqlToCommandConverter ddlConverter;
-
- public ExecutionServiceImpl(
- TopologyService topSrvc,
- MessageService msgSrvc,
- QueryPlanCache planCache,
- SchemaHolder schemaHolder,
- QueryTaskExecutor taskExecutor,
- RowHandler<Row> handler
- ) {
- this.handler = handler;
- this.msgSrvc = msgSrvc;
- this.schemaHolder = schemaHolder;
- this.taskExecutor = taskExecutor;
-
- locNodeId = topSrvc.localMember().id();
- qryPlanCache = planCache;
- running = new ConcurrentHashMap<>();
- ddlConverter = new DdlSqlToCommandConverter();
- iteratorsHolder = new ClosableIteratorsHolder(LOG);
- mailboxRegistry = new MailboxRegistryImpl(topSrvc);
- exchangeSrvc = new ExchangeServiceImpl(taskExecutor, mailboxRegistry, msgSrvc);
- mappingSrvc = new MappingServiceImpl(topSrvc);
- // TODO: fix this
- affSrvc = cacheId -> Objects::hashCode;
-
- topSrvc.addEventHandler(new NodeLeaveHandler(this::onNodeLeft));
-
- init();
- }
-
- private void init() {
- msgSrvc.register((n, m) -> onMessage(n, (QueryStartRequest) m), SqlQueryMessageGroup.QUERY_START_REQUEST);
- msgSrvc.register((n, m) -> onMessage(n, (QueryStartResponse) m), SqlQueryMessageGroup.QUERY_START_RESPONSE);
- msgSrvc.register((n, m) -> onMessage(n, (ErrorMessage) m), SqlQueryMessageGroup.ERROR_MESSAGE);
-
- iteratorsHolder.init();
- }
-
- /** {@inheritDoc} */
- @Override public List<Cursor<List<?>>> executeQuery(
- String schema,
- String qry,
- Object[] params
- ) {
- PlanningContext pctx = createContext(topologyVersion(), locNodeId, schema, qry, params);
-
- List<QueryPlan> qryPlans = qryPlanCache.queryPlan(pctx, new CacheKey(pctx.schemaName(), pctx.query()), this::prepareQuery);
-
- return executePlans(qryPlans, pctx);
- }
-
- /**
- * Executes prepared plans.
- * @param qryPlans Query plans.
- * @param pctx Query context.
- * @return List of query result cursors.
- */
- @NotNull public List<Cursor<List<?>>> executePlans(
- Collection<QueryPlan> qryPlans,
- PlanningContext pctx
- ) {
- List<Cursor<List<?>>> cursors = new ArrayList<>(qryPlans.size());
-
- for (QueryPlan plan : qryPlans) {
- UUID qryId = UUID.randomUUID();
-
- Cursor<List<?>> cur = executePlan(qryId, pctx, plan);
-
- cursors.add(cur);
- }
-
- return cursors;
- }
-
- /** {@inheritDoc} */
- @Override public void cancelQuery(UUID qryId) {
- QueryInfo info = running.get(qryId);
-
- if (info != null)
- info.cancel();
- }
-
- /** */
- protected long topologyVersion() {
- return 1L;
- }
-
- /** */
- private PlanningContext createContext(long topVer, String originator,
- @Nullable String schema, String qry, Object[] params) {
- RelTraitDef<?>[] traitDefs = {
- ConventionTraitDef.INSTANCE,
- RelCollationTraitDef.INSTANCE,
- DistributionTraitDef.INSTANCE,
- RewindabilityTraitDef.INSTANCE,
- CorrelationTraitDef.INSTANCE,
- };
-
- return PlanningContext.builder()
- .localNodeId(locNodeId)
- .originatingNodeId(originator)
- .parentContext(Contexts.empty())
- .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
- .defaultSchema(schema != null
- ? schemaHolder.schema().getSubSchema(schema)
- : schemaHolder.schema())
- .traitDefs(traitDefs)
- .build())
- .query(qry)
- .parameters(params)
- .topologyVersion(topVer)
- .build();
- }
-
- /** */
- private List<QueryPlan> prepareQuery(PlanningContext ctx) {
- try {
- String qry = ctx.query();
-
- assert qry != null;
-
- // Parse query.
- SqlNode sqlNode = ctx.planner().parse(qry);
-
- if (single(sqlNode))
- return singletonList(prepareSingle(sqlNode, ctx));
-
- List<SqlNode> nodes = ((SqlNodeList) sqlNode).getList();
- List<QueryPlan> res = new ArrayList<>(nodes.size());
-
- for (SqlNode node : nodes)
- res.add(prepareSingle(node, ctx));
-
- return res;
- }
- catch (SqlParseException e) {
- throw new IgniteInternalException("Failed to parse query", e);
- }
- catch (ValidationException e) {
- throw new IgniteInternalException("Failed to validate query", e);
- }
- catch (Exception e) {
- throw new IgniteInternalException("Failed to plan query.", e);
- }
- }
-
- /** */
- private List<QueryPlan> prepareFragment(PlanningContext ctx) {
- return ImmutableList.of(new FragmentPlan(fromJson(ctx, ctx.query())));
- }
-
- /** */
- private QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) throws ValidationException {
- assert single(sqlNode);
-
- ctx.planner().reset();
-
- switch (sqlNode.getKind()) {
- case SELECT:
- case ORDER_BY:
- case WITH:
- case VALUES:
- case UNION:
- case EXCEPT:
- case INTERSECT:
- return prepareQuery(sqlNode, ctx);
-
- case INSERT:
- case DELETE:
- case UPDATE:
- return prepareDml(sqlNode, ctx);
-
- case EXPLAIN:
- return prepareExplain(sqlNode, ctx);
-
- case CREATE_TABLE:
- case DROP_TABLE:
- return prepareDdl(sqlNode, ctx);
-
- default:
- throw new IgniteInternalException("Unsupported operation [" +
- "sqlNodeKind=" + sqlNode.getKind() + "; " +
- "querySql=\"" + ctx.query() + "\"]");
- }
- }
-
- /** */
- private QueryPlan prepareQuery(SqlNode sqlNode, PlanningContext ctx) {
- IgnitePlanner planner = ctx.planner();
-
- // Validate
- ValidationResult validated = planner.validateAndGetTypeMetadata(sqlNode);
-
- sqlNode = validated.sqlNode();
-
- IgniteRel igniteRel = optimize(sqlNode, planner, LOG);
-
- // Split query plan to query fragments.
- List<Fragment> fragments = new Splitter().go(igniteRel);
-
- QueryTemplate template = new QueryTemplate(mappingSrvc, fragments);
-
- return new MultiStepQueryPlan(template, queryFieldsMetadata(ctx, validated.dataType(), validated.origins()));
- }
-
- /** */
- private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws ValidationException {
- IgnitePlanner planner = ctx.planner();
-
- // Validate
- sqlNode = planner.validate(sqlNode);
-
- // Convert to Relational operators graph
- IgniteRel igniteRel = optimize(sqlNode, planner, LOG);
-
- // Split query plan to query fragments.
- List<Fragment> fragments = new Splitter().go(igniteRel);
-
- QueryTemplate template = new QueryTemplate(mappingSrvc, fragments);
-
- return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
- }
-
- /** */
- private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
- assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
-
- SqlDdl ddlNode = (SqlDdl)sqlNode;
-
- return new DdlPlan(ddlConverter.convert(ddlNode, ctx));
- }
-
- /** */
- private QueryPlan prepareExplain(SqlNode explain, PlanningContext ctx) throws ValidationException {
- IgnitePlanner planner = ctx.planner();
-
- SqlNode sql = ((SqlExplain)explain).getExplicandum();
-
- // Validate
- sql = planner.validate(sql);
-
- // Convert to Relational operators graph
- IgniteRel igniteRel = optimize(sql, planner, LOG);
-
- String plan = RelOptUtil.toString(igniteRel, SqlExplainLevel.ALL_ATTRIBUTES);
-
- return new ExplainPlan(plan, explainFieldsMetadata(ctx));
- }
-
- /** */
- private FieldsMetadata explainFieldsMetadata(PlanningContext ctx) {
- IgniteTypeFactory factory = ctx.typeFactory();
- RelDataType planStrDataType =
- factory.createSqlType(SqlTypeName.VARCHAR, PRECISION_NOT_SPECIFIED);
- Map.Entry<String, RelDataType> planField = new IgniteBiTuple<>(ExplainPlan.PLAN_COL_NAME, planStrDataType);
- RelDataType planDataType = factory.createStructType(singletonList(planField));
-
- return queryFieldsMetadata(ctx, planDataType, null);
- }
-
- /** */
- private Cursor<List<?>> executePlan(UUID qryId, PlanningContext pctx, QueryPlan plan) {
- switch (plan.type()) {
- case DML:
- // TODO a barrier between previous operation and this one
- case QUERY:
- return executeQuery(qryId, (MultiStepPlan) plan, pctx);
- case EXPLAIN:
- return executeExplain((ExplainPlan)plan);
- case DDL:
- return executeDdl((DdlPlan)plan, pctx);
-
- default:
- throw new AssertionError("Unexpected plan type: " + plan);
- }
- }
-
- /** */
- private Cursor<List<?>> executeDdl(DdlPlan plan, PlanningContext pctx) {
- throw new UnsupportedOperationException("plan=" + plan + ", ctx=" + pctx);
- }
-
- /** */
- private Cursor<List<?>> executeQuery(UUID qryId, MultiStepPlan plan, PlanningContext pctx) {
- plan.init(pctx);
-
- List<Fragment> fragments = plan.fragments();
-
- // Local execution
- Fragment fragment = first(fragments);
-
- if (IgniteUtils.assertionsEnabled()) {
- assert fragment != null;
-
- FragmentMapping mapping = plan.mapping(fragment);
-
- assert mapping != null;
-
- List<String> nodes = mapping.nodeIds();
-
- assert nodes != null && nodes.size() == 1 && first(nodes).equals(pctx.localNodeId());
- }
-
- FragmentDescription fragmentDesc = new FragmentDescription(
- fragment.fragmentId(),
- plan.mapping(fragment),
- plan.target(fragment),
- plan.remotes(fragment));
-
- ExecutionContext<Row> ectx = new ExecutionContext<>(
- taskExecutor,
- pctx,
- qryId,
- fragmentDesc,
- handler,
- Commons.parametersMap(pctx.parameters()));
-
- Node<Row> node = new LogicalRelImplementor<>(ectx, affSrvc, mailboxRegistry,
- exchangeSrvc).go(fragment.root());
-
- QueryInfo info = new QueryInfo(ectx, plan, node);
-
- // register query
- register(info);
-
- // start remote execution
- for (int i = 1; i < fragments.size(); i++) {
- fragment = fragments.get(i);
- fragmentDesc = new FragmentDescription(
- fragment.fragmentId(),
- plan.mapping(fragment),
- plan.target(fragment),
- plan.remotes(fragment));
-
- Throwable ex = null;
- for (String nodeId : fragmentDesc.nodeIds()) {
- if (ex != null)
- info.onResponse(nodeId, fragment.fragmentId(), ex);
- else {
- try {
- QueryStartRequest req = FACTORY.queryStartRequest()
- .queryId(qryId)
- .fragmentId(fragment.fragmentId())
- .schema(pctx.schemaName())
- .root(fragment.serialized())
- .topologyVersion(pctx.topologyVersion())
- .fragmentDescription(fragmentDesc)
- .parameters(pctx.parameters())
- .build();
-
- msgSrvc.send(nodeId, req);
- }
- catch (Throwable e) {
- info.onResponse(nodeId, fragment.fragmentId(), ex = e);
- }
- }
- }
- }
-
- return Commons.createCursor(new TransformingIterator<>(info.iterator(), row -> {
- int rowSize = ectx.rowHandler().columnCount(row);
-
- List<Object> res = new ArrayList<>(rowSize);
-
- for (int i = 0; i < rowSize; i++)
- res.add(ectx.rowHandler().get(i, row));
-
- return res;
- }));
- }
-
- /** */
- private Cursor<List<?>> executeExplain(ExplainPlan plan) {
- Cursor<List<?>> cur = Commons.createCursor(singletonList(singletonList(plan.plan())));
- // TODO: fix this
-// cur.fieldsMeta(plan.fieldsMeta().queryFieldsMetadata(pctx.typeFactory()));
-
- return cur;
- }
-
- /** */
- private void executeFragment(UUID qryId, FragmentPlan plan, PlanningContext pctx, FragmentDescription fragmentDesc) {
- ExecutionContext<Row> ectx = new ExecutionContext<>(taskExecutor, pctx, qryId,
- fragmentDesc, handler, Commons.parametersMap(pctx.parameters()));
-
- long frId = fragmentDesc.fragmentId();
- String origNodeId = pctx.originatingNodeId();
-
- Outbox<Row> node = new LogicalRelImplementor<>(
- ectx,
- affSrvc,
- mailboxRegistry,
- exchangeSrvc
- ).go(plan.root());
-
- try {
- msgSrvc.send(
- origNodeId,
- FACTORY.queryStartResponse()
- .queryId(qryId)
- .fragmentId(frId)
- .build()
- );
- }
- catch (IgniteInternalCheckedException e) {
- IgniteInternalException wrpEx = new IgniteInternalException("Failed to send reply. [nodeId=" + origNodeId + ']', e);
-
- throw wrpEx;
- }
-
- node.init();
- }
-
- /** */
- private void register(QueryInfo info) {
- UUID qryId = info.ctx.queryId();
-
- running.put(qryId, info);
- }
-
- /** */
- private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlType,
- @Nullable List<List<String>> origins) {
- RelDataType resultType = TypeUtils.getResultType(
- ctx.typeFactory(), ctx.catalogReader(), sqlType, origins);
- return new FieldsMetadataImpl(resultType, origins);
- }
-
- /** */
- private boolean single(SqlNode sqlNode) {
- return !(sqlNode instanceof SqlNodeList);
- }
-
- /** */
- private void onMessage(String nodeId, QueryStartRequest msg) {
- assert nodeId != null && msg != null;
-
- try {
- PlanningContext pctx = createContext(msg.topologyVersion(), nodeId, msg.schema(),
- msg.root(), msg.parameters());
-
- List<QueryPlan> qryPlans = qryPlanCache.queryPlan(
- pctx,
- new CacheKey(pctx.schemaName(), pctx.query()),
- this::prepareFragment
- );
-
- assert qryPlans.size() == 1 && qryPlans.get(0).type() == QueryPlan.Type.FRAGMENT;
-
- FragmentPlan plan = (FragmentPlan)qryPlans.get(0);
-
- executeFragment(msg.queryId(), plan, pctx, msg.fragmentDescription());
- }
- catch (Throwable ex) {
- LOG.error("Failed to start query fragment", ex);
-
- mailboxRegistry.outboxes(msg.queryId(), msg.fragmentId(), -1)
- .forEach(Outbox::close);
- mailboxRegistry.inboxes(msg.queryId(), msg.fragmentId(), -1)
- .forEach(Inbox::close);
-
- try {
- msgSrvc.send(
- nodeId,
- FACTORY.queryStartResponse()
- .queryId(msg.queryId())
- .fragmentId(msg.fragmentId())
- .error(ex)
- .build()
- );
- }
- catch (Exception e) {
- LOG.error("Error occurred during send error message", e);
-
- IgniteInternalException wrpEx = new IgniteInternalException("Error occurred during send error message", e);
-
- e.addSuppressed(ex);
-
- throw wrpEx;
- }
-
- throw ex;
- }
- }
-
- /** */
- private void onMessage(String 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 onMessage(String nodeId, ErrorMessage msg) {
- assert nodeId != null && msg != null;
-
- QueryInfo info = running.get(msg.queryId());
-
- if (info != null)
- info.onError(new RemoteException(nodeId, msg.queryId(), msg.fragmentId(), msg.error()));
- }
-
- /** */
- private void onNodeLeft(ClusterNode node) {
- running.forEach((uuid, queryInfo) -> queryInfo.onNodeLeft(node.id()));
- }
-
- /** */
- private enum QueryState {
- /** */
- RUNNING,
-
- /** */
- CLOSING,
-
- /** */
- CLOSED
- }
-
- /** */
- private static final class RemoteFragmentKey {
- /** */
- private final String nodeId;
-
- /** */
- private final long fragmentId;
-
- /** */
- private RemoteFragmentKey(String 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 res = nodeId.hashCode();
- res = 31 * res + (int) (fragmentId ^ (fragmentId >>> 32));
- return res;
- }
- }
-
- /** */
- private final class QueryInfo implements Cancellable {
- /** */
- private final ExecutionContext<Row> ctx;
-
- /** */
- private final RootNode<Row> root;
-
- /** remote nodes */
- private final Set<String> remotes;
-
- /** node to fragment */
- private final Set<RemoteFragmentKey> waiting;
-
- /** */
- private volatile QueryState state;
-
- /** */
- private QueryInfo(ExecutionContext<Row> ctx, MultiStepPlan plan, Node<Row> root) {
- this.ctx = ctx;
-
- RootNode<Row> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
- rootNode.register(root);
-
- this.root = rootNode;
-
- remotes = new HashSet<>();
- waiting = new HashSet<>();
-
- for (int i = 1; i < plan.fragments().size(); i++) {
- Fragment fragment = plan.fragments().get(i);
- List<String> nodes = plan.mapping(fragment).nodeIds();
-
- remotes.addAll(nodes);
-
- for (String node : nodes)
- waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
- }
-
- state = QueryState.RUNNING;
- }
-
- /** */
- public Iterator<Row> iterator() {
- return iteratorsHolder.iterator(root);
- }
-
- /** {@inheritDoc} */
- @Override public void cancel() {
- root.close();
- }
-
- /**
- * Can be called multiple times after receive each error at {@link #onResponse(RemoteFragmentKey, Throwable)}.
- */
- private void tryClose() {
- QueryState state0 = null;
-
- synchronized (this) {
- if (state == QueryState.CLOSED)
- return;
-
- if (state == QueryState.RUNNING)
- state0 = state = QueryState.CLOSING;
-
- // 1) close local fragment
- root.closeInternal();
-
- if (state == QueryState.CLOSING && waiting.isEmpty())
- state0 = state = QueryState.CLOSED;
- }
-
- if (state0 == QueryState.CLOSED) {
- // 2) unregister runing query
- running.remove(ctx.queryId());
-
- IgniteInternalException wrpEx = null;
-
- // 3) close remote fragments
- for (String nodeId : remotes) {
- try {
- exchangeSrvc.closeOutbox(nodeId, ctx.queryId(), -1, -1);
- }
- catch (IgniteInternalCheckedException e) {
- if (wrpEx == null)
- wrpEx = new IgniteInternalException("Failed to send cancel message. [nodeId=" + nodeId + ']', e);
- else
- wrpEx.addSuppressed(e);
- }
- }
-
- // 4) Cancel local fragment
- root.context().execute(ctx::cancel, root::onError);
-
- if (wrpEx != null)
- throw wrpEx;
- }
- }
-
- /** */
- private void onNodeLeft(String 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 (!nullOrEmpty(fragments)) {
- IgniteInternalCheckedException ex = new IgniteInternalCheckedException(
- "Failed to start query, node left. nodeId=" + nodeId);
-
- for (RemoteFragmentKey fragment : fragments)
- onResponse(fragment, ex);
- }
- }
-
- /** */
- private void onResponse(String nodeId, long fragmentId, Throwable error) {
- onResponse(new RemoteFragmentKey(nodeId, fragmentId), error);
- }
-
- /** */
- private void onResponse(RemoteFragmentKey fragment, Throwable error) {
- QueryState state;
- synchronized (this) {
- waiting.remove(fragment);
- state = this.state;
- }
-
- if (error != null)
- onError(error);
- else if (state == QueryState.CLOSING)
- tryClose();
- }
-
- /** */
- private void onError(Throwable error) {
- root.onError(error);
-
- tryClose();
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
deleted file mode 100644
index eed5440..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
+++ /dev/null
@@ -1,654 +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.exec;
-
-import java.util.Comparator;
-import java.util.List;
-import java.util.Objects;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.function.Supplier;
-
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.core.Spool;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractSetOpNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.CorrelatedNestedLoopJoinNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.FilterNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.IndexSpoolNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.LimitNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.MergeJoinNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.MinusNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.NestedLoopJoinNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Node;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.ProjectNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.ScanNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.SortAggregateNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.SortNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.TableSpoolNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.UnionAllNode;
-import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
-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.IgniteHashIndexSpool;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteLimit;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMergeJoin;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSortedIndexSpool;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableFunctionScan;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableSpool;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTrimExchange;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteUnionAll;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
-import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteMapHashAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteMapSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceHashAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
-import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
-import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
-import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.combinedRowType;
-import static org.apache.ignite.internal.util.ArrayUtils.asList;
-import static org.apache.ignite.internal.util.CollectionUtils.first;
-
-/**
- * Implements a query plan.
- */
-@SuppressWarnings("TypeMayBeWeakened")
-public class LogicalRelImplementor<Row> implements IgniteRelVisitor<Node<Row>> {
- /** */
- public static final String CNLJ_NOT_SUPPORTED_JOIN_ASSERTION_MSG = "only INNER and LEFT join supported by IgniteCorrelatedNestedLoop";
-
- /** */
- private final ExecutionContext<Row> ctx;
-
- /** */
- private final AffinityService affSrvc;
-
- /** */
- private final ExchangeService exchangeSvc;
-
- /** */
- private final MailboxRegistry mailboxRegistry;
-
- /** */
- private final ExpressionFactory<Row> expressionFactory;
-
- /**
- * @param ctx Root context.
- * @param affSrvc Affinity service.
- * @param mailboxRegistry Mailbox registry.
- * @param exchangeSvc Exchange service.
- */
- public LogicalRelImplementor(
- ExecutionContext<Row> ctx,
- AffinityService affSrvc,
- MailboxRegistry mailboxRegistry,
- ExchangeService exchangeSvc
- ) {
- this.affSrvc = affSrvc;
- this.mailboxRegistry = mailboxRegistry;
- this.exchangeSvc = exchangeSvc;
- this.ctx = ctx;
-
- expressionFactory = ctx.expressionFactory();
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteSender rel) {
- IgniteDistribution distribution = rel.distribution();
-
- Destination<Row> dest = distribution.destination(ctx, affSrvc, ctx.target());
-
- // Outbox fragment ID is used as exchange ID as well.
- Outbox<Row> outbox =
- new Outbox<>(ctx, rel.getRowType(), exchangeSvc, mailboxRegistry, rel.exchangeId(), rel.targetFragmentId(), dest);
-
- Node<Row> input = visit(rel.getInput());
-
- outbox.register(input);
-
- mailboxRegistry.register(outbox);
-
- return outbox;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteFilter rel) {
- Predicate<Row> pred = expressionFactory.predicate(rel.getCondition(), rel.getRowType());
-
- FilterNode<Row> node = new FilterNode<>(ctx, rel.getRowType(), pred);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteTrimExchange rel) {
- assert TraitUtils.distribution(rel).getType() == HASH_DISTRIBUTED;
-
- IgniteDistribution distr = rel.distribution();
- Destination<Row> dest = distr.destination(ctx, affSrvc, ctx.group(rel.sourceId()));
- String localNodeId = ctx.planningContext().localNodeId();
-
- FilterNode<Row> node = new FilterNode<>(ctx, rel.getRowType(), r -> Objects.equals(localNodeId, first(dest.targets(r))));
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteProject rel) {
- Function<Row, Row> prj = expressionFactory.project(rel.getProjects(), rel.getInput().getRowType());
-
- ProjectNode<Row> node = new ProjectNode<>(ctx, rel.getRowType(), prj);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteNestedLoopJoin rel) {
- RelDataType outType = rel.getRowType();
- RelDataType leftType = rel.getLeft().getRowType();
- RelDataType rightType = rel.getRight().getRowType();
- JoinRelType joinType = rel.getJoinType();
-
- RelDataType rowType = combinedRowType(ctx.getTypeFactory(), leftType, rightType);
- Predicate<Row> cond = expressionFactory.predicate(rel.getCondition(), rowType);
-
- Node<Row> node = NestedLoopJoinNode.create(ctx, outType, leftType, rightType, joinType, cond);
-
- Node<Row> leftInput = visit(rel.getLeft());
- Node<Row> rightInput = visit(rel.getRight());
-
- node.register(asList(leftInput, rightInput));
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteCorrelatedNestedLoopJoin rel) {
- RelDataType outType = rel.getRowType();
- RelDataType leftType = rel.getLeft().getRowType();
- RelDataType rightType = rel.getRight().getRowType();
-
- RelDataType rowType = combinedRowType(ctx.getTypeFactory(), leftType, rightType);
- Predicate<Row> cond = expressionFactory.predicate(rel.getCondition(), rowType);
-
- assert rel.getJoinType() == JoinRelType.INNER || rel.getJoinType() == JoinRelType.LEFT
- : CNLJ_NOT_SUPPORTED_JOIN_ASSERTION_MSG;
-
- Node<Row> node = new CorrelatedNestedLoopJoinNode<>(ctx, outType, cond, rel.getVariablesSet(),
- rel.getJoinType());
-
- Node<Row> leftInput = visit(rel.getLeft());
- Node<Row> rightInput = visit(rel.getRight());
-
- node.register(asList(leftInput, rightInput));
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteMergeJoin rel) {
- RelDataType outType = rel.getRowType();
- RelDataType leftType = rel.getLeft().getRowType();
- RelDataType rightType = rel.getRight().getRowType();
- JoinRelType joinType = rel.getJoinType();
-
- int pairsCnt = rel.analyzeCondition().pairs().size();
-
- Comparator<Row> comp = expressionFactory.comparator(
- rel.leftCollation().getFieldCollations().subList(0, pairsCnt),
- rel.rightCollation().getFieldCollations().subList(0, pairsCnt)
- );
-
- Node<Row> node = MergeJoinNode.create(ctx, outType, leftType, rightType, joinType, comp);
-
- Node<Row> leftInput = visit(rel.getLeft());
- Node<Row> rightInput = visit(rel.getRight());
-
- node.register(asList(leftInput, rightInput));
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteIndexScan rel) {
- // TODO: fix this
-// RexNode condition = rel.condition();
-// List<RexNode> projects = rel.projects();
-//
- IgniteTable tbl = rel.getTable().unwrap(IgniteTable.class);
- IgniteTypeFactory typeFactory = ctx.getTypeFactory();
-
- ImmutableBitSet requiredColumns = rel.requiredColumns();
-// List<RexNode> lowerCond = rel.lowerBound();
-// List<RexNode> upperCond = rel.upperBound();
-//
- RelDataType rowType = tbl.getRowType(typeFactory, requiredColumns);
-
-// Predicate<Row> filters = condition == null ? null : expressionFactory.predicate(condition, rowType);
-// Supplier<Row> lower = lowerCond == null ? null : expressionFactory.rowSource(lowerCond);
-// Supplier<Row> upper = upperCond == null ? null : expressionFactory.rowSource(upperCond);
-// Function<Row, Row> prj = projects == null ? null : expressionFactory.project(projects, rowType);
-//
-// IgniteIndex idx = tbl.getIndex(rel.indexName());
-//
-// ColocationGroup group = ctx.group(rel.sourceId());
-
- Iterable<Row> rowsIter = (Iterable<Row>) List.of(new Object[]{0, 0}, new Object[]{1, 1});//idx.scan(ctx, group, filters, lower, upper, prj, requiredColumns);
-
- return new ScanNode<>(ctx, rowType, rowsIter);
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteTableScan rel) {
- RexNode condition = rel.condition();
- List<RexNode> projects = rel.projects();
- ImmutableBitSet requiredColunms = rel.requiredColumns();
-
- IgniteTable tbl = rel.getTable().unwrap(IgniteTable.class);
- IgniteTypeFactory typeFactory = ctx.getTypeFactory();
-
- RelDataType rowType = tbl.getRowType(typeFactory, requiredColunms);
-
- Predicate<Row> filters = condition == null ? null : expressionFactory.predicate(condition, rowType);
- Function<Row, Row> prj = projects == null ? null : expressionFactory.project(projects, rowType);
-
- ColocationGroup group = ctx.group(rel.sourceId());
-
- Iterable<Row> rowsIter = tbl.scan(ctx, group, filters, prj, requiredColunms);
-
- return new ScanNode<>(ctx, rowType, rowsIter);
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteValues rel) {
- List<RexLiteral> vals = Commons.flat(Commons.cast(rel.getTuples()));
-
- RelDataType rowType = rel.getRowType();
-
- return new ScanNode<>(ctx, rowType, expressionFactory.values(vals, rowType));
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteUnionAll rel) {
- UnionAllNode<Row> node = new UnionAllNode<>(ctx, rel.getRowType());
-
- List<Node<Row>> inputs = Commons.transform(rel.getInputs(), this::visit);
-
- node.register(inputs);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteLimit rel) {
- Supplier<Integer> offset = (rel.offset() == null) ? null : expressionFactory.execute(rel.offset());
- Supplier<Integer> fetch = (rel.fetch() == null) ? null : expressionFactory.execute(rel.fetch());
-
- LimitNode<Row> node = new LimitNode<>(ctx, rel.getRowType(), offset, fetch);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteSort rel) {
- RelCollation collation = rel.getCollation();
-
- SortNode<Row> node = new SortNode<>(ctx, rel.getRowType(), expressionFactory.comparator(collation));
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteTableSpool rel) {
- TableSpoolNode<Row> node = new TableSpoolNode<>(ctx, rel.getRowType(), rel.readType == Spool.Type.LAZY);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteSortedIndexSpool rel) {
- RelCollation collation = rel.collation();
-
- assert rel.indexCondition() != null : rel;
-
- List<RexNode> lowerBound = rel.indexCondition().lowerBound();
- List<RexNode> upperBound = rel.indexCondition().upperBound();
-
- Predicate<Row> filter = expressionFactory.predicate(rel.condition(), rel.getRowType());
- Supplier<Row> lower = lowerBound == null ? null : expressionFactory.rowSource(lowerBound);
- Supplier<Row> upper = upperBound == null ? null : expressionFactory.rowSource(upperBound);
-
- IndexSpoolNode<Row> node = IndexSpoolNode.createTreeSpool(
- ctx,
- rel.getRowType(),
- collation,
- expressionFactory.comparator(collation),
- filter,
- lower,
- upper
- );
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteHashIndexSpool rel) {
- Supplier<Row> searchRow = expressionFactory.rowSource(rel.searchRow());
-
- IndexSpoolNode<Row> node = IndexSpoolNode.createHashSpool(
- ctx,
- rel.getRowType(),
- ImmutableBitSet.of(rel.keys()),
- searchRow
- );
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteSetOp rel) {
- RelDataType rowType = rel.getRowType();
-
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- List<Node<Row>> inputs = Commons.transform(rel.getInputs(), this::visit);
-
- AbstractSetOpNode<Row> node;
-
- if (rel instanceof Minus)
- node = new MinusNode<>(ctx, rowType, rel.aggregateType(), rel.all(), rowFactory);
- else if (rel instanceof Intersect)
- node = new IntersectNode<>(ctx, rowType, rel.aggregateType(), rel.all(), rowFactory, rel.getInputs().size());
- else
- throw new AssertionError();
-
- node.register(inputs);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteTableFunctionScan rel) {
- Supplier<Iterable<Object[]>> dataSupplier = expressionFactory.execute(rel.getCall());
-
- RelDataType rowType = rel.getRowType();
-
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- return new ScanNode<>(ctx, rowType, new TableFunctionScan<>(dataSupplier, rowFactory));
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteTableModify rel) {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteReceiver rel) {
- Inbox<Row> inbox = mailboxRegistry.register(
- new Inbox<>(ctx, exchangeSvc, mailboxRegistry, rel.exchangeId(), rel.sourceFragmentId()));
-
- // 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, rel.getRowType(), ctx.remotes(rel.exchangeId()), expressionFactory.comparator(rel.collation()));
-
- return inbox;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteSingleHashAggregate rel) {
- AggregateType type = AggregateType.SINGLE;
-
- RelDataType rowType = rel.getRowType();
- RelDataType inputType = rel.getInput().getRowType();
-
- Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
- type, rel.getAggCallList(), inputType);
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- HashAggregateNode<Row> node = new HashAggregateNode<>(ctx, rowType, type, rel.getGroupSets(), accFactory, rowFactory);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteMapHashAggregate rel) {
- AggregateType type = AggregateType.MAP;
-
- RelDataType rowType = rel.getRowType();
- RelDataType inputType = rel.getInput().getRowType();
-
- Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
- type, rel.getAggCallList(), inputType);
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- HashAggregateNode<Row> node = new HashAggregateNode<>(ctx, rowType, type, rel.getGroupSets(), accFactory, rowFactory);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteReduceHashAggregate rel) {
- AggregateType type = AggregateType.REDUCE;
-
- RelDataType rowType = rel.getRowType();
-
- Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
- type, rel.getAggregateCalls(), null);
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- HashAggregateNode<Row> node = new HashAggregateNode<>(ctx, rowType, type, rel.getGroupSets(), accFactory, rowFactory);
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteSingleSortAggregate rel) {
- AggregateType type = AggregateType.SINGLE;
-
- RelDataType rowType = rel.getRowType();
- RelDataType inputType = rel.getInput().getRowType();
-
- Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
- type,
- rel.getAggCallList(),
- inputType
- );
-
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- SortAggregateNode<Row> node = new SortAggregateNode<>(
- ctx,
- rowType,
- type,
- rel.getGroupSet(),
- accFactory,
- rowFactory,
- expressionFactory.comparator(rel.collation())
- );
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteMapSortAggregate rel) {
- AggregateType type = AggregateType.MAP;
-
- RelDataType rowType = rel.getRowType();
- RelDataType inputType = rel.getInput().getRowType();
-
- Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
- type,
- rel.getAggCallList(),
- inputType
- );
-
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- SortAggregateNode<Row> node = new SortAggregateNode<>(
- ctx,
- rowType,
- type,
- rel.getGroupSet(),
- accFactory,
- rowFactory,
- expressionFactory.comparator(rel.collation())
- );
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteReduceSortAggregate rel) {
- AggregateType type = AggregateType.REDUCE;
-
- RelDataType rowType = rel.getRowType();
-
- Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
- type,
- rel.getAggregateCalls(),
- null
- );
-
- RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
-
- SortAggregateNode<Row> node = new SortAggregateNode<>(
- ctx,
- rowType,
- type,
- rel.getGroupSet(),
- accFactory,
- rowFactory,
- expressionFactory.comparator(rel.collation())
- );
-
- Node<Row> input = visit(rel.getInput());
-
- node.register(input);
-
- return node;
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteRel rel) {
- return rel.accept(this);
- }
-
- /** {@inheritDoc} */
- @Override public Node<Row> visit(IgniteExchange rel) {
- throw new AssertionError();
- }
-
- /** */
- private Node<Row> visit(RelNode rel) {
- return visit((IgniteRel) rel);
- }
-
- /** */
- public <T extends Node<Row>> T go(IgniteRel rel) {
- return (T)visit(rel);
- }
-}
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
deleted file mode 100644
index cdc6b54..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java
+++ /dev/null
@@ -1,99 +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.exec;
-
-import java.util.Collection;
-import java.util.UUID;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public interface MailboxRegistry {
- /**
- * Tries to register and inbox node and returns it if success or returns previously registered inbox otherwise.
- *
- * @param inbox Inbox.
- * @return Registered inbox.
- */
- <T> Inbox<T> register(Inbox<T> 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 qryId Query ID.
- * @param exchangeId Exchange ID.
- *
- * @return Registered outbox. May be {@code null} if execution was cancelled.
- */
- Outbox<?> outbox(UUID qryId, long exchangeId);
-
- /**
- * Returns a registered inbox by provided query ID, exchange ID pair.
- *
- * @param qryId Query ID.
- * @param exchangeId Exchange ID.
- *
- * @return Registered inbox. May be {@code null} if execution was cancelled.
- */
- Inbox<?> inbox(UUID qryId, long exchangeId);
-
- /**
- * Returns all registered inboxes for provided query ID.
- *
- * @param qryId Query ID. {@code null} means return inboxes with any query id.
- * @param fragmentId Fragment Id. {@code -1} means return inboxes with any fragment id.
- * @param exchangeId Exchange Id. {@code -1} means return inboxes with any exchange id.
- * @return Registered inboxes.
- */
- Collection<Inbox<?>> inboxes(@Nullable UUID qryId, long fragmentId, long exchangeId);
-
- /**
- * Returns all registered outboxes for provided query ID.
- *
- * @param qryId Query ID. {@code null} means return outboxes with any query id.
- * @param fragmentId Fragment Id. {@code -1} means return outboxes with any fragment id.
- * @param exchangeId Exchange Id. {@code -1} means return outboxes with any exchange id.
- * @return Registered outboxes.
- */
- Collection<Outbox<?>> outboxes(@Nullable UUID qryId, long fragmentId, long exchangeId);
-}
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
deleted file mode 100644
index e9c46a9..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java
+++ /dev/null
@@ -1,169 +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.exec;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Mailbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
-import org.apache.ignite.internal.processors.query.calcite.util.NodeLeaveHandler;
-import org.apache.ignite.internal.tostring.S;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.TopologyService;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class MailboxRegistryImpl implements MailboxRegistry {
- /** */
- private static final Predicate<Mailbox<?>> ALWAYS_TRUE = o -> true;
-
- /** */
- private final Map<MailboxKey, Outbox<?>> locals;
-
- /** */
- private final Map<MailboxKey, Inbox<?>> remotes;
-
- public MailboxRegistryImpl(TopologyService topSrvc) {
- locals = new ConcurrentHashMap<>();
- remotes = new ConcurrentHashMap<>();
-
- topSrvc.addEventHandler(new NodeLeaveHandler(this::onNodeLeft));
- }
-
- /** {@inheritDoc} */
- @Override public <T> Inbox<T> register(Inbox<T> inbox) {
- Inbox<T> old = (Inbox<T>)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()), inbox);
- }
-
- /** {@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()), outbox);
- }
-
- /** {@inheritDoc} */
- @Override public Outbox<?> outbox(UUID qryId, long exchangeId) {
- return locals.get(new MailboxKey(qryId, exchangeId));
- }
-
- /** {@inheritDoc} */
- @Override public Inbox<?> inbox(UUID qryId, long exchangeId) {
- return remotes.get(new MailboxKey(qryId, exchangeId));
- }
-
- /** {@inheritDoc} */
- @Override public Collection<Inbox<?>> inboxes(@Nullable UUID qryId, long fragmentId, long exchangeId) {
- return remotes.values().stream()
- .filter(makeFilter(qryId, fragmentId, exchangeId))
- .collect(Collectors.toList());
- }
-
- /** {@inheritDoc} */
- @Override public Collection<Outbox<?>> outboxes(@Nullable UUID qryId, long fragmentId, long exchangeId) {
- return locals.values().stream()
- .filter(makeFilter(qryId, fragmentId, exchangeId))
- .collect(Collectors.toList());
- }
-
- /** */
- private void onNodeLeft(ClusterNode node) {
- locals.values().forEach(n -> n.onNodeLeft(node.id()));
- remotes.values().forEach(n -> n.onNodeLeft(node.id()));
- }
-
- /** */
- private static Predicate<Mailbox<?>> makeFilter(@Nullable UUID qryId, long fragmentId, long exchangeId) {
- Predicate<Mailbox<?>> filter = ALWAYS_TRUE;
- if (qryId != null)
- filter = filter.and(mailbox -> Objects.equals(mailbox.queryId(), qryId));
- if (fragmentId != -1)
- filter = filter.and(mailbox -> mailbox.fragmentId() == fragmentId);
- if (exchangeId != -1)
- filter = filter.and(mailbox -> mailbox.exchangeId() == exchangeId);
-
- return filter;
- }
-
- /** {@inheritDoc} */
- @Override public String toString() {
- return S.toString(MailboxRegistryImpl.class, this);
- }
-
- /** */
- private static class MailboxKey {
- /** */
- private final UUID qryId;
-
- /** */
- private final long exchangeId;
-
- /** */
- private MailboxKey(UUID qryId, long exchangeId) {
- this.qryId = qryId;
- 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 qryId.equals(that.qryId);
- }
-
- /** {@inheritDoc} */
- @Override public int hashCode() {
- int res = qryId.hashCode();
- res = 31 * res + (int) (exchangeId ^ (exchangeId >>> 32));
- return res;
- }
-
- /** {@inheritDoc} */
- @Override public String toString() {
- return S.toString(MailboxKey.class, this);
- }
- }
-}
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
index 92f4546..abcad0a 100644
--- 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
@@ -30,7 +30,7 @@ public class QueryTaskExecutorImpl implements QueryTaskExecutor, Thread.Uncaught
private static final IgniteLogger LOG = IgniteLogger.forClass(QueryTaskExecutorImpl.class);
/** */
- private final StripedThreadPoolExecutor stripedThreadPoolExecutor;
+ private StripedThreadPoolExecutor stripedThreadPoolExecutor;
/** */
private Thread.UncaughtExceptionHandler eHnd;
@@ -38,7 +38,7 @@ public class QueryTaskExecutorImpl implements QueryTaskExecutor, Thread.Uncaught
/**
* @param stripedThreadPoolExecutor Executor.
*/
- public QueryTaskExecutorImpl(StripedThreadPoolExecutor stripedThreadPoolExecutor) {
+ public void stripedThreadPoolExecutor(StripedThreadPoolExecutor stripedThreadPoolExecutor) {
this.stripedThreadPoolExecutor = stripedThreadPoolExecutor;
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
deleted file mode 100644
index f25161c..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
+++ /dev/null
@@ -1,519 +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.exec.rel;
-
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.stream.Collectors;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistry;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * A part of exchange.
- */
-public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, SingleNode<Row> {
- /** */
- private final ExchangeService exchange;
-
- /** */
- private final MailboxRegistry registry;
-
- /** */
- private final long exchangeId;
-
- /** */
- private final long srcFragmentId;
-
- /** */
- private final Map<String, Buffer> perNodeBuffers;
-
- /** */
- private volatile Collection<String> srcNodeIds;
-
- /** */
- private Comparator<Row> comp;
-
- /** */
- private List<Buffer> buffers;
-
- /** */
- private int requested;
-
- /** */
- private boolean inLoop;
-
- /**
- * @param ctx Execution context.
- * @param exchange Exchange service.
- * @param registry Mailbox registry.
- * @param exchangeId Exchange ID.
- * @param srcFragmentId Source fragment ID.
- */
- public Inbox(
- ExecutionContext<Row> ctx,
- ExchangeService exchange,
- MailboxRegistry registry,
- long exchangeId,
- long srcFragmentId
- ) {
- super(ctx, ctx.getTypeFactory().createUnknownType());
- this.exchange = exchange;
- this.registry = registry;
-
- this.srcFragmentId = srcFragmentId;
- this.exchangeId = exchangeId;
-
- perNodeBuffers = new HashMap<>();
- }
-
- /** {@inheritDoc} */
- @Override public long exchangeId() {
- return exchangeId;
- }
-
- /**
- * Inits this Inbox.
- *
- * @param ctx Execution context.
- * @param srcNodeIds Source node IDs.
- * @param comp Optional comparator for merge exchange.
- */
- public void init(
- ExecutionContext<Row> ctx, RelDataType rowType, Collection<String> srcNodeIds, @Nullable Comparator<Row> comp) {
- assert context().fragmentId() == ctx.fragmentId() : "different fragments unsupported: previous=" + context().fragmentId() +
- " current=" + ctx.fragmentId();
-
- // It's important to set proper context here because
- // the one, that is created on a first message
- // received doesn't have all context variables in place.
- context(ctx);
- rowType(rowType);
-
- this.comp = comp;
-
- // memory barier
- this.srcNodeIds = new HashSet<>(srcNodeIds);
- }
-
- /** {@inheritDoc} */
- @Override public void request(int rowsCnt) throws Exception {
- assert srcNodeIds != null;
- assert rowsCnt > 0 && requested == 0;
-
- checkState();
-
- requested = rowsCnt;
-
- if (!inLoop)
- context().execute(this::doPush, this::onError);
- }
-
- /** {@inheritDoc} */
- @Override public void closeInternal() {
- super.closeInternal();
-
- registry.unregister(this);
- }
-
- /** {@inheritDoc} */
- @Override protected Downstream<Row> requestDownstream(int idx) {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override public void register(List<Node<Row>> sources) {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override protected void rewindInternal() {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Pushes a batch into a buffer.
- *
- * @param srcNodeId Source node id.
- * @param batchId Batch ID.
- * @param last Last batch flag.
- * @param rows Rows.
- */
- public void onBatchReceived(String srcNodeId, int batchId, boolean last, List<Row> rows) throws Exception {
- Buffer buf = getOrCreateBuffer(srcNodeId);
-
- boolean waitingBefore = buf.check() == State.WAITING;
-
- buf.offer(batchId, last, rows);
-
- if (requested > 0 && waitingBefore && buf.check() != State.WAITING)
- push();
- }
-
- /** */
- private void doPush() throws Exception {
- checkState();
-
- push();
- }
-
- /** */
- private void push() throws Exception {
- if (buffers == null) {
- for (String node : srcNodeIds)
- checkNode(node);
-
- buffers = srcNodeIds.stream()
- .map(this::getOrCreateBuffer)
- .collect(Collectors.toList());
-
- assert buffers.size() == perNodeBuffers.size();
- }
-
- if (comp != null)
- pushOrdered();
- else
- pushUnordered();
- }
-
- /** */
- private void pushOrdered() throws Exception {
- PriorityQueue<Pair<Row, Buffer>> heap =
- new PriorityQueue<>(Math.max(buffers.size(), 1), Map.Entry.comparingByKey(comp));
-
- Iterator<Buffer> it = buffers.iterator();
-
- while (it.hasNext()) {
- Buffer buf = it.next();
-
- switch (buf.check()) {
- case END:
- it.remove();
-
- break;
- case READY:
- heap.offer(Pair.of(buf.peek(), buf));
-
- break;
- case WAITING:
-
- return;
- }
- }
-
- inLoop = true;
- try {
- while (requested > 0 && !heap.isEmpty()) {
- checkState();
-
- Buffer buf = heap.poll().right;
-
- requested--;
- downstream().push(buf.remove());
-
- switch (buf.check()) {
- case END:
- buffers.remove(buf);
-
- break;
- case READY:
- heap.offer(Pair.of(buf.peek(), buf));
-
- break;
- case WAITING:
-
- return;
- }
- }
- }
- finally {
- inLoop = false;
- }
-
- if (requested > 0 && heap.isEmpty()) {
- assert buffers.isEmpty();
-
- requested = 0;
- downstream().end();
- }
- }
-
- /** */
- private void pushUnordered() throws Exception {
- int idx = 0, noProgress = 0;
-
- inLoop = true;
- try {
- while (requested > 0 && !buffers.isEmpty()) {
- checkState();
-
- Buffer buf = buffers.get(idx);
-
- switch (buf.check()) {
- case END:
- buffers.remove(idx--);
-
- break;
- case READY:
- noProgress = 0;
- requested--;
- downstream().push(buf.remove());
-
- break;
- case WAITING:
- if (++noProgress >= buffers.size())
- return;
-
- break;
- }
-
- if (++idx == buffers.size())
- idx = 0;
- }
- }
- finally {
- inLoop = false;
- }
-
- if (requested > 0 && buffers.isEmpty()) {
- requested = 0;
- downstream().end();
- }
- }
-
- /** */
- private void acknowledge(String nodeId, int batchId) throws IgniteInternalCheckedException {
- exchange.acknowledge(nodeId, queryId(), srcFragmentId, exchangeId, batchId);
- }
-
- /** */
- private Buffer getOrCreateBuffer(String nodeId) {
- return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
- }
-
- /** */
- private Buffer createBuffer(String nodeId) {
- return new Buffer(nodeId);
- }
-
- /** */
- public void onNodeLeft(String nodeId) {
- if (context().originatingNodeId().equals(nodeId) && srcNodeIds == null)
- context().execute(this::close, this::onError);
- else if (srcNodeIds != null && srcNodeIds.contains(nodeId))
- context().execute(() -> onNodeLeft0(nodeId), this::onError);
- }
-
- /** */
- private void onNodeLeft0(String nodeId) throws Exception {
- checkState();
-
- if (getOrCreateBuffer(nodeId).check() != State.END)
- throw new IgniteInternalCheckedException("Failed to execute query, node left [nodeId=" + nodeId + ']');
- }
-
- /** */
- private void checkNode(String nodeId) throws IgniteInternalCheckedException {
- if (!exchange.alive(nodeId))
- throw new IgniteInternalCheckedException("Failed to execute query, node left [nodeId=" + nodeId + ']');
- }
-
- /** */
- private static final class Batch<Row> implements Comparable<Batch<Row>> {
- /** */
- private final int batchId;
-
- /** */
- private final boolean last;
-
- /** */
- private final List<Row> rows;
-
- /** */
- private int idx;
-
- /** */
- private Batch(int batchId, boolean last, List<Row> rows) {
- this.batchId = batchId;
- this.last = last;
- this.rows = rows;
- }
-
- /** {@inheritDoc} */
- @Override public boolean equals(Object o) {
- if (this == o)
- return true;
- if (o == null || getClass() != o.getClass())
- return false;
-
- Batch<Row> batch = (Batch<Row>) o;
-
- return batchId == batch.batchId;
- }
-
- /** {@inheritDoc} */
- @Override public int hashCode() {
- return batchId;
- }
-
- /** {@inheritDoc} */
- @Override public int compareTo(@NotNull Inbox.Batch<Row> o) {
- return Integer.compare(batchId, o.batchId);
- }
- }
-
- /** */
- private enum State {
- /** */
- END,
-
- /** */
- READY,
-
- /** */
- WAITING
- }
-
- /** */
- private static final Batch<?> WAITING = new Batch<>(0, false, null);
-
- /** */
- private static final Batch<?> END = new Batch<>(0, false, null);
-
- /** */
- private final class Buffer {
- /** */
- private final String nodeId;
-
- /** */
- private int lastEnqueued = -1;
-
- /** */
- private final PriorityQueue<Batch<Row>> batches = new PriorityQueue<>(IO_BATCH_CNT);
-
- /** */
- private Batch<Row> curr = waitingMark();
-
- /** */
- private Buffer(String nodeId) {
- this.nodeId = nodeId;
- }
-
- /** */
- private void offer(int id, boolean last, List<Row> rows) {
- batches.offer(new Batch<>(id, last, rows));
- }
-
- /** */
- private Batch<Row> pollBatch() {
- if (batches.isEmpty() || batches.peek().batchId != lastEnqueued + 1)
- return waitingMark();
-
- Batch<Row> batch = batches.poll();
-
- assert batch != null && batch.batchId == lastEnqueued + 1;
-
- lastEnqueued = batch.batchId;
-
- return batch;
- }
-
- /** */
- private State check() {
- if (finished())
- return State.END;
-
- if (waiting())
- return State.WAITING;
-
- if (isEnd()) {
- curr = finishedMark();
-
- return State.END;
- }
-
- return State.READY;
- }
-
- /** */
- private Row peek() {
- assert curr != null;
- assert curr != WAITING;
- assert curr != END;
- assert !isEnd();
-
- return curr.rows.get(curr.idx);
- }
-
- /** */
- private Row remove() throws IgniteInternalCheckedException {
- assert curr != null;
- assert curr != WAITING;
- assert curr != END;
- assert !isEnd();
-
- Row row = curr.rows.set(curr.idx++, null);
-
- if (curr.idx == curr.rows.size()) {
- acknowledge(nodeId, curr.batchId);
-
- if (!isEnd())
- curr = pollBatch();
- }
-
- return row;
- }
-
- /** */
- private boolean finished() {
- return curr == END;
- }
-
- /** */
- private boolean waiting() {
- return curr == WAITING && (curr = pollBatch()) == WAITING;
- }
-
- /** */
- private boolean isEnd() {
- return curr.last && curr.idx == curr.rows.size();
- }
-
- /** */
- private Batch<Row> finishedMark() {
- return (Batch<Row>) END;
- }
-
- /** */
- private Batch<Row> waitingMark() {
- return (Batch<Row>) WAITING;
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
deleted file mode 100644
index 483c8a5..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
+++ /dev/null
@@ -1,361 +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.exec.rel;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistry;
-import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-
-import static org.apache.ignite.internal.processors.query.calcite.util.Commons.IN_BUFFER_SIZE;
-import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
-
-/**
- * A part of exchange.
- */
-public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, SingleNode<Row>, Downstream<Row> {
- /** */
- private final ExchangeService exchange;
-
- /** */
- private final MailboxRegistry registry;
-
- /** */
- private final long exchangeId;
-
- /** */
- private final long targetFragmentId;
-
- /** */
- private final Destination<Row> dest;
-
- /** */
- private final Deque<Row> inBuf = new ArrayDeque<>(inBufSize);
-
- /** */
- private final Map<String, Buffer> nodeBuffers = new HashMap<>();
-
- /** */
- private int waiting;
-
- /**
- * @param ctx Execution context.
- * @param exchange Exchange service.
- * @param registry Mailbox registry.
- * @param exchangeId Exchange ID.
- * @param targetFragmentId Target fragment ID.
- * @param dest Destination.
- */
- public Outbox(
- ExecutionContext<Row> ctx,
- RelDataType rowType,
- ExchangeService exchange,
- MailboxRegistry registry,
- long exchangeId,
- long targetFragmentId,
- Destination<Row> dest
- ) {
- super(ctx, rowType);
- this.exchange = exchange;
- this.registry = registry;
- this.targetFragmentId = targetFragmentId;
- this.exchangeId = exchangeId;
- this.dest = dest;
- }
-
- /** {@inheritDoc} */
- @Override public long exchangeId() {
- return exchangeId;
- }
-
- /**
- * callback method.
- *
- * @param nodeId Target ID.
- * @param batchId Batch ID.
- */
- public void onAcknowledge(String nodeId, int batchId) throws Exception {
- assert nodeBuffers.containsKey(nodeId);
-
- checkState();
-
- nodeBuffers.get(nodeId).acknowledge(batchId);
- }
-
- /** */
- public void init() {
- try {
- checkState();
-
- flush();
- }
- catch (Throwable t) {
- onError(t);
- }
- }
-
- /** {@inheritDoc} */
- @Override public void request(int rowCnt) {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override public void push(Row row) throws Exception {
- assert waiting > 0;
-
- checkState();
-
- waiting--;
-
- inBuf.add(row);
-
- flush();
- }
-
- /** {@inheritDoc} */
- @Override public void end() throws Exception {
- assert waiting > 0;
-
- checkState();
-
- waiting = -1;
-
- flush();
- }
-
- /** {@inheritDoc} */
- @Override protected void onErrorInternal(Throwable e) {
- log.error("Error occurred during execution", e);
-
- try {
- sendError(e);
- }
- catch (IgniteInternalCheckedException ex) {
- log.error("Error occurred during send error message", e);
- }
- finally {
- Commons.closeQuiet(this);
- }
- }
-
- /** {@inheritDoc} */
- @Override public void closeInternal() {
- super.closeInternal();
-
- registry.unregister(this);
-
- // Send cancel message for the Inbox to close Inboxes created by batch message race.
- for (String node : dest.targets())
- getOrCreateBuffer(node).close();
- }
-
- /** {@inheritDoc} */
- @Override public void onRegister(Downstream<Row> downstream) {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override protected void rewindInternal() {
- throw new UnsupportedOperationException();
- }
-
- /** {@inheritDoc} */
- @Override protected Downstream<Row> requestDownstream(int idx) {
- if (idx != 0)
- throw new IndexOutOfBoundsException();
-
- return this;
- }
-
- /** */
- private void sendBatch(String nodeId, int batchId, boolean last, List<Row> rows) throws IgniteInternalCheckedException {
- exchange.sendBatch(nodeId, queryId(), targetFragmentId, exchangeId, batchId, last, rows);
- }
-
- /** */
- private void sendError(Throwable err) throws IgniteInternalCheckedException {
- exchange.sendError(context().originatingNodeId(), queryId(), fragmentId(), err);
- }
-
- /** */
- private void sendInboxClose(String nodeId) {
- try {
- exchange.closeInbox(nodeId, queryId(), targetFragmentId, exchangeId);
- }
- catch (IgniteInternalCheckedException e) {
- log.error("Failed to send cancel message", e);
- }
- }
-
- /** */
- private Buffer getOrCreateBuffer(String nodeId) {
- return nodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
- }
-
- /** */
- private Buffer createBuffer(String nodeId) {
- return new Buffer(nodeId);
- }
-
- /** */
- private void flush() throws Exception {
- while (!inBuf.isEmpty()) {
- checkState();
-
- Collection<Buffer> buffers = dest.targets(inBuf.peek()).stream()
- .map(this::getOrCreateBuffer)
- .collect(Collectors.toList());
-
- assert !nullOrEmpty(buffers);
-
- if (!buffers.stream().allMatch(Buffer::ready))
- return;
-
- Row row = inBuf.remove();
-
- for (Buffer dest : buffers)
- dest.add(row);
- }
-
- assert inBuf.isEmpty();
-
- if (waiting == 0)
- source().request(waiting = IN_BUFFER_SIZE);
- else if (waiting == -1) {
- for (String node : dest.targets())
- getOrCreateBuffer(node).end();
- }
- }
-
- /** */
- public void onNodeLeft(String nodeId) {
- if (nodeId.equals(context().originatingNodeId()))
- context().execute(this::close, this::onError);
- }
-
- /** */
- private final class Buffer {
- /** */
- private final String nodeId;
-
- /** */
- private int hwm = -1;
-
- /** */
- private int lwm = -1;
-
- /** */
- private List<Row> curr;
-
- /** */
- private Buffer(String nodeId) {
- this.nodeId = nodeId;
-
- curr = new ArrayList<>(IO_BATCH_SIZE);
- }
-
- /**
- * 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 (hwm == Integer.MAX_VALUE)
- return false;
-
- return curr.size() < IO_BATCH_SIZE || hwm - lwm < IO_BATCH_CNT;
- }
-
- /**
- * Adds a row to current batch.
- *
- * @param row Row.
- */
- public void add(Row row) throws IgniteInternalCheckedException {
- assert ready();
-
- if (curr.size() == IO_BATCH_SIZE) {
- sendBatch(nodeId, ++hwm, false, curr);
-
- curr = new ArrayList<>(IO_BATCH_SIZE);
- }
-
- curr.add(row);
- }
-
- /**
- * Signals data is over.
- */
- public void end() throws IgniteInternalCheckedException {
- if (hwm == Integer.MAX_VALUE)
- return;
-
- int batchId = hwm + 1;
- hwm = Integer.MAX_VALUE;
-
- List<Row> tmp = curr;
- curr = null;
-
- sendBatch(nodeId, batchId, true, tmp);
- }
-
- /**
- * Callback method.
- *
- * @param id batch ID.
- */
- private void acknowledge(int id) throws Exception {
- if (lwm > id)
- return;
-
- boolean readyBefore = ready();
-
- lwm = id;
-
- if (!readyBefore && ready())
- flush();
- }
-
- /** */
- public void close() {
- int currBatchId = hwm;
-
- if (hwm == Integer.MAX_VALUE)
- return;
-
- hwm = Integer.MAX_VALUE;
-
- curr = null;
-
- if (currBatchId >= 0)
- sendInboxClose(nodeId);
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
index 2c39021..b4417d9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
@@ -22,6 +22,7 @@ import java.util.Deque;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.UUID;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
@@ -120,7 +121,20 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
/** {@inheritDoc} */
@Override public void closeInternal() {
- context().execute(() -> sources().forEach(Commons::closeQuiet), this::onError);
+ closeInternal0(ex.get() == null);
+ }
+
+ /** */
+ private void closeInternal0(boolean sync) {
+ try {
+ if (sync)
+ context().submit(() -> sources().forEach(Commons::closeQuiet), this::onError).get();
+ else
+ context().execute(() -> sources().forEach(Commons::closeQuiet), this::onError);
+ }
+ catch (InterruptedException | ExecutionException e) {
+ log.warn("Execution is cancelled.", e);
+ }
}
/** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ErrorMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ErrorMessage.java
deleted file mode 100644
index edf4048..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ErrorMessage.java
+++ /dev/null
@@ -1,45 +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.message;
-
-import java.io.Serializable;
-import java.util.UUID;
-
-import org.apache.ignite.network.NetworkMessage;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.ERROR_MESSAGE, autoSerializable = false)
-public interface ErrorMessage extends NetworkMessage, Serializable {
- /**
- * @return Query ID.
- */
- UUID queryId();
-
- /**
- * @return Fragment ID.
- */
- long fragmentId();
-
- /**
- * @return Error.
- */
- Throwable error();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAwareMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAwareMessage.java
deleted file mode 100644
index 05b6cfe..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAwareMessage.java
+++ /dev/null
@@ -1,38 +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.message;
-
-import java.io.Serializable;
-import java.util.UUID;
-
-import org.apache.ignite.network.NetworkMessage;
-
-/**
- * Execution context is used to determine a stripe where to process a message.
- */
-public interface ExecutionContextAwareMessage extends NetworkMessage, Serializable {
- /**
- * @return Query ID.
- */
- UUID queryId();
-
- /**
- * @return Fragment ID.
- */
- long fragmentId();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCloseMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCloseMessage.java
deleted file mode 100644
index b108a7d..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCloseMessage.java
+++ /dev/null
@@ -1,45 +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.message;
-
-import java.io.Serializable;
-import java.util.UUID;
-
-import org.apache.ignite.network.NetworkMessage;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.INBOX_CLOSE_MESSAGE)
-public interface InboxCloseMessage extends NetworkMessage, Serializable {
- /**
- * @return Query ID.
- */
- UUID queryId();
-
- /**
- * @return Fragment ID.
- */
- long fragmentId();
-
- /**
- * @return Exchange ID.
- */
- long exchangeId();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
deleted file mode 100644
index 4c334f0..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
+++ /dev/null
@@ -1,31 +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.message;
-
-import org.apache.ignite.network.NetworkMessage;
-
-/**
- *
- */
-public interface MessageListener {
- /**
- * @param nodeId Sender node ID.
- * @param msg Message.
- */
- void onMessage(String nodeId, NetworkMessage msg);
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
deleted file mode 100644
index cdf24e9..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
+++ /dev/null
@@ -1,50 +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.message;
-
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-import org.apache.ignite.network.NetworkMessage;
-
-/**
- *
- */
-public interface MessageService {
- /**
- * Sends a message to given node.
- *
- * @param nodeId Node ID.
- * @param msg Message.
- */
- void send(String nodeId, NetworkMessage msg) throws IgniteInternalCheckedException;
-
- /**
- * Checks whether a node with given ID is alive.
- *
- * @param nodeId Node ID.
- * @return {@code True} if node is alive.
- */
- boolean alive(String nodeId);
-
- /**
- * Registers a listener for messages of a given type.
- *
- * @param lsnr Listener.
- * @param msgId Message id.
- */
- void register(MessageListener lsnr, short msgId);
-}
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
deleted file mode 100644
index 8b95faa..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java
+++ /dev/null
@@ -1,149 +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.message;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteLogger;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.MessagingService;
-import org.apache.ignite.network.NetworkAddress;
-import org.apache.ignite.network.NetworkMessage;
-import org.apache.ignite.network.TopologyService;
-
-/**
- *
- */
-public class MessageServiceImpl implements MessageService {
- private static final UUID QUERY_ID_STUB = UUID.randomUUID();
-
- private static final IgniteLogger LOG = IgniteLogger.forClass(MessageServiceImpl.class);
-
- private final TopologyService topSrvc;
-
- private final MessagingService messagingSrvc;
-
- /** */
- private final String locNodeId;
-
- /** */
- private final QueryTaskExecutor taskExecutor;
-
- /** */
- private Map<Short, MessageListener> lsnrs;
-
- /** */
- public MessageServiceImpl(
- TopologyService topSrvc,
- MessagingService messagingSrvc,
- QueryTaskExecutor taskExecutor
- ) {
- this.topSrvc = topSrvc;
- this.messagingSrvc = messagingSrvc;
- this.taskExecutor = taskExecutor;
-
- locNodeId = topSrvc.localMember().id();
-
- messagingSrvc.addMessageHandler(this::onMessage);
- }
-
- /** {@inheritDoc} */
- @Override public void send(String nodeId, NetworkMessage msg) throws IgniteInternalCheckedException {
- if (locNodeId.equals(nodeId))
- onMessage(nodeId, msg);
- else {
- ClusterNode node = topSrvc.allMembers().stream()
- .filter(cn -> nodeId.equals(cn.id()))
- .findFirst()
- .orElseThrow(() -> new IgniteInternalException("Failed to send message to node (has node left grid?): " + nodeId));
-
- try {
- messagingSrvc.send(node, msg).get();
- } catch (Exception ex) {
- if (ex instanceof IgniteInternalCheckedException)
- throw (IgniteInternalCheckedException)ex;
-
- throw new IgniteInternalCheckedException(ex);
- }
- }
- }
-
- /** {@inheritDoc} */
- @Override public void register(MessageListener lsnr, short type) {
- if (lsnrs == null)
- lsnrs = new HashMap<>();
-
- MessageListener old = lsnrs.put(type, lsnr);
-
- assert old == null : old;
- }
-
- /** {@inheritDoc} */
- @Override public boolean alive(String nodeId) {
- return topSrvc.allMembers().stream()
- .map(ClusterNode::id)
- .anyMatch(id -> id.equals(nodeId));
- }
-
- /** */
- protected void onMessage(String nodeId, NetworkMessage msg) {
- if (msg instanceof ExecutionContextAwareMessage) {
- ExecutionContextAwareMessage msg0 = (ExecutionContextAwareMessage) msg;
- taskExecutor.execute(msg0.queryId(), msg0.fragmentId(), () -> onMessageInternal(nodeId, msg));
- }
- else {
- taskExecutor.execute(
- QUERY_ID_STUB,
- ThreadLocalRandom.current().nextLong(1024),
- () -> onMessageInternal(nodeId, msg)
- );
- }
- }
-
- /** */
- private void onMessage(NetworkMessage msg, NetworkAddress addr, String correlationId) {
- ClusterNode node = topSrvc.getByAddress(addr);
-
- if (node == null) {
- LOG.warn("Received a message from a node that has not yet" +
- " joined the cluster: addr={}, msg={}", addr, msg);
-
- return;
- }
-
- if (msg.groupType() == 3)
- onMessage(node.id(), msg);
- }
-
- /** */
- private void onMessageInternal(String nodeId, NetworkMessage msg) {
- MessageListener lsnr = Objects.requireNonNull(
- lsnrs.get(msg.messageType()),
- "there is no listener for msgType=" + msg.groupType()
- );
-
- lsnr.onMessage(nodeId, msg);
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/OutboxCloseMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/OutboxCloseMessage.java
deleted file mode 100644
index debf881..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/OutboxCloseMessage.java
+++ /dev/null
@@ -1,45 +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.message;
-
-import java.io.Serializable;
-import java.util.UUID;
-
-import org.apache.ignite.network.NetworkMessage;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.OUTBOX_CLOSE_MESSAGE)
-public interface OutboxCloseMessage extends NetworkMessage, Serializable {
- /**
- * @return Query ID.
- */
- UUID queryId();
-
- /**
- * @return Fragment ID.
- */
- long fragmentId();
-
- /**
- * @return Exchange ID.
- */
- long exchangeId();
-}
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
deleted file mode 100644
index 54dd27c..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
+++ /dev/null
@@ -1,36 +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.message;
-
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.QUERY_BATCH_ACK)
-public interface QueryBatchAcknowledgeMessage extends ExecutionContextAwareMessage {
- /**
- * @return Exchange ID.
- */
- long exchangeId();
-
- /**
- * @return Batch ID.
- */
- int batchId();
-}
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
deleted file mode 100644
index f7e70bb..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java
+++ /dev/null
@@ -1,48 +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.message;
-
-import java.util.List;
-
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.QUERY_BATCH_MESSAGE, autoSerializable = false)
-public interface QueryBatchMessage extends ExecutionContextAwareMessage {
- /**
- * @return Exchange ID.
- */
- long exchangeId();
-
- /**
- * @return Batch ID.
- */
- int batchId();
-
- /**
- * @return Last batch flag.
- */
- boolean last();
-
- /**
- * @return Rows.
- */
- List<Object> rows();
-}
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
deleted file mode 100644
index bb6b710..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.message;
-
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.QUERY_START_REQUEST, autoSerializable = false)
-public interface QueryStartRequest extends ExecutionContextAwareMessage {
- /**
- * @return Schema name.
- */
- String schema();
-
- /**
- * @return Fragment description.
- */
- FragmentDescription fragmentDescription();
-
- /**
- * @return Topology version.
- */
- long topologyVersion();
-
- /**
- * @return Fragment plan.
- */
- String root();
-
- /**
- * @return Query parameters.
- */
- Object[] parameters();
-}
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
deleted file mode 100644
index f6d2f9b..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java
+++ /dev/null
@@ -1,45 +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.message;
-
-import java.io.Serializable;
-import java.util.UUID;
-
-import org.apache.ignite.network.NetworkMessage;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- *
- */
-@Transferable(value = SqlQueryMessageGroup.QUERY_START_RESPONSE, autoSerializable = false)
-public interface QueryStartResponse extends NetworkMessage, Serializable {
- /**
- * @return Query ID.
- */
- UUID queryId();
-
- /**
- * @return Fragment ID.
- */
- long fragmentId();
-
- /**
- * @return Error.
- */
- Throwable error();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/SqlQueryMessageGroup.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/SqlQueryMessageGroup.java
deleted file mode 100644
index e8cbc84..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/SqlQueryMessageGroup.java
+++ /dev/null
@@ -1,40 +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.message;
-
-import org.apache.ignite.network.annotations.MessageGroup;
-
-/**
- * Message types for the sql query processing module.
- */
-@MessageGroup(groupType = 3, groupName = "SqlQueryMessages")
-public final class SqlQueryMessageGroup {
- public static final short QUERY_START_REQUEST = 0;
-
- public static final short QUERY_START_RESPONSE = 1;
-
- public static final short ERROR_MESSAGE = 2;
-
- public static final short QUERY_BATCH_MESSAGE = 3;
-
- public static final short QUERY_BATCH_ACK = 4;
-
- public static final short INBOX_CLOSE_MESSAGE = 5;
-
- public static final short OUTBOX_CLOSE_MESSAGE = 6;
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/AffinityService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/AffinityService.java
deleted file mode 100644
index dd78685..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/AffinityService.java
+++ /dev/null
@@ -1,33 +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.metadata;
-
-import java.util.function.ToIntFunction;
-
-/**
- *
- */
-public interface AffinityService {
- /**
- * Creates a partition mapping function on the basis of affinity function of cache with given ID.
- *
- * @param cacheId Cache ID.
- * @return Affinity function.
- */
- ToIntFunction<Object> affinity(int cacheId);
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java
index 743bb58..fa6b1f8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java
@@ -17,13 +17,13 @@
package org.apache.ignite.internal.processors.query.calcite.metadata;
-import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Set;
+import java.util.UUID;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
@@ -36,7 +36,7 @@ import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
import static org.apache.ignite.internal.util.IgniteUtils.firstNotNull;
/** */
-public class ColocationGroup implements Serializable {
+public class ColocationGroup {
/** */
private static final int SYNTHETIC_PARTITIONS_COUNT = 512;
// TODO: IgniteSystemProperties.getInteger("IGNITE_CALCITE_SYNTHETIC_PARTITIONS_COUNT", 512);
@@ -45,18 +45,18 @@ public class ColocationGroup implements Serializable {
private List<Long> sourceIds;
/** */
- private List<String> nodeIds;
+ private List<UUID> nodeIds;
/** */
- private List<List<String>> assignments;
+ private List<List<UUID>> assignments;
/** */
- public static ColocationGroup forNodes(List<String> nodeIds) {
+ public static ColocationGroup forNodes(List<UUID> nodeIds) {
return new ColocationGroup(null, nodeIds, null);
}
/** */
- public static ColocationGroup forAssignments(List<List<String>> assignments) {
+ public static ColocationGroup forAssignments(List<List<UUID>> assignments) {
return new ColocationGroup(null, null, assignments);
}
@@ -66,7 +66,7 @@ public class ColocationGroup implements Serializable {
}
/** */
- private ColocationGroup(List<Long> sourceIds, List<String> nodeIds, List<List<String>> assignments) {
+ private ColocationGroup(List<Long> sourceIds, List<UUID> nodeIds, List<List<UUID>> assignments) {
this.sourceIds = sourceIds;
this.nodeIds = nodeIds;
this.assignments = assignments;
@@ -82,7 +82,7 @@ public class ColocationGroup implements Serializable {
/**
* @return Lists of nodes capable to execute a query fragment for what the mapping is calculated.
*/
- public List<String> nodeIds() {
+ public List<UUID> nodeIds() {
return nodeIds == null ? Collections.emptyList() : nodeIds;
}
@@ -90,7 +90,7 @@ public class ColocationGroup implements Serializable {
* @return List of partitions (index) and nodes (items) having an appropriate partition in
* OWNING state, calculated for distributed tables, involved in query execution.
*/
- public List<List<String>> assignments() {
+ public List<List<UUID>> assignments() {
return assignments == null ? Collections.emptyList() : assignments;
}
@@ -123,7 +123,7 @@ public class ColocationGroup implements Serializable {
else
sourceIds = Commons.combine(this.sourceIds, other.sourceIds);
- List<String> nodeIds;
+ List<UUID> nodeIds;
if (this.nodeIds == null || other.nodeIds == null)
nodeIds = firstNotNull(this.nodeIds, other.nodeIds);
else
@@ -134,16 +134,16 @@ public class ColocationGroup implements Serializable {
"Replicated query parts are not co-located on all nodes");
}
- List<List<String>> assignments;
+ List<List<UUID>> assignments;
if (this.assignments == null || other.assignments == null) {
assignments = firstNotNull(this.assignments, other.assignments);
if (assignments != null && nodeIds != null) {
- Set<String> filter = new HashSet<>(nodeIds);
- List<List<String>> assignments0 = new ArrayList<>(assignments.size());
+ Set<UUID> filter = new HashSet<>(nodeIds);
+ List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
for (int i = 0; i < assignments.size(); i++) {
- List<String> assignment = Commons.intersect(filter, assignments.get(i));
+ List<UUID> assignment = Commons.intersect(filter, assignments.get(i));
if (assignment.isEmpty()) { // TODO check with partition filters
throw new ColocationMappingException("Failed to map fragment to location. " +
@@ -159,9 +159,9 @@ public class ColocationGroup implements Serializable {
else {
assert this.assignments.size() == other.assignments.size();
assignments = new ArrayList<>(this.assignments.size());
- Set<String> filter = nodeIds == null ? null : new HashSet<>(nodeIds);
+ Set<UUID> filter = nodeIds == null ? null : new HashSet<>(nodeIds);
for (int i = 0; i < this.assignments.size(); i++) {
- List<String> assignment = Commons.intersect(this.assignments.get(i), other.assignments.get(i));
+ List<UUID> assignment = Commons.intersect(this.assignments.get(i), other.assignments.get(i));
if (filter != null)
assignment.retainAll(filter);
@@ -182,10 +182,10 @@ public class ColocationGroup implements Serializable {
return this;
if (assignments != null) {
- List<List<String>> assignments = new ArrayList<>(this.assignments.size());
- Set<String> nodes = new HashSet<>();
- for (List<String> assignment : this.assignments) {
- String first = first(assignment);
+ List<List<UUID>> assignments = new ArrayList<>(this.assignments.size());
+ Set<UUID> nodes = new HashSet<>();
+ for (List<UUID> assignment : this.assignments) {
+ UUID first = first(assignment);
if (first != null)
nodes.add(first);
assignments.add(first != null ? Collections.singletonList(first) : Collections.emptyList());
@@ -198,13 +198,13 @@ public class ColocationGroup implements Serializable {
}
/** */
- public ColocationGroup mapToNodes(List<String> nodeIds) {
+ public ColocationGroup mapToNodes(List<UUID> nodeIds) {
return !nullOrEmpty(this.nodeIds) ? this : forNodes0(nodeIds);
}
/** */
- @NotNull private ColocationGroup forNodes0(List<String> nodeIds) {
- List<List<String>> assignments = new ArrayList<>(SYNTHETIC_PARTITIONS_COUNT);
+ @NotNull private ColocationGroup forNodes0(List<UUID> nodeIds) {
+ List<List<UUID>> assignments = new ArrayList<>(SYNTHETIC_PARTITIONS_COUNT);
for (int i = 0; i < SYNTHETIC_PARTITIONS_COUNT; i++)
assignments.add(asList(nodeIds.get(i % nodeIds.size())));
return new ColocationGroup(sourceIds, nodeIds, assignments);
@@ -216,11 +216,11 @@ public class ColocationGroup implements Serializable {
* @param nodeId Cluster node ID.
* @return List of partitions to scan on the given node.
*/
- public int[] partitions(String nodeId) {
+ public int[] partitions(UUID nodeId) {
IgniteIntList parts = new IgniteIntList(assignments.size());
for (int i = 0; i < assignments.size(); i++) {
- List<String> assignment = assignments.get(i);
+ List<UUID> assignment = assignments.get(i);
if (Objects.equals(nodeId, first(assignment)))
parts.add(i);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
index cc1dc43..f5ede72 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
@@ -17,12 +17,12 @@
package org.apache.ignite.internal.processors.query.calcite.metadata;
-import java.io.Serializable;
import java.util.List;
import java.util.Map;
+import java.util.UUID;
/** */
-public class FragmentDescription implements Serializable {
+public class FragmentDescription {
/** */
private long fragmentId;
@@ -33,7 +33,7 @@ public class FragmentDescription implements Serializable {
private ColocationGroup target;
/** */
- private Map<Long, List<String>> remoteSources;
+ private Map<Long, List<UUID>> remoteSources;
/** */
public FragmentDescription() {
@@ -41,7 +41,7 @@ public class FragmentDescription implements Serializable {
/** */
public FragmentDescription(long fragmentId, FragmentMapping mapping, ColocationGroup target,
- Map<Long, List<String>> remoteSources) {
+ Map<Long, List<UUID>> remoteSources) {
this.fragmentId = fragmentId;
this.mapping = mapping;
this.target = target;
@@ -54,7 +54,7 @@ public class FragmentDescription implements Serializable {
}
/** */
- public List<String> nodeIds() {
+ public List<UUID> nodeIds() {
return mapping.nodeIds();
}
@@ -64,7 +64,7 @@ public class FragmentDescription implements Serializable {
}
/** */
- public Map<Long, List<String>> remotes() {
+ public Map<Long, List<UUID>> remotes() {
return remoteSources;
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java
index 0d3ae61..5d8f2ad 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java
@@ -17,9 +17,9 @@
package org.apache.ignite.internal.processors.query.calcite.metadata;
-import java.io.Serializable;
import java.util.Collections;
import java.util.List;
+import java.util.UUID;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -34,7 +34,7 @@ import static org.apache.ignite.internal.util.IgniteUtils.firstNotNull;
/**
*
*/
-public class FragmentMapping implements Serializable {
+public class FragmentMapping {
/** */
private List<ColocationGroup> colocationGroups;
@@ -58,7 +58,7 @@ public class FragmentMapping implements Serializable {
}
/** */
- public static FragmentMapping create(String nodeId) {
+ public static FragmentMapping create(UUID nodeId) {
return new FragmentMapping(ColocationGroup.forNodes(Collections.singletonList(nodeId)));
}
@@ -111,21 +111,21 @@ public class FragmentMapping implements Serializable {
}
/** */
- public List<String> nodeIds() {
+ public List<UUID> nodeIds() {
return colocationGroups.stream()
.flatMap(g -> g.nodeIds().stream())
.distinct().collect(Collectors.toList());
}
/** */
- public FragmentMapping finalize(Supplier<List<String>> nodesSource) {
+ public FragmentMapping finalize(Supplier<List<UUID>> nodesSource) {
if (colocationGroups.isEmpty())
return this;
List<ColocationGroup> colocationGroups = this.colocationGroups;
colocationGroups = Commons.transform(colocationGroups, ColocationGroup::finalaze);
- List<String> nodes = nodeIds(), nodes0 = nodes.isEmpty() ? nodesSource.get() : nodes;
+ List<UUID> nodes = nodeIds(), nodes0 = nodes.isEmpty() ? nodesSource.get() : nodes;
colocationGroups = Commons.transform(colocationGroups, g -> g.mapToNodes(nodes0));
return new FragmentMapping(colocationGroups);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
index 9882aa1..06d60a4 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
@@ -32,7 +32,7 @@ import org.apache.calcite.util.ImmutableIntList;
import org.apache.calcite.util.Util;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSortedIndexSpool;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMinusBase;
import org.jetbrains.annotations.Nullable;
import static org.apache.calcite.util.NumberUtil.multiply;
@@ -118,9 +118,9 @@ public class IgniteMdRowCount extends RelMdRowCount {
}
/**
- * Estimation of row count for set op (MINUS, INTERSECT).
+ * Estimation of row count for MINUS (EXCEPT) operator.
*/
- public double getRowCount(IgniteSetOp rel, RelMetadataQuery mq) {
+ public double getRowCount(IgniteMinusBase rel, RelMetadataQuery mq) {
return rel.estimateRowCount(mq);
}
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 85fb1d5..e59dbfe 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
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.query.calcite.metadata;
import java.util.List;
+import java.util.UUID;
import java.util.function.Predicate;
import org.apache.ignite.network.ClusterNode;
@@ -36,5 +37,5 @@ public interface MappingService {
* @param nodeFilter Node filter.
* @return Nodes mapping for intermediate fragments.
*/
- List<String> executionNodes(long topVer, boolean single, @Nullable Predicate<ClusterNode> nodeFilter);
+ List<UUID> executionNodes(long topVer, boolean single, @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
deleted file mode 100644
index eb9344d..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java
+++ /dev/null
@@ -1,60 +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.metadata;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.TopologyService;
-import org.jetbrains.annotations.Nullable;
-
-import static java.util.Collections.singletonList;
-import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
-
-/**
- *
- */
-public class MappingServiceImpl implements MappingService {
- /** */
- private final TopologyService topSrvc;
-
- public MappingServiceImpl(TopologyService topSrvc) {
- this.topSrvc = topSrvc;
- }
-
- /** {@inheritDoc} */
- @Override public List<String> executionNodes(long topVer, boolean single, @Nullable Predicate<ClusterNode> nodeFilter) {
- List<ClusterNode> nodes = new ArrayList<>(topSrvc.allMembers());
-
- if (nodeFilter != null)
- nodes = nodes.stream().filter(nodeFilter).collect(Collectors.toList());
-
- if (single && nodes.size() > 1)
- nodes = singletonList(nodes.get(ThreadLocalRandom.current().nextInt(nodes.size())));
-
- if (nullOrEmpty(nodes))
- throw new IllegalStateException("failed to map query to execution nodes. Nodes list is empty.");
-
- return Commons.transform(nodes, ClusterNode::id);
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RemoteException.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RemoteException.java
deleted file mode 100644
index a8810bb..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RemoteException.java
+++ /dev/null
@@ -1,68 +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.metadata;
-
-import java.util.UUID;
-
-/**
- *
- */
-public class RemoteException extends RuntimeException {
- /** */
- private final String nodeId;
-
- /** */
- private final UUID queryId;
-
- /** */
- private final long fragmentId;
-
- /**
- * @param cause Cause.
- * @param nodeId Node ID.
- * @param queryId Query ID.
- * @param fragmentId Fragment ID.
- */
- public RemoteException(String nodeId, UUID queryId, long fragmentId, Throwable cause) {
- super("Remote query execution", cause);
- this.nodeId = nodeId;
- this.queryId = queryId;
- this.fragmentId = fragmentId;
- }
-
- /**
- * @return Node ID.
- */
- public String nodeId() {
- return nodeId;
- }
-
- /**
- * @return Query ID.
- */
- public UUID queryId() {
- return queryId;
- }
-
- /**
- * @return Fragment ID.
- */
- public long fragmentId() {
- return fragmentId;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java
index 6f9f1b2..2bfba75 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java
@@ -21,6 +21,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.UUID;
import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
@@ -35,7 +36,7 @@ import static org.apache.ignite.internal.util.IgniteUtils.newHashMap;
*/
public abstract class AbstractMultiStepPlan implements MultiStepPlan {
/** */
- protected final FieldsMetadata fieldsMetadata;
+ protected final Object fieldsMetadata;
/** */
protected final QueryTemplate queryTemplate;
@@ -44,7 +45,7 @@ public abstract class AbstractMultiStepPlan implements MultiStepPlan {
protected ExecutionPlan executionPlan;
/** */
- protected AbstractMultiStepPlan(QueryTemplate queryTemplate, FieldsMetadata fieldsMetadata) {
+ protected AbstractMultiStepPlan(QueryTemplate queryTemplate, Object fieldsMetadata) {
this.queryTemplate = queryTemplate;
this.fieldsMetadata = fieldsMetadata;
}
@@ -55,11 +56,6 @@ public abstract class AbstractMultiStepPlan implements MultiStepPlan {
}
/** {@inheritDoc} */
- @Override public FieldsMetadata fieldsMetadata() {
- return fieldsMetadata;
- }
-
- /** {@inheritDoc} */
@Override public FragmentMapping mapping(Fragment fragment) {
return mapping(fragment.fragmentId());
}
@@ -74,13 +70,13 @@ public abstract class AbstractMultiStepPlan implements MultiStepPlan {
}
/** {@inheritDoc} */
- @Override public Map<Long, List<String>> remotes(Fragment fragment) {
+ @Override public Map<Long, List<UUID>> remotes(Fragment fragment) {
List<IgniteReceiver> remotes = fragment.remotes();
if (nullOrEmpty(remotes))
return null;
- HashMap<Long, List<String>> res = newHashMap(remotes.size());
+ HashMap<Long, List<UUID>> res = newHashMap(remotes.size());
for (IgniteReceiver remote : remotes)
res.put(remote.exchangeId(), mapping(remote.sourceFragmentId()).nodeIds());
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
deleted file mode 100644
index a1c5bf9..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
+++ /dev/null
@@ -1,78 +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.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/Cloner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java
index af89e28..eda9fd9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java
@@ -47,7 +47,9 @@ import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceH
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
import static org.apache.ignite.internal.util.ArrayUtils.asList;
@@ -229,7 +231,17 @@ public class Cloner implements IgniteRelVisitor<IgniteRel> {
}
/** {@inheritDoc} */
- @Override public IgniteRel visit(IgniteSetOp rel) {
+ @Override public IgniteRel visit(IgniteSingleMinus rel) {
+ return rel.clone(cluster, Commons.transform(rel.getInputs(), rel0 -> visit((IgniteRel) rel0)));
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteMapMinus rel) {
+ return rel.clone(cluster, Commons.transform(rel.getInputs(), rel0 -> visit((IgniteRel) rel0)));
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteReduceMinus rel) {
return rel.clone(cluster, Commons.transform(rel.getInputs(), rel0 -> visit((IgniteRel) rel0)));
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DdlPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DdlPlan.java
deleted file mode 100644
index 79d3880..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DdlPlan.java
+++ /dev/null
@@ -1,46 +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.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
-
-/** */
-public class DdlPlan implements QueryPlan {
- /** */
- private final DdlCommand cmd;
-
- /** */
- public DdlPlan(DdlCommand cmd) {
- this.cmd = cmd;
- }
-
- /** */
- public DdlCommand command() {
- return cmd;
- }
-
- /** {@inheritDoc} */
- @Override public Type type() {
- return Type.DDL;
- }
-
- /** {@inheritDoc} */
- @Override public QueryPlan copy() {
- return this;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DummyPlanCache.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DummyPlanCache.java
deleted file mode 100644
index f07658e..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DummyPlanCache.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-
-public class DummyPlanCache implements QueryPlanCache {
- /** {@inheritDoc} */
- @Override public List<QueryPlan> queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory) {
- return factory.create(ctx);
- }
-
- /** {@inheritDoc} */
- @Override public void clear() {
-
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ExplainPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ExplainPlan.java
deleted file mode 100644
index 50194c0..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ExplainPlan.java
+++ /dev/null
@@ -1,58 +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;
-
-/**
- * Query explain plan.
- */
-public class ExplainPlan implements QueryPlan {
- /** Column name. */
- public static final String PLAN_COL_NAME = "PLAN";
-
- /** */
- private final FieldsMetadata fieldsMeta;
-
- /** */
- private final String plan;
-
- /** */
- public ExplainPlan(String plan, FieldsMetadata fieldsMeta) {
- this.fieldsMeta = fieldsMeta;
- this.plan = plan;
- }
-
- /** {@inheritDoc} */
- @Override public Type type() {
- return Type.EXPLAIN;
- }
-
- /** {@inheritDoc} */
- @Override public QueryPlan copy() {
- return this;
- }
-
- /** */
- public FieldsMetadata fieldsMeta() {
- return fieldsMeta;
- }
-
- /** */
- public String plan() {
- return plan;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
deleted file mode 100644
index 82b7fe6..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-
-import org.apache.calcite.rel.type.RelDataType;
-
-/**
- *
- */
-public interface FieldsMetadata {
- /**
- * @return Result row type.
- */
- RelDataType rowType();
-
- /**
- * @return Result row origins (or where a field value comes from).
- */
- List<List<String>> origins();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
deleted file mode 100644
index b6c7750..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-
-import org.apache.calcite.rel.type.RelDataType;
-
-/** */
-public class FieldsMetadataImpl implements FieldsMetadata {
- /** */
- private final RelDataType rowType;
-
- /** */
- private final List<List<String>> origins;
-
- /** */
- public FieldsMetadataImpl(RelDataType rowType, List<List<String>> origins) {
- this.rowType = rowType;
- this.origins = origins;
- }
-
- /** {@inheritDoc} */
- @Override public RelDataType rowType() {
- return rowType;
- }
-
- /** {@inheritDoc} */
- @Override public List<List<String>> origins() {
- return origins;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java
index 9f85b0e..aaba73c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.query.calcite.prepare;
import java.util.List;
+import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
import java.util.function.Supplier;
@@ -149,7 +150,7 @@ public class Fragment {
}
/** */
- private FragmentMapping mapping(PlanningContext ctx, RelMetadataQuery mq, Supplier<List<String>> nodesSource) {
+ private FragmentMapping mapping(PlanningContext ctx, RelMetadataQuery mq, Supplier<List<UUID>> nodesSource) {
try {
FragmentMapping mapping = IgniteMdFragmentMapping._fragmentMapping(root, mq);
@@ -175,7 +176,7 @@ public class Fragment {
}
/** */
- @NotNull private Supplier<List<String>> nodesSource(MappingService mappingSrvc, PlanningContext ctx) {
+ @NotNull private Supplier<List<UUID>> nodesSource(MappingService mappingSrvc, PlanningContext ctx) {
return () -> mappingSrvc.executionNodes(ctx.topologyVersion(), single(), null);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentPlan.java
deleted file mode 100644
index c4485f3..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentPlan.java
+++ /dev/null
@@ -1,49 +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.plan.RelOptCluster;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-
-/** */
-public class FragmentPlan implements QueryPlan {
- /** */
- private final IgniteRel root;
-
- /** */
- public FragmentPlan(IgniteRel root) {
- RelOptCluster cluster = PlanningContext.empty().cluster();
-
- this.root = new Cloner(cluster).visit(root);
- }
-
- /** */
- public IgniteRel root() {
- return root;
- }
-
- /** {@inheritDoc} */
- @Override public Type type() {
- return Type.FRAGMENT;
- }
-
- /** {@inheritDoc} */
- @Override public QueryPlan copy() {
- return this;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java
index 7ca4c2e..8f5aec2 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java
@@ -47,7 +47,9 @@ import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceH
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
/** */
@@ -178,18 +180,28 @@ public class IgniteRelShuttle implements IgniteRelVisitor<IgniteRel> {
}
/** {@inheritDoc} */
- @Override public IgniteRel visit(IgniteSetOp rel) {
+ @Override public IgniteRel visit(IgniteRel rel) {
+ return rel.accept(this);
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteSingleMinus rel) {
return processNode(rel);
}
/** {@inheritDoc} */
- @Override public IgniteRel visit(IgniteTableFunctionScan rel) {
+ @Override public IgniteRel visit(IgniteMapMinus rel) {
return processNode(rel);
}
/** {@inheritDoc} */
- @Override public IgniteRel visit(IgniteRel rel) {
- return rel.accept(this);
+ @Override public IgniteRel visit(IgniteReduceMinus rel) {
+ return processNode(rel);
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteTableFunctionScan rel) {
+ return processNode(rel);
}
/**
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepDmlPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepDmlPlan.java
deleted file mode 100644
index de2aae7..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepDmlPlan.java
+++ /dev/null
@@ -1,40 +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;
-
-/**
- * Distributed dml plan.
- */
-public class MultiStepDmlPlan extends AbstractMultiStepPlan {
- /**
- * @param fieldsMeta Fields metadata.
- */
- public MultiStepDmlPlan(QueryTemplate queryTemplate, FieldsMetadata fieldsMeta) {
- super(queryTemplate, fieldsMeta);
- }
-
- /** {@inheritDoc} */
- @Override public Type type() {
- return Type.DML;
- }
-
- /** {@inheritDoc} */
- @Override public QueryPlan copy() {
- return new MultiStepDmlPlan(queryTemplate, fieldsMetadata);
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
index 39da9a1..6337018 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.calcite.prepare;
import java.util.List;
import java.util.Map;
+import java.util.UUID;
import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
@@ -33,11 +34,6 @@ public interface MultiStepPlan extends QueryPlan {
List<Fragment> fragments();
/**
- * @return Fields metadata.
- */
- FieldsMetadata fieldsMetadata();
-
- /**
* @param fragment Fragment.
* @return Mapping for a given fragment.
*/
@@ -47,7 +43,7 @@ public interface MultiStepPlan extends QueryPlan {
ColocationGroup target(Fragment fragment);
/** */
- Map<Long, List<String>> remotes(Fragment fragment);
+ Map<Long, List<UUID>> remotes(Fragment fragment);
/**
* Inits query fragments.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
index 866514f..9ef2cfb 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
@@ -24,7 +24,7 @@ public class MultiStepQueryPlan extends AbstractMultiStepPlan {
/**
* @param fieldsMeta Fields metadata.
*/
- public MultiStepQueryPlan(QueryTemplate queryTemplate, FieldsMetadata fieldsMeta) {
+ public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
super(queryTemplate, fieldsMeta);
}
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 f2a479d..722e8f3 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
@@ -48,9 +48,9 @@ import org.apache.ignite.internal.processors.query.calcite.rule.FilterSpoolMerge
import org.apache.ignite.internal.processors.query.calcite.rule.HashAggregateConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.LogicalScanConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.MergeJoinConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rule.MinusConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.NestedLoopJoinConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.ProjectConverterRule;
-import org.apache.ignite.internal.processors.query.calcite.rule.SetOpConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.SortAggregateConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.SortConverterRule;
import org.apache.ignite.internal.processors.query.calcite.rule.TableFunctionScanConverterRule;
@@ -144,7 +144,6 @@ public enum PlannerPhase {
CoreRules.UNION_MERGE,
CoreRules.MINUS_MERGE,
- CoreRules.INTERSECT_MERGE,
CoreRules.UNION_REMOVE,
CoreRules.JOIN_COMMUTE,
CoreRules.AGGREGATE_REMOVE,
@@ -179,10 +178,8 @@ public enum PlannerPhase {
HashAggregateConverterRule.MAP_REDUCE,
SortAggregateConverterRule.SINGLE,
SortAggregateConverterRule.MAP_REDUCE,
- SetOpConverterRule.SINGLE_MINUS,
- SetOpConverterRule.MAP_REDUCE_MINUS,
- SetOpConverterRule.SINGLE_INTERSECT,
- SetOpConverterRule.MAP_REDUCE_INTERSECT,
+ MinusConverterRule.SINGLE,
+ MinusConverterRule.MAP_REDUCE,
ProjectConverterRule.INSTANCE,
FilterConverterRule.INSTANCE,
TableModifyConverterRule.INSTANCE,
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
index fe82832..8a895a4 100644
--- 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
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.query.calcite.prepare;
import java.util.Properties;
+import java.util.UUID;
import java.util.function.Function;
import org.apache.calcite.config.CalciteConnectionConfig;
@@ -55,10 +56,10 @@ public final class PlanningContext implements Context {
private final Context parentCtx;
/** */
- private final String locNodeId;
+ private final UUID locNodeId;
/** */
- private final String originatingNodeId;
+ private final UUID originatingNodeId;
/** */
private final String qry;
@@ -90,8 +91,8 @@ public final class PlanningContext implements Context {
private PlanningContext(
FrameworkConfig cfg,
Context parentCtx,
- String locNodeId,
- String originatingNodeId,
+ UUID locNodeId,
+ UUID originatingNodeId,
String qry,
Object[] parameters,
long topVer
@@ -113,14 +114,14 @@ public final class PlanningContext implements Context {
/**
* @return Local node ID.
*/
- public String localNodeId() {
+ public UUID localNodeId() {
return locNodeId;
}
/**
* @return Originating node ID (the node, who started the execution).
*/
- public String originatingNodeId() {
+ public UUID originatingNodeId() {
return originatingNodeId == null ? locNodeId : originatingNodeId;
}
@@ -304,10 +305,10 @@ public final class PlanningContext implements Context {
private Context parentCtx = Contexts.empty();
/** */
- private String locNodeId;
+ private UUID locNodeId;
/** */
- private String originatingNodeId;
+ private UUID originatingNodeId;
/** */
private String qry;
@@ -322,7 +323,7 @@ public final class PlanningContext implements Context {
* @param locNodeId Local node ID.
* @return Builder for chaining.
*/
- public Builder localNodeId(@NotNull String locNodeId) {
+ public Builder localNodeId(@NotNull UUID locNodeId) {
this.locNodeId = locNodeId;
return this;
}
@@ -331,7 +332,7 @@ public final class PlanningContext implements Context {
* @param originatingNodeId Originating node ID (the node, who started the execution).
* @return Builder for chaining.
*/
- public Builder originatingNodeId(@NotNull String originatingNodeId) {
+ public Builder originatingNodeId(@NotNull UUID originatingNodeId) {
this.originatingNodeId = originatingNodeId;
return this;
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
deleted file mode 100644
index 07f94f8..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-
-/**
- *
- */
-public interface QueryPlanCache {
- /**
- * @param ctx Context.
- * @param key Cache key.
- * @param factory Factory method to generate a plan on cache miss.
- * @return Query plan.
- */
- List<QueryPlan> queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory);
-
- /**
- * Clear cache.
- */
- void clear();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
deleted file mode 100644
index f3391d3..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.prepare;
-
-import java.util.List;
-
-/**
- *
- */
-public interface QueryPlanFactory {
- /**
- * @param ctx Planning context.
- * @return Query plan.
- */
- List<QueryPlan> create(PlanningContext ctx);
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
index 444f254..95e5f2f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
@@ -23,7 +23,9 @@ import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceH
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
/**
* A visitor to traverse an Ignite relational nodes tree.
@@ -157,7 +159,17 @@ public interface IgniteRelVisitor<T> {
/**
* See {@link IgniteRelVisitor#visit(IgniteRel)}
*/
- T visit(IgniteSetOp rel);
+ T visit(IgniteSingleMinus rel);
+
+ /**
+ * See {@link IgniteRelVisitor#visit(IgniteRel)}
+ */
+ T visit(IgniteMapMinus rel);
+
+ /**
+ * See {@link IgniteRelVisitor#visit(IgniteRel)}
+ */
+ T visit(IgniteReduceMinus rel);
/**
* See {@link IgniteRelVisitor#visit(IgniteRel)}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteIntersect.java
deleted file mode 100644
index 25e5f90..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteIntersect.java
+++ /dev/null
@@ -1,68 +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.rel.set;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Intersect;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-
-/**
- * Base class for physical INTERSECT set op.
- */
-public abstract class IgniteIntersect extends Intersect implements IgniteSetOp {
- /** */
- IgniteIntersect(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
- super(cluster, traits, inputs, all);
- }
-
- /** {@inheritDoc} */
- protected IgniteIntersect(RelInput input) {
- super(TraitUtils.changeTraits(input, IgniteConvention.INSTANCE));
- }
-
- /** {@inheritDoc} */
- @Override public double estimateRowCount(RelMetadataQuery mq) {
- final List<RelNode> inputs = getInputs();
-
- double rows = mq.getRowCount(inputs.get(0));
-
- for (int i = 1; i < inputs.size(); i++)
- rows = 0.5 * Math.min(rows, mq.getRowCount(inputs.get(i)));
-
- return rows;
- }
-
- /** {@inheritDoc} */
- @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
- return computeSetOpCost(planner, mq);
- }
-
- /** {@inheritDoc} */
- @Override public boolean all() {
- return all;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapIntersect.java
deleted file mode 100644
index 0364f79..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapIntersect.java
+++ /dev/null
@@ -1,74 +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.rel.set;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Physical node for MAP phase of INTERSECT operator.
- */
-public class IgniteMapIntersect extends IgniteIntersect implements IgniteMapSetOp {
- /** */
- public IgniteMapIntersect(
- RelOptCluster cluster,
- RelTraitSet traitSet,
- List<RelNode> inputs,
- boolean all
- ) {
- super(cluster, traitSet, inputs, all);
- }
-
- /** */
- public IgniteMapIntersect(RelInput input) {
- super(input);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteMapIntersect copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
- return new IgniteMapIntersect(getCluster(), traitSet, inputs, all);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteRel clone(RelOptCluster cluster, List<IgniteRel> inputs) {
- return new IgniteMapIntersect(cluster, getTraitSet(), Commons.cast(inputs), all);
- }
-
- /** {@inheritDoc} */
- @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
- return visitor.visit(this);
- }
-
- /** {@inheritDoc} */
- @Override protected RelDataType deriveRowType() {
- return buildRowType();
- }
-
- /** {@inheritDoc} */
- @Override public int aggregateFieldsCount() {
- return getInput(0).getRowType().getFieldCount() + getInputs().size();
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
index a26d16d..9fb6aac 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
@@ -18,20 +18,31 @@
package org.apache.ignite.internal.processors.query.calcite.rel.set;
import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import com.google.common.collect.ImmutableList;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelInput;
import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.util.Pair;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
/**
* Physical node for MAP phase of MINUS (EXCEPT) operator.
*/
-public class IgniteMapMinus extends IgniteMinus implements IgniteMapSetOp {
+public class IgniteMapMinus extends IgniteMinusBase {
/** */
public IgniteMapMinus(
RelOptCluster cluster,
@@ -64,11 +75,67 @@ public class IgniteMapMinus extends IgniteMinus implements IgniteMapSetOp {
/** {@inheritDoc} */
@Override protected RelDataType deriveRowType() {
- return buildRowType();
+ RelDataTypeFactory typeFactory = Commons.typeFactory(getCluster());
+
+ assert typeFactory instanceof IgniteTypeFactory;
+
+ RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
+
+ builder.add("GROUP_KEY", typeFactory.createJavaType(Object.class/*GroupKey.class*/));
+ builder.add("COUNTERS", typeFactory.createJavaType(int[].class));
+
+ return builder.build();
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inputTraits
+ ) {
+ boolean rewindable = inputTraits.stream()
+ .map(TraitUtils::rewindability)
+ .allMatch(RewindabilityTrait::rewindable);
+
+ if (rewindable)
+ return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
+
+ return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
+ Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inputTraits
+ ) {
+ if (inputTraits.stream().allMatch(t -> TraitUtils.distribution(t).satisfies(IgniteDistributions.single())))
+ return ImmutableList.of(); // If all distributions are single or broadcast IgniteSingleMinus should be used.
+
+ if (inputTraits.stream().anyMatch(t -> TraitUtils.distribution(t) == IgniteDistributions.single()))
+ return ImmutableList.of(); // Mixing of single and random is prohibited.
+
+ return ImmutableList.of(
+ Pair.of(nodeTraits.replace(IgniteDistributions.random()), Commons.transform(inputTraits,
+ t -> t.replace(IgniteDistributions.random())))
+ );
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inTraits
+ ) {
+ Set<CorrelationId> correlationIds = inTraits.stream()
+ .map(TraitUtils::correlation)
+ .flatMap(corrTr -> corrTr.correlationIds().stream())
+ .collect(Collectors.toSet());
+
+ return ImmutableList.of(Pair.of(nodeTraits.replace(CorrelationTrait.correlations(correlationIds)),
+ inTraits));
}
/** {@inheritDoc} */
- @Override public int aggregateFieldsCount() {
+ @Override protected int aggregateFieldsCount() {
return getInput(0).getRowType().getFieldCount() + COUNTER_FIELDS_CNT;
}
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapSetOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapSetOp.java
deleted file mode 100644
index 22502ad..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapSetOp.java
+++ /dev/null
@@ -1,108 +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.rel.set;
-
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey;
-import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Physical node for MAP phase of set op (MINUS, INTERSECT).
- */
-public interface IgniteMapSetOp extends IgniteSetOp {
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits
- ) {
- boolean rewindable = inputTraits.stream()
- .map(TraitUtils::rewindability)
- .allMatch(RewindabilityTrait::rewindable);
-
- if (rewindable)
- return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
-
- return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
- Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits
- ) {
- if (inputTraits.stream().allMatch(t -> TraitUtils.distribution(t).satisfies(IgniteDistributions.single())))
- return ImmutableList.of(); // If all distributions are single or broadcast IgniteSingleMinus should be used.
-
- return ImmutableList.of(
- Pair.of(nodeTraits.replace(IgniteDistributions.random()), Commons.transform(inputTraits,
- t -> TraitUtils.distribution(t) == IgniteDistributions.broadcast() ?
- // Allow broadcast with trim-exchange to be used in map-reduce set-op.
- t.replace(IgniteDistributions.hash(ImmutableList.of(0))) :
- t.replace(IgniteDistributions.random())))
- );
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inTraits
- ) {
- Set<CorrelationId> correlationIds = inTraits.stream()
- .map(TraitUtils::correlation)
- .flatMap(corrTr -> corrTr.correlationIds().stream())
- .collect(Collectors.toSet());
-
- return ImmutableList.of(Pair.of(nodeTraits.replace(CorrelationTrait.correlations(correlationIds)),
- inTraits));
- }
-
- /** Build RowType for MAP node. */
- public default RelDataType buildRowType() {
- RelDataTypeFactory typeFactory = Commons.typeFactory(getCluster());
-
- assert typeFactory instanceof IgniteTypeFactory;
-
- RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
-
- builder.add("GROUP_KEY", typeFactory.createJavaType(GroupKey.class));
- builder.add("COUNTERS", typeFactory.createJavaType(int[].class));
-
- return builder.build();
- }
-
- /** {@inheritDoc} */
- @Override public default AggregateType aggregateType() {
- return AggregateType.MAP;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMinus.java
deleted file mode 100644
index 38b98ab..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMinus.java
+++ /dev/null
@@ -1,71 +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.rel.set;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Minus;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-
-/**
- * Base class for physical MINUS (EXCEPT) set op.
- */
-public abstract class IgniteMinus extends Minus implements IgniteSetOp {
- /** Count of counter fields used to aggregate results. */
- protected static final int COUNTER_FIELDS_CNT = 2;
-
- /** */
- IgniteMinus(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
- super(cluster, traits, inputs, all);
- }
-
- /** {@inheritDoc} */
- protected IgniteMinus(RelInput input) {
- super(TraitUtils.changeTraits(input, IgniteConvention.INSTANCE));
- }
-
- /** {@inheritDoc} */
- @Override public double estimateRowCount(RelMetadataQuery mq) {
- final List<RelNode> inputs = getInputs();
-
- double rows = mq.getRowCount(inputs.get(0));
-
- for (int i = 1; i < inputs.size(); i++)
- rows -= 0.5 * Math.min(rows, mq.getRowCount(inputs.get(i)));
-
- return rows;
- }
-
- /** {@inheritDoc} */
- @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
- return computeSetOpCost(planner, mq);
- }
-
- /** {@inheritDoc} */
- @Override public boolean all() {
- return all;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceIntersect.java
deleted file mode 100644
index 18c4e49..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceIntersect.java
+++ /dev/null
@@ -1,90 +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.rel.set;
-
-import java.util.List;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.SetOp;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlExplainLevel;
-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.IgniteRelVisitor;
-
-/**
- * Physical node for REDUCE phase of INTERSECT operator.
- */
-public class IgniteReduceIntersect extends IgniteIntersect implements IgniteReduceSetOp {
- /** */
- public IgniteReduceIntersect(
- RelOptCluster cluster,
- RelTraitSet traitSet,
- RelNode input,
- boolean all,
- RelDataType rowType
- ) {
- super(cluster, traitSet, ImmutableList.of(input), all);
-
- this.rowType = rowType;
- }
-
- /** */
- public IgniteReduceIntersect(RelInput input) {
- this(
- input.getCluster(),
- input.getTraitSet().replace(IgniteConvention.INSTANCE),
- input.getInput(),
- input.getBoolean("all", false),
- input.getRowType("rowType")
- );
- }
-
- /** {@inheritDoc} */
- @Override public RelWriter explainTerms(RelWriter pw) {
- super.explainTerms(pw)
- .itemIf("rowType", rowType, pw.getDetailLevel() == SqlExplainLevel.ALL_ATTRIBUTES);
-
- return pw;
- }
-
- /** {@inheritDoc} */
- @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
- return new IgniteReduceIntersect(getCluster(), traitSet, sole(inputs), all, rowType);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteReduceIntersect clone(RelOptCluster cluster, List<IgniteRel> inputs) {
- return new IgniteReduceIntersect(cluster, getTraitSet(), sole(inputs), all, rowType);
- }
-
- /** {@inheritDoc} */
- @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
- return visitor.visit(this);
- }
-
- /** {@inheritDoc} */
- @Override public int aggregateFieldsCount() {
- return rowType.getFieldCount() + 2 /* At least two fields required for count aggregation. */;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
index f86646f..c068e8e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.processors.query.calcite.rel.set;
import java.util.List;
-
import com.google.common.collect.ImmutableList;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
@@ -28,14 +27,20 @@ import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.core.SetOp;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
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.IgniteRelVisitor;
+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.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
/**
* Physical node for REDUCE phase of MINUS (EXCEPT) operator.
*/
-public class IgniteReduceMinus extends IgniteMinus implements IgniteReduceSetOp {
+public class IgniteReduceMinus extends IgniteMinusBase {
/** */
public IgniteReduceMinus(
RelOptCluster cluster,
@@ -69,6 +74,46 @@ public class IgniteReduceMinus extends IgniteMinus implements IgniteReduceSetOp
}
/** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inputTraits
+ ) {
+ return ImmutableList.of(
+ Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY), ImmutableList.of(inputTraits.get(0))));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
+ List<RelTraitSet> inTraits) {
+ IgniteDistribution distr = TraitUtils.distribution(nodeTraits);
+
+ if (IgniteDistributions.single().satisfies(distr)) {
+ return Pair.of(nodeTraits.replace(IgniteDistributions.single()),
+ Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
+ }
+
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inputTraits
+ ) {
+ return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()),
+ ImmutableList.of(sole(inputTraits).replace(IgniteDistributions.single()))));
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inTraits
+ ) {
+ return ImmutableList.of(Pair.of(nodeTraits.replace(TraitUtils.correlation(inTraits.get(0))),
+ inTraits));
+ }
+
+ /** {@inheritDoc} */
@Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
return new IgniteReduceMinus(getCluster(), traitSet, sole(inputs), all, rowType);
}
@@ -84,7 +129,7 @@ public class IgniteReduceMinus extends IgniteMinus implements IgniteReduceSetOp
}
/** {@inheritDoc} */
- @Override public int aggregateFieldsCount() {
+ @Override protected int aggregateFieldsCount() {
return rowType.getFieldCount() + COUNTER_FIELDS_CNT;
}
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceSetOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceSetOp.java
deleted file mode 100644
index 06669c2..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceSetOp.java
+++ /dev/null
@@ -1,75 +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.rel.set;
-
-import java.util.List;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Physical node for REDUCE phase of set op (MINUS, INTERSECT).
- */
-public interface IgniteReduceSetOp extends IgniteSetOp {
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits
- ) {
- return ImmutableList.of(
- Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY), ImmutableList.of(inputTraits.get(0))));
- }
-
- /** {@inheritDoc} */
- @Override public default Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
- List<RelTraitSet> inTraits) {
- if (TraitUtils.distribution(nodeTraits) == IgniteDistributions.single())
- return Pair.of(nodeTraits, Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
-
- return null;
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits
- ) {
- return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()),
- ImmutableList.of(inputTraits.get(0).replace(IgniteDistributions.single()))));
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inTraits
- ) {
- return ImmutableList.of(Pair.of(nodeTraits.replace(TraitUtils.correlation(inTraits.get(0))),
- inTraits));
- }
-
- /** {@inheritDoc} */
- @Override public default AggregateType aggregateType() {
- return AggregateType.REDUCE;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSetOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSetOp.java
deleted file mode 100644
index 127aa09..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSetOp.java
+++ /dev/null
@@ -1,78 +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.rel.set;
-
-import java.util.List;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
-import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCost;
-import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCostFactory;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitsAwareIgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Base interface for physical set op node (MINUS, INTERSECT).
- */
-public interface IgniteSetOp extends TraitsAwareIgniteRel {
- /** ALL flag of set op. */
- public boolean all();
-
- /** {@inheritDoc} */
- @Override public default Pair<RelTraitSet, List<RelTraitSet>> passThroughCollation(RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits) {
- // Operation erases collation.
- return Pair.of(nodeTraits.replace(RelCollations.EMPTY),
- Commons.transform(inputTraits, t -> t.replace(RelCollations.EMPTY)));
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits) {
- // Operation erases collation.
- return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
- Commons.transform(inputTraits, t -> t.replace(RelCollations.EMPTY))));
- }
-
- /** Gets count of fields for aggregation for this node. Required for memory consumption calculation. */
- public int aggregateFieldsCount();
-
- /** Compute cost for set op. */
- public default RelOptCost computeSetOpCost(RelOptPlanner planner, RelMetadataQuery mq) {
- IgniteCostFactory costFactory = (IgniteCostFactory)planner.getCostFactory();
-
- double inputRows = 0;
-
- for (RelNode input : getInputs())
- inputRows += mq.getRowCount(input);
-
- double mem = 0.5 * inputRows * aggregateFieldsCount() * IgniteCost.AVERAGE_FIELD_SIZE;
-
- return costFactory.makeCost(inputRows, inputRows * IgniteCost.ROW_PASS_THROUGH_COST, 0, mem, 0);
- }
-
- /** Aggregate type. */
- public AggregateType aggregateType();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleIntersect.java
deleted file mode 100644
index 65f129b..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleIntersect.java
+++ /dev/null
@@ -1,68 +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.rel.set;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Physical node for INTERSECT operator which inputs satisfy SINGLE distribution.
- */
-public class IgniteSingleIntersect extends IgniteIntersect implements IgniteSingleSetOp {
- /** {@inheritDoc} */
- public IgniteSingleIntersect(
- RelOptCluster cluster,
- RelTraitSet traitSet,
- List<RelNode> inputs,
- boolean all
- ) {
- super(cluster, traitSet, inputs, all);
- }
-
- /** */
- public IgniteSingleIntersect(RelInput input) {
- super(input);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteSingleIntersect copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
- return new IgniteSingleIntersect(getCluster(), traitSet, inputs, all);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteRel clone(RelOptCluster cluster, List<IgniteRel> inputs) {
- return new IgniteSingleIntersect(cluster, getTraitSet(), Commons.cast(inputs), all);
- }
-
- /** {@inheritDoc} */
- @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
- return visitor.visit(this);
- }
-
- /** {@inheritDoc} */
- @Override public int aggregateFieldsCount() {
- return getInput(0).getRowType().getFieldCount() + getInputs().size();
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
index 71a1d35..3165bb5 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
@@ -18,19 +18,28 @@
package org.apache.ignite.internal.processors.query.calcite.rel.set;
import java.util.List;
-
+import java.util.Set;
+import java.util.stream.Collectors;
+import com.google.common.collect.ImmutableList;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelInput;
import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.util.Pair;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTrait;
+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.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
/**
* Physical node for MINUS (EXCEPT) operator which inputs satisfy SINGLE distribution.
*/
-public class IgniteSingleMinus extends IgniteMinus implements IgniteSingleSetOp {
+public class IgniteSingleMinus extends IgniteMinusBase {
/** {@inheritDoc} */
public IgniteSingleMinus(
RelOptCluster cluster,
@@ -47,6 +56,63 @@ public class IgniteSingleMinus extends IgniteMinus implements IgniteSingleSetOp
}
/** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inputTraits
+ ) {
+ boolean rewindable = inputTraits.stream()
+ .map(TraitUtils::rewindability)
+ .allMatch(RewindabilityTrait::rewindable);
+
+ if (rewindable)
+ return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
+
+ return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
+ Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inTraits) {
+ IgniteDistribution distr = TraitUtils.distribution(nodeTraits);
+
+ if (IgniteDistributions.single().satisfies(distr)) {
+ return Pair.of(nodeTraits.replace(IgniteDistributions.single()),
+ Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
+ }
+
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inputTraits
+ ) {
+ boolean single = inputTraits.stream()
+ .map(TraitUtils::distribution)
+ .allMatch(d -> d.satisfies(IgniteDistributions.single()));
+
+ if (!single)
+ return ImmutableList.of();
+
+ return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()), inputTraits));
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+ RelTraitSet nodeTraits,
+ List<RelTraitSet> inTraits
+ ) {
+ Set<CorrelationId> correlationIds = inTraits.stream()
+ .map(TraitUtils::correlation)
+ .flatMap(corrTr -> corrTr.correlationIds().stream())
+ .collect(Collectors.toSet());
+
+ return ImmutableList.of(Pair.of(nodeTraits.replace(CorrelationTrait.correlations(correlationIds)),
+ inTraits));
+ }
+
+ /** {@inheritDoc} */
@Override public IgniteSingleMinus copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
return new IgniteSingleMinus(getCluster(), traitSet, inputs, all);
}
@@ -62,7 +128,7 @@ public class IgniteSingleMinus extends IgniteMinus implements IgniteSingleSetOp
}
/** {@inheritDoc} */
- @Override public int aggregateFieldsCount() {
+ @Override protected int aggregateFieldsCount() {
return getInput(0).getRowType().getFieldCount() + COUNTER_FIELDS_CNT;
}
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleSetOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleSetOp.java
deleted file mode 100644
index 3dd5249..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleSetOp.java
+++ /dev/null
@@ -1,97 +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.rel.set;
-
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
-import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Physical node for set op (MINUS, INTERSECT) operator which inputs satisfy SINGLE distribution.
- */
-public interface IgniteSingleSetOp extends IgniteSetOp {
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits
- ) {
- boolean rewindable = inputTraits.stream()
- .map(TraitUtils::rewindability)
- .allMatch(RewindabilityTrait::rewindable);
-
- if (rewindable)
- return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
-
- return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
- Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
- }
-
- /** {@inheritDoc} */
- @Override public default Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
- List<RelTraitSet> inTraits) {
- if (TraitUtils.distribution(nodeTraits) == IgniteDistributions.single())
- return Pair.of(nodeTraits, Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
-
- return null;
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inputTraits
- ) {
- boolean single = inputTraits.stream()
- .map(TraitUtils::distribution)
- .allMatch(d -> d.satisfies(IgniteDistributions.single()));
-
- if (!single)
- return ImmutableList.of();
-
- return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()), inputTraits));
- }
-
- /** {@inheritDoc} */
- @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
- RelTraitSet nodeTraits,
- List<RelTraitSet> inTraits
- ) {
- Set<CorrelationId> correlationIds = inTraits.stream()
- .map(TraitUtils::correlation)
- .flatMap(corrTr -> corrTr.correlationIds().stream())
- .collect(Collectors.toSet());
-
- return ImmutableList.of(Pair.of(nodeTraits.replace(CorrelationTrait.correlations(correlationIds)),
- inTraits));
- }
-
- /** {@inheritDoc} */
- @Override public default AggregateType aggregateType() {
- return AggregateType.SINGLE;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SetOpConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SetOpConverterRule.java
deleted file mode 100644
index c2b0728..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SetOpConverterRule.java
+++ /dev/null
@@ -1,186 +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.rule;
-
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.PhysicalNode;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.SetOp;
-import org.apache.calcite.rel.logical.LogicalIntersect;
-import org.apache.calcite.rel.logical.LogicalMinus;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.util.Util;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapIntersect;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceIntersect;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleIntersect;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-
-/**
- * Set op (MINUS, INTERSECT) converter rule.
- */
-public class SetOpConverterRule {
- /** */
- public static final RelOptRule SINGLE_MINUS = new SingleMinusConverterRule();
-
- /** */
- public static final RelOptRule SINGLE_INTERSECT = new SingleIntersectConverterRule();
-
- /** */
- public static final RelOptRule MAP_REDUCE_MINUS = new MapReduceMinusConverterRule();
-
- /** */
- public static final RelOptRule MAP_REDUCE_INTERSECT = new MapReduceIntersectConverterRule();
-
- /** */
- private SetOpConverterRule() {
- // No-op.
- }
-
- /** */
- private abstract static class SingleSetOpConverterRule<T extends SetOp> extends AbstractIgniteConverterRule<T> {
- /** */
- SingleSetOpConverterRule(Class<T> cls, String desc) {
- super(cls, desc);
- }
-
- /** Node factory method. */
- abstract PhysicalNode createNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all);
-
- /** {@inheritDoc} */
- @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, T setOp) {
- RelOptCluster cluster = setOp.getCluster();
- RelTraitSet inTrait = cluster.traitSetOf(IgniteConvention.INSTANCE).replace(IgniteDistributions.single());
- RelTraitSet outTrait = cluster.traitSetOf(IgniteConvention.INSTANCE).replace(IgniteDistributions.single());
- List<RelNode> inputs = Util.transform(setOp.getInputs(), rel -> convert(rel, inTrait));
-
- return createNode(cluster, outTrait, inputs, setOp.all);
- }
- }
-
- /** */
- private static class SingleMinusConverterRule extends SingleSetOpConverterRule<LogicalMinus> {
- /** */
- SingleMinusConverterRule() {
- super(LogicalMinus.class, "SingleMinusConverterRule");
- }
-
- /** {@inheritDoc} */
- @Override PhysicalNode createNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
- boolean all) {
- return new IgniteSingleMinus(cluster, traits, inputs, all);
- }
- }
-
- /** */
- private static class SingleIntersectConverterRule extends SingleSetOpConverterRule<LogicalIntersect> {
- /** */
- SingleIntersectConverterRule() {
- super(LogicalIntersect.class, "SingleIntersectConverterRule");
- }
-
- /** {@inheritDoc} */
- @Override PhysicalNode createNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
- boolean all) {
- return new IgniteSingleIntersect(cluster, traits, inputs, all);
- }
- }
-
- /** */
- private abstract static class MapReduceSetOpConverterRule<T extends SetOp> extends AbstractIgniteConverterRule<T> {
- /** */
- MapReduceSetOpConverterRule(Class<T> cls, String desc) {
- super(cls, desc);
- }
-
- /** Map node factory method. */
- abstract PhysicalNode createMapNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
- boolean all);
-
- /** Reduce node factory method. */
- abstract PhysicalNode createReduceNode(RelOptCluster cluster, RelTraitSet traits, RelNode input,
- boolean all, RelDataType rowType);
-
- /** {@inheritDoc} */
- @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, T setOp) {
- RelOptCluster cluster = setOp.getCluster();
- RelTraitSet inTrait = cluster.traitSetOf(IgniteConvention.INSTANCE);
- RelTraitSet outTrait = cluster.traitSetOf(IgniteConvention.INSTANCE);
- List<RelNode> inputs = Util.transform(setOp.getInputs(), rel -> convert(rel, inTrait));
-
- RelNode map = createMapNode(cluster, outTrait, inputs, setOp.all);
-
- return createReduceNode(
- cluster,
- outTrait.replace(IgniteDistributions.single()),
- convert(map, inTrait.replace(IgniteDistributions.single())),
- setOp.all,
- cluster.getTypeFactory().leastRestrictive(Util.transform(inputs, RelNode::getRowType))
- );
- }
- }
-
- /** */
- private static class MapReduceMinusConverterRule extends MapReduceSetOpConverterRule<LogicalMinus> {
- /** */
- MapReduceMinusConverterRule() {
- super(LogicalMinus.class, "MapReduceMinusConverterRule");
- }
-
- /** {@inheritDoc} */
- @Override PhysicalNode createMapNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
- boolean all) {
- return new IgniteMapMinus(cluster, traits, inputs, all);
- }
-
- /** {@inheritDoc} */
- @Override PhysicalNode createReduceNode(RelOptCluster cluster, RelTraitSet traits, RelNode input, boolean all,
- RelDataType rowType) {
- return new IgniteReduceMinus(cluster, traits, input, all, rowType);
- }
- }
-
- /** */
- private static class MapReduceIntersectConverterRule extends MapReduceSetOpConverterRule<LogicalIntersect> {
- /** */
- MapReduceIntersectConverterRule() {
- super(LogicalIntersect.class, "MapReduceIntersectConverterRule");
- }
-
- /** {@inheritDoc} */
- @Override PhysicalNode createMapNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
- boolean all) {
- return new IgniteMapIntersect(cluster, traits, inputs, all);
- }
-
- /** {@inheritDoc} */
- @Override PhysicalNode createReduceNode(RelOptCluster cluster, RelTraitSet traits, RelNode input, boolean all,
- RelDataType rowType) {
- return new IgniteReduceIntersect(cluster, traits, input, all, rowType);
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java
index c0aba90..5d6ef72 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java
@@ -22,6 +22,9 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactor
/** */
public interface ColumnDescriptor {
/** */
+ boolean field();
+
+ /** */
boolean key();
/** */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptorImpl.java
deleted file mode 100644
index 74cd5c1..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptorImpl.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.schema;
-
-import java.util.function.Supplier;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.jetbrains.annotations.Nullable;
-
-/** */
-public class ColumnDescriptorImpl implements ColumnDescriptor {
- private final boolean key;
-
- private final String name;
-
- /** */
- private final @Nullable Supplier<Object> dfltVal;
-
- /** */
- private final int fieldIdx;
-
- /** */
- private final Class<?> storageType;
-
- public ColumnDescriptorImpl(
- String name,
- boolean key,
- int fieldIdx,
- Class<?> storageType,
- @Nullable Supplier<Object> dfltVal
- ) {
- this.key = key;
- this.name = name;
- this.dfltVal = dfltVal;
- this.fieldIdx = fieldIdx;
- this.storageType = storageType;
- }
-
- /** {@inheritDoc} */
- @Override public boolean key() {
- return key;
- }
-
- /** {@inheritDoc} */
- @Override public boolean hasDefaultValue() {
- return dfltVal != null;
- }
-
- /** {@inheritDoc} */
- @Override public Object defaultValue() {
- return dfltVal != null ? dfltVal.get() : null;
- }
-
- /** {@inheritDoc} */
- @Override public String name() {
- return name;
- }
-
- /** {@inheritDoc} */
- @Override public int fieldIndex() {
- return fieldIdx;
- }
-
- /** {@inheritDoc} */
- @Override public RelDataType logicalType(IgniteTypeFactory f) {
- return f.createJavaType(storageType);
- }
-
- /** {@inheritDoc} */
- @Override public Class<?> storageType() {
- return storageType;
- }
-
- /** {@inheritDoc} */
- @Override public void set(Object dst, Object val) {
- throw new AssertionError();
- }
-}
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 14a0bc6..9554cb4 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
@@ -17,8 +17,6 @@
package org.apache.ignite.internal.processors.query.calcite.schema;
import java.util.Map;
-import java.util.function.Function;
-import java.util.function.Predicate;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptTable;
@@ -27,13 +25,11 @@ import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.schema.TranslatableTable;
import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalIndexScan;
import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan;
import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.jetbrains.annotations.Nullable;
/**
* Ignite table.
@@ -82,23 +78,6 @@ public interface IgniteTable extends TranslatableTable {
IgniteLogicalIndexScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName);
/**
- * Creates rows iterator over the table.
- *
- * @param execCtx Execution context.
- * @param group Colocation group.
- * @param filter Row filter.
- * @param rowTransformer Row transformer.
- * @param usedColumns Used columns enumeration.
- * @return Rows iterator.
- */
- <Row> Iterable<Row> scan(
- ExecutionContext<Row> execCtx,
- ColocationGroup group,
- Predicate<Row> filter,
- Function<Row, Row> rowTransformer,
- @Nullable ImmutableBitSet usedColumns);
-
- /**
* Returns nodes mapping.
*
* @param ctx Planning context.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
deleted file mode 100644
index f70ac14..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
+++ /dev/null
@@ -1,207 +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.schema;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.function.Function;
-import java.util.function.Predicate;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelReferentialConstraint;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.Statistic;
-import org.apache.calcite.schema.impl.AbstractTable;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalIndexScan;
-import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.internal.processors.query.calcite.Stubs.resultSetGenerator;
-
-/**
- * Ignite table implementation.
- */
-public class IgniteTableImpl extends AbstractTable implements IgniteTable {
- /** */
- private final TableDescriptor desc;
-
- /** */
- private final Statistic statistic;
-
- /** */
- private volatile Collection<Object[]> rows;
-
- /** */
- private final Map<String, IgniteIndex> indexes = new ConcurrentHashMap<>();
-
- /**
- * @param desc Table descriptor.
- */
- public IgniteTableImpl(TableDescriptor desc) {
- this.desc = desc;
- statistic = new StatisticsImpl();
- }
-
- /** {@inheritDoc} */
- @Override public RelDataType getRowType(RelDataTypeFactory typeFactory, ImmutableBitSet requiredColumns) {
- return desc.rowType((IgniteTypeFactory)typeFactory, requiredColumns);
- }
-
- /** {@inheritDoc} */
- @Override public Statistic getStatistic() {
- return statistic;
- }
-
-
- /** {@inheritDoc} */
- @Override public TableDescriptor descriptor() {
- return desc;
- }
-
- /** {@inheritDoc} */
- @Override public IgniteLogicalTableScan toRel(RelOptCluster cluster, RelOptTable relOptTbl) {
- RelTraitSet traitSet = cluster.traitSetOf(distribution())
- .replace(RewindabilityTrait.REWINDABLE);
-
- return IgniteLogicalTableScan.create(cluster, traitSet, relOptTbl, null, null, null);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteLogicalIndexScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName) {
- RelTraitSet traitSet = cluster.traitSetOf(Convention.Impl.NONE)
- .replace(distribution())
- .replace(RewindabilityTrait.REWINDABLE)
- .replace(getIndex(idxName).collation());
-
- return IgniteLogicalIndexScan.create(cluster, traitSet, relOptTbl, idxName, null, null, null);
- }
-
- /** {@inheritDoc} */
- @Override public <Row> Iterable<Row> scan(
- ExecutionContext<Row> execCtx,
- ColocationGroup group,
- Predicate<Row> filter,
- Function<Row, Row> rowTransformer,
- @Nullable ImmutableBitSet usedColumns
- ) {
- String locNodeId = execCtx.planningContext().localNodeId();
-
- if (group.nodeIds().contains(locNodeId))
- return (Iterable<Row>) rows(execCtx.getTypeFactory());
-
- return Collections.emptyList();
- }
-
- /** {@inheritDoc} */
- @Override public IgniteDistribution distribution() {
- return desc.distribution();
- }
-
- /** {@inheritDoc} */
- @Override public ColocationGroup colocationGroup(PlanningContext ctx) {
- return desc.colocationGroup(ctx);
- }
-
- /** {@inheritDoc} */
- @Override public Map<String, IgniteIndex> indexes() {
- return Collections.unmodifiableMap(indexes);
- }
-
- /** {@inheritDoc} */
- @Override public void addIndex(IgniteIndex idxTbl) {
- indexes.put(idxTbl.name(), idxTbl);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteIndex getIndex(String idxName) {
- return indexes.get(idxName);
- }
-
- /** {@inheritDoc} */
- @Override public void removeIndex(String idxName) {
- indexes.remove(idxName);
- }
-
- /** {@inheritDoc} */
- @Override public <C> C unwrap(Class<C> aCls) {
- if (aCls.isInstance(desc))
- return aCls.cast(desc);
-
- return super.unwrap(aCls);
- }
-
- private Collection<Object[]> rows(IgniteTypeFactory typeFactory) {
- if (rows == null) {
- synchronized (this) {
- if (rows == null)
- rows = resultSetGenerator(10, typeFactory, getRowType(typeFactory));
- }
- }
-
- return rows;
- }
-
- /** */
- private class StatisticsImpl implements Statistic {
- /** {@inheritDoc} */
- @Override public Double getRowCount() {
- return (double)rows(new IgniteTypeFactory()).size();
- }
-
- /** {@inheritDoc} */
- @Override public boolean isKey(ImmutableBitSet cols) {
- return false; // TODO
- }
-
- /** {@inheritDoc} */
- @Override public List<ImmutableBitSet> getKeys() {
- return null; // TODO
- }
-
- /** {@inheritDoc} */
- @Override public List<RelReferentialConstraint> getReferentialConstraints() {
- return ImmutableList.of();
- }
-
- /** {@inheritDoc} */
- @Override public List<RelCollation> getCollations() {
- return ImmutableList.of(); // The method isn't used
- }
-
- /** {@inheritDoc} */
- @Override public IgniteDistribution getDistribution() {
- return distribution();
- }
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
deleted file mode 100644
index 3368bb4..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
+++ /dev/null
@@ -1,30 +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.schema;
-
-import org.apache.calcite.schema.SchemaPlus;
-
-/**
- *
- */
-public interface SchemaHolder {
- /**
- * @return Schema.
- */
- SchemaPlus schema();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
deleted file mode 100644
index 935bb8b..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
+++ /dev/null
@@ -1,124 +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.schema;
-
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.ignite.internal.schema.Column;
-import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.network.TopologyService;
-
-import static org.apache.ignite.internal.processors.query.calcite.util.Commons.nativeTypeToClass;
-
-/**
- * Holds actual schema and mutates it on schema change, requested by Ignite.
- */
-public class SchemaHolderImpl implements SchemaHolder {
- /** */
- private final Map<String, IgniteSchema> igniteSchemas = new HashMap<>();
-
- private final TopologyService topSrvc;
-
- /** */
- private volatile SchemaPlus calciteSchema;
-
- public SchemaHolderImpl(
- TopologyService topSrvc
- ) {
- this.topSrvc = topSrvc;
-
- SchemaPlus newCalciteSchema = Frameworks.createRootSchema(false);
- newCalciteSchema.add("PUBLIC", new IgniteSchema("PUBLIC"));
- calciteSchema = newCalciteSchema;
- }
-
- /** {@inheritDoc} */
- @Override public SchemaPlus schema() {
- return calciteSchema;
- }
-
- public synchronized void onSchemaCreated(String schemaName) {
- igniteSchemas.putIfAbsent(schemaName, new IgniteSchema(schemaName));
- rebuild();
- }
-
- public synchronized void onSchemaDropped(String schemaName) {
- igniteSchemas.remove(schemaName);
- rebuild();
- }
-
- public synchronized void onSqlTypeCreated(
- String schemaName,
- String tableName,
- SchemaDescriptor descriptor
- ) {
- IgniteSchema schema = igniteSchemas.computeIfAbsent(schemaName, IgniteSchema::new);
-
- tableName = tableName.substring(schemaName.length() + 1);
-
- List<ColumnDescriptor> colDescriptors = descriptor.columnNames().stream()
- .map(descriptor::column)
- .sorted(Comparator.comparingInt(Column::schemaIndex))
- .map(col -> new ColumnDescriptorImpl(
- col.name(),
- descriptor.isKeyColumn(col.schemaIndex()),
- col.schemaIndex(),
- nativeTypeToClass(col.type()),
- col::defaultValue
- ))
- .collect(Collectors.toList());
-
- TableDescriptorImpl desc = new TableDescriptorImpl(topSrvc, colDescriptors);
-
- schema.addTable(tableName, new IgniteTableImpl(desc));
-
- rebuild();
- }
-
- public void onSqlTypeUpdated(
- String schemaName,
- String tableName,
- SchemaDescriptor descriptor
- ) {
- onSqlTypeCreated(schemaName, tableName, descriptor);
- }
-
- public synchronized void onSqlTypeDropped(
- String schemaName,
- String tableName
- ) {
- IgniteSchema schema = igniteSchemas.computeIfAbsent(schemaName, IgniteSchema::new);
-
- schema.removeTable(tableName);
-
- rebuild();
- }
-
- private void rebuild() {
- SchemaPlus newCalciteSchema = Frameworks.createRootSchema(false);
- newCalciteSchema.add("PUBLIC", new IgniteSchema("PUBLIC"));
- igniteSchemas.forEach(newCalciteSchema::add);
- calciteSchema = newCalciteSchema;
- }
-}
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
index 9f0c124..a8a16d0 100644
--- 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
@@ -23,8 +23,6 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelProtoDataType;
import org.apache.calcite.sql2rel.InitializerExpressionFactory;
import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
@@ -37,14 +35,6 @@ public interface TableDescriptor extends RelProtoDataType, InitializerExpression
*/
IgniteDistribution distribution();
- /**
- * Returns nodes mapping.
- *
- * @param ctx Planning context.
- * @return Nodes mapping.
- */
- ColocationGroup colocationGroup(PlanningContext ctx);
-
/** {@inheritDoc} */
@Override default RelDataType apply(RelDataTypeFactory factory) {
return rowType((IgniteTypeFactory)factory, null);
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
deleted file mode 100644
index 2d0761e..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
+++ /dev/null
@@ -1,149 +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.schema;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.calcite.plan.RelOptTable;
-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.calcite.schema.ColumnStrategy;
-import org.apache.calcite.sql2rel.InitializerContext;
-import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-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;
-import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.TopologyService;
-
-/**
- *
- */
-public class TableDescriptorImpl extends NullInitializerExpressionFactory implements TableDescriptor {
- /** */
- private static final ColumnDescriptor[] DUMMY = new ColumnDescriptor[0];
-
- private final TopologyService topSrvc;
-
- /** */
- private final ColumnDescriptor[] descriptors;
-
- /** */
- private final Map<String, ColumnDescriptor> descriptorsMap;
-
- /** */
- private final ImmutableBitSet insertFields;
-
- /** */
- public TableDescriptorImpl(
- TopologyService topSrvc,
- List<ColumnDescriptor> columnDescriptors
- ) {
- Map<String, ColumnDescriptor> descriptorsMap = new HashMap<>(columnDescriptors.size());
- for (ColumnDescriptor descriptor : columnDescriptors)
- descriptorsMap.put(descriptor.name(), descriptor);
-
- this.descriptors = columnDescriptors.toArray(DUMMY);
- this.descriptorsMap = descriptorsMap;
- this.topSrvc = topSrvc;
-
- insertFields = ImmutableBitSet.range(columnDescriptors.size());
- }
-
- /** {@inheritDoc} */
- @Override public RelDataType insertRowType(IgniteTypeFactory factory) {
- return rowType(factory, insertFields);
- }
-
- /** {@inheritDoc} */
- @Override public IgniteDistribution distribution() {
- return IgniteDistributions.random();
- }
-
- /** {@inheritDoc} */
- @Override public boolean isUpdateAllowed(RelOptTable tbl, int colIdx) {
- return false;
- }
-
- /** {@inheritDoc} */
- @Override public ColumnStrategy generationStrategy(RelOptTable tbl, int colIdx) {
- if (descriptors[colIdx].hasDefaultValue())
- return ColumnStrategy.DEFAULT;
-
- return super.generationStrategy(tbl, colIdx);
- }
-
- /** {@inheritDoc} */
- @Override public RexNode newColumnDefaultValue(RelOptTable tbl, int colIdx, InitializerContext ctx) {
- final ColumnDescriptor desc = descriptors[colIdx];
-
- if (!desc.hasDefaultValue())
- return super.newColumnDefaultValue(tbl, colIdx, ctx);
-
- final RexBuilder rexBuilder = ctx.getRexBuilder();
- final IgniteTypeFactory typeFactory = (IgniteTypeFactory) rexBuilder.getTypeFactory();
-
- return rexBuilder.makeLiteral(desc.defaultValue(), desc.logicalType(typeFactory), false);
- }
-
- /** {@inheritDoc} */
- @Override public RelDataType rowType(IgniteTypeFactory factory, ImmutableBitSet usedColumns) {
- RelDataTypeFactory.Builder b = new RelDataTypeFactory.Builder(factory);
-
- if (usedColumns == null) {
- for (int i = 0; i < descriptors.length; i++)
- b.add(descriptors[i].name(), descriptors[i].logicalType(factory));
- }
- else {
- for (int i = usedColumns.nextSetBit(0); i != -1; i = usedColumns.nextSetBit(i + 1))
- b.add(descriptors[i].name(), descriptors[i].logicalType(factory));
- }
-
- return TypeUtils.sqlType(factory, b.build());
- }
-
- /** {@inheritDoc} */
- @Override public ColumnDescriptor columnDescriptor(String fieldName) {
- return fieldName == null ? null : descriptorsMap.get(fieldName);
- }
-
- /** {@inheritDoc} */
- @Override public ColocationGroup colocationGroup(PlanningContext ctx) {
- return partitionedGroup();
- }
-
- /** */
- private ColocationGroup partitionedGroup() {
- List<List<String>> assignments = topSrvc.allMembers().stream()
- .map(ClusterNode::id)
- .map(Collections::singletonList)
- .collect(Collectors.toList());
-
- return ColocationGroup.forAssignments(assignments);
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityAdapter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityAdapter.java
deleted file mode 100644
index 09db5c1..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityAdapter.java
+++ /dev/null
@@ -1,51 +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.trait;
-
-import java.util.function.ToIntFunction;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
-import org.apache.ignite.internal.util.IgniteUtils;
-
-/** */
-final class AffinityAdapter<Row> implements ToIntFunction<Row> {
- /** */
- private final ToIntFunction<Object> affinity;
-
- /** */
- private final int[] keys;
-
- /** */
- private final RowHandler<Row> hndlr;
-
- /** */
- AffinityAdapter(ToIntFunction<Object> affinity, int[] keys, RowHandler<Row> hndlr) {
- this.affinity = affinity;
- this.keys = keys;
- this.hndlr = hndlr;
- }
-
- /** {@inheritDoc} */
- @Override public int applyAsInt(Row r) {
- int hash = 0;
- for (int i = 0; i < keys.length; i++)
- hash = 31 * hash + affinity.applyAsInt(hndlr.get(keys[i], r));
-
- return IgniteUtils.safeAbs(hash);
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java
deleted file mode 100644
index f6ae3c1..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java
+++ /dev/null
@@ -1,41 +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.trait;
-
-import java.util.List;
-
-/** */
-public final class AllNodes<Row> implements Destination<Row> {
- /** */
- private final List<String> nodes;
-
- /** */
- public AllNodes(List<String> nodes) {
- this.nodes = nodes;
- }
-
- /** {@inheritDoc} */
- @Override public List<String> targets(Row row) {
- return nodes;
- }
-
- /** {@inheritDoc} */
- @Override public List<String> targets() {
- return nodes;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
deleted file mode 100644
index 56d5eaa..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
+++ /dev/null
@@ -1,36 +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.trait;
-
-import java.util.List;
-
-/**
- * Determines where to send a row to.
- */
-public interface Destination<Row> {
- /**
- * @param row Row.
- * @return Target nodes list for specific row.
- */
- List<String> targets(Row row);
-
- /**
- * @return All target nodes.
- */
- List<String> targets();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
index 074682a..2586dae 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
@@ -17,21 +17,12 @@
package org.apache.ignite.internal.processors.query.calcite.trait;
-import java.util.Collections;
-import java.util.List;
import java.util.Objects;
import org.apache.calcite.rel.RelDistribution;
import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.util.IgniteUtils;
import static org.apache.ignite.internal.processors.query.calcite.Stubs.intFoo;
-import static org.apache.ignite.internal.util.CollectionUtils.first;
-import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
/**
* Distribution function.
@@ -71,18 +62,6 @@ public abstract class DistributionFunction {
}
/**
- * Creates a destination based on this function algorithm, given nodes mapping and given distribution keys.
- *
- * @param ctx Execution context.
- * @param affinityService Affinity function source.
- * @param group Target mapping.
- * @param keys Distribution keys.
- * @return Destination function.
- */
- abstract <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
- ColocationGroup group, ImmutableIntList keys);
-
- /**
* @return Function name. This name used for equality checking and in {@link RelNode#getDigest()}.
*/
protected String name0() {
@@ -156,12 +135,6 @@ public abstract class DistributionFunction {
@Override public RelDistribution.Type type() {
return RelDistribution.Type.ANY;
}
-
- /** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
- ColocationGroup m, ImmutableIntList k) {
- throw new IllegalStateException();
- }
}
/** */
@@ -173,14 +146,6 @@ public abstract class DistributionFunction {
@Override public RelDistribution.Type type() {
return RelDistribution.Type.BROADCAST_DISTRIBUTED;
}
-
- /** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
- ColocationGroup m, ImmutableIntList k) {
- assert m != null && !nullOrEmpty(m.nodeIds());
-
- return new AllNodes<>(m.nodeIds());
- }
}
/** */
@@ -192,14 +157,6 @@ public abstract class DistributionFunction {
@Override public RelDistribution.Type type() {
return RelDistribution.Type.RANDOM_DISTRIBUTED;
}
-
- /** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
- ColocationGroup m, ImmutableIntList k) {
- assert m != null && !nullOrEmpty(m.nodeIds());
-
- return new RandomNode<>(m.nodeIds());
- }
}
/** */
@@ -211,15 +168,6 @@ public abstract class DistributionFunction {
@Override public RelDistribution.Type type() {
return RelDistribution.Type.SINGLETON;
}
-
- /** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
- ColocationGroup m, ImmutableIntList k) {
- if (m == null || m.nodeIds() == null || m.nodeIds().size() != 1)
- throw new IllegalStateException();
-
- return new AllNodes<>(Collections.singletonList(Objects.requireNonNull(first(m.nodeIds()))));
- }
}
/** */
@@ -230,24 +178,6 @@ public abstract class DistributionFunction {
@Override public RelDistribution.Type type() {
return RelDistribution.Type.HASH_DISTRIBUTED;
}
-
- /** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affSrvc,
- ColocationGroup m, ImmutableIntList k) {
- assert m != null && !nullOrEmpty(m.assignments()) && !k.isEmpty();
-
- List<List<String>> assignments = m.assignments();
-
- if (IgniteUtils.assertionsEnabled()) {
- for (List<String> assignment : assignments)
- assert nullOrEmpty(assignment) || assignment.size() == 1;
- }
-
- AffinityAdapter<Row> affinity = new AffinityAdapter<>(affSrvc.affinity(intFoo()/*CU.UNDEFINED_CACHE_ID*/), k.toIntArray(),
- ctx.rowHandler());
-
- return new Partitioned<>(assignments, affinity);
- }
}
/** */
@@ -282,23 +212,6 @@ public abstract class DistributionFunction {
return RelDistribution.Type.HASH_DISTRIBUTED;
}
- /** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affSrvc,
- ColocationGroup m, ImmutableIntList k) {
- assert m != null && !nullOrEmpty(m.assignments()) && k.size() == 1;
-
- List<List<String>> assignments = m.assignments();
-
- if (IgniteUtils.assertionsEnabled()) {
- for (List<String> assignment : assignments)
- assert nullOrEmpty(assignment) || assignment.size() == 1;
- }
-
- AffinityAdapter<Row> affinity = new AffinityAdapter<>(affSrvc.affinity(cacheId), k.toIntArray(), ctx.rowHandler());
-
- return new Partitioned<>(assignments, affinity);
- }
-
/** */
public Object identity() {
return identity;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
index 6ef5cbf..f0ee64a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
@@ -24,12 +24,10 @@ import com.google.common.collect.Ordering;
import org.apache.calcite.plan.RelMultipleTrait;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.rel.RelDistribution;
import org.apache.calcite.util.ImmutableIntList;
import org.apache.calcite.util.mapping.Mapping;
import org.apache.calcite.util.mapping.Mappings;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
import static org.apache.calcite.rel.RelDistribution.Type.ANY;
import static org.apache.calcite.rel.RelDistribution.Type.BROADCAST_DISTRIBUTED;
@@ -72,7 +70,7 @@ public final class DistributionTrait implements IgniteDistribution {
}
/** {@inheritDoc} */
- @Override public Type getType() {
+ @Override public RelDistribution.Type getType() {
return function.type();
}
@@ -82,11 +80,6 @@ public final class DistributionTrait implements IgniteDistribution {
}
/** {@inheritDoc} */
- @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ectx, AffinityService affSrvc, ColocationGroup target) {
- return function.destination(ectx, affSrvc, target, keys);
- }
-
- /** {@inheritDoc} */
@Override public ImmutableIntList getKeys() {
return keys;
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
index 5e53013..22da20a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
@@ -20,9 +20,6 @@ package org.apache.ignite.internal.processors.query.calcite.trait;
import org.apache.calcite.rel.RelDistribution;
import org.apache.calcite.util.ImmutableIntList;
import org.apache.calcite.util.mapping.Mappings;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
/**
* Ignite distribution trait.
@@ -33,17 +30,6 @@ public interface IgniteDistribution extends RelDistribution {
*/
DistributionFunction function();
- /**
- * Creates a destination based on this function algorithm, given nodes mapping and distribution keys.
- *
- * @param ectx Execution context.
- * @param affinityService Affinity function source.
- * @param targetGroup Target mapping.
- * @return Destination function.
- */
- <Row> Destination<Row> destination(ExecutionContext<Row> ectx, AffinityService affinityService,
- ColocationGroup targetGroup);
-
/** {@inheritDoc} */
@Override ImmutableIntList getKeys();
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java
deleted file mode 100644
index e2ec63c..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java
+++ /dev/null
@@ -1,50 +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.trait;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.function.ToIntFunction;
-import java.util.stream.Collectors;
-
-/** */
-public final class Partitioned<Row> implements Destination<Row> {
- /** */
- private final List<List<String>> assignments;
-
- /** */
- private final ToIntFunction<Row> partFun;
-
- /** */
- public Partitioned(List<List<String>> assignments, ToIntFunction<Row> partFun) {
- this.assignments = assignments;
- this.partFun = partFun;
- }
-
- /** {@inheritDoc} */
- @Override public List<String> targets(Row row) {
- return assignments.get(partFun.applyAsInt(row) % assignments.size());
- }
-
- /** {@inheritDoc} */
- @Override public List<String> targets() {
- return assignments.stream()
- .flatMap(Collection::stream)
- .distinct().collect(Collectors.toList());
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java
deleted file mode 100644
index 33e8e0f..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java
+++ /dev/null
@@ -1,48 +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.trait;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-
-/** */
-public final class RandomNode<Row> implements Destination<Row> {
- /** */
- private final Random random;
-
- /** */
- private final List<String> nodes;
-
- /** */
- public RandomNode(List<String> nodes) {
- this.nodes = nodes;
-
- random = new Random();
- }
-
- /** {@inheritDoc} */
- @Override public List<String> targets(Row row) {
- return Collections.singletonList(nodes.get(random.nextInt(nodes.size())));
- }
-
- /** {@inheritDoc} */
- @Override public List<String> targets() {
- return nodes;
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
index 595f79c..31e9a74 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
@@ -18,20 +18,16 @@
package org.apache.ignite.internal.processors.query.calcite.util;
import java.io.StringReader;
-import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
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.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
@@ -67,11 +63,8 @@ import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningConte
import org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable;
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.schema.NativeType;
import org.apache.ignite.internal.util.ArrayUtils;
-import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.lang.IgniteException;
-import org.apache.ignite.lang.IgniteLogger;
import org.codehaus.commons.compiler.CompilerFactoryFactory;
import org.codehaus.commons.compiler.IClassBodyEvaluator;
import org.codehaus.commons.compiler.ICompilerFactory;
@@ -132,35 +125,6 @@ public final class Commons {
/** */
private Commons(){}
- public static <T> Cursor<T> createCursor(Iterable<T> iterable) {
- return createCursor(iterable.iterator());
- }
-
- public static <T> Cursor<T> createCursor(Iterator<T> iter) {
- return new Cursor<>() {
- @Override public void remove() {
- iter.remove();
- }
-
- @Override public boolean hasNext() {
- return iter.hasNext();
- }
-
- @Override public T next() {
- return iter.next();
- }
-
- @NotNull @Override public Iterator<T> iterator() {
- return iter;
- }
-
- @Override public void close() throws Exception {
- if (iter instanceof AutoCloseable)
- ((AutoCloseable)iter).close();
- }
- };
- }
-
/**
* Combines two lists.
*/
@@ -290,22 +254,13 @@ public final class Commons {
((AutoCloseable) o).close();
}
- /**
- * Closes given resource logging possible checked exception.
- *
- * @param o Resource to close. If it's {@code null} - it's no-op.
- * @param log Logger to log possible checked exception.
- */
- public static void close(Object o, @NotNull IgniteLogger log) {
- if (o instanceof AutoCloseable) {
- try {
- ((AutoCloseable)o).close();
- }
- catch (Exception e) {
- log.warn("Failed to close resource: " + e.getMessage(), e);
- }
- }
- }
+// /**
+// * @param o Object to close.
+// */
+// public static void close(Object o, IgniteLogger log) {
+// if (o instanceof AutoCloseable)
+// U.close((AutoCloseable) o, log);
+// }
//
// /**
// * @param o Object to close.
@@ -491,46 +446,4 @@ public final class Commons {
}
}
}
-
- public static Class<?> nativeTypeToClass(NativeType type) {
- assert type != null;
-
- switch (type.spec()) {
- case INT8:
- return Byte.class;
-
- case INT16:
- return Short.class;
-
- case INT32:
- return Integer.class;
-
- case INT64:
- return Long.class;
-
- case FLOAT:
- return Float.class;
-
- case DOUBLE:
- return Double.class;
-
- case DECIMAL:
- return BigDecimal.class;
-
- case UUID:
- return UUID.class;
-
- case STRING:
- return String.class;
-
- case BYTES:
- return byte[].class;
-
- case BITMASK:
- return BitSet.class;
-
- default:
- throw new IllegalArgumentException("Unsupported type " + type.spec());
- }
- }
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/NodeLeaveHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/NodeLeaveHandler.java
deleted file mode 100644
index 5aa03fe..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/NodeLeaveHandler.java
+++ /dev/null
@@ -1,40 +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.util;
-
-import java.util.function.Consumer;
-
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.TopologyEventHandler;
-
-public class NodeLeaveHandler implements TopologyEventHandler {
- private final Consumer<ClusterNode> onDisappeared;
-
- public NodeLeaveHandler(Consumer<ClusterNode> onDisappeared) {
- this.onDisappeared = onDisappeared;
- }
-
- /** {@inheritDoc} */
- @Override public void onAppeared(ClusterNode member) {
- // NO-OP
- }
-
- /** {@inheritDoc} */
- @Override public void onDisappeared(ClusterNode member) {
- onDisappeared.accept(member);
- }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java
index d84c953..962e678 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java
@@ -47,6 +47,7 @@ public class StripedThreadPoolExecutor implements ExecutorService {
* Create striped thread pool.
*
* @param concurrentLvl Concurrency level.
+ * @param igniteInstanceName Node name.
* @param threadNamePrefix Thread name prefix.
* @param allowCoreThreadTimeOut Sets the policy governing whether core threads may time out and
* terminate if no tasks arrive within the keep-alive time.
@@ -55,6 +56,7 @@ public class StripedThreadPoolExecutor implements ExecutorService {
*/
public StripedThreadPoolExecutor(
int concurrentLvl,
+ String igniteInstanceName,
String threadNamePrefix,
UncaughtExceptionHandler eHnd,
boolean allowCoreThreadTimeOut,
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java
index 0b0f46f..92742b8 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java
@@ -50,16 +50,17 @@ public class AbstractExecutionTest extends IgniteAbstractTest {
/** */
@BeforeEach
- public void setup() {
- taskExecutor = new QueryTaskExecutorImpl(
- new StripedThreadPoolExecutor(
- 4,
- "calciteQry",
- this::handle,
- true,
- 60_000L
- )
- );
+ public void setup() throws Exception {
+ taskExecutor = new QueryTaskExecutorImpl();
+
+ taskExecutor.stripedThreadPoolExecutor(new StripedThreadPoolExecutor(
+ 4,
+ "calcite-exec-test-ignite",
+ "calciteQry",
+ this::handle,
+ true,
+ 60_000L
+ ));
}
/** */
@@ -77,7 +78,7 @@ public class AbstractExecutionTest extends IgniteAbstractTest {
return new ExecutionContext<>(
taskExecutor,
PlanningContext.builder()
- .localNodeId(UUID.randomUUID().toString())
+ .localNodeId(UUID.randomUUID())
.build(),
UUID.randomUUID(),
fragmentDesc,
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
index dd1f5cd..778df9b 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
@@ -24,7 +24,6 @@ import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.function.BiFunction;
-import java.util.function.Function;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -62,7 +61,6 @@ import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql2rel.InitializerContext;
import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
import org.apache.ignite.internal.processors.query.calcite.externalize.RelJsonReader;
import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
import org.apache.ignite.internal.processors.query.calcite.prepare.Cloner;
@@ -90,7 +88,6 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem
import org.apache.ignite.internal.testframework.IgniteAbstractTest;
import org.apache.ignite.internal.testframework.IgniteTestUtils;
import org.apache.ignite.internal.util.ArrayUtils;
-import org.jetbrains.annotations.Nullable;
import static org.apache.calcite.tools.Frameworks.createRootSchema;
import static org.apache.calcite.tools.Frameworks.newConfigBuilder;
@@ -438,10 +435,10 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
CorrelationTraitDef.INSTANCE
};
- List<String> nodes = new ArrayList<>(4);
+ List<UUID> nodes = new ArrayList<>(4);
for (int i = 0; i < 4; i++)
- nodes.add(UUID.randomUUID().toString());
+ nodes.add(UUID.randomUUID());
PlanningContext ctx = PlanningContext.builder()
.localNodeId(first(nodes))
@@ -556,17 +553,6 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
}
/** {@inheritDoc} */
- @Override public <Row> Iterable<Row> scan(
- ExecutionContext<Row> execCtx,
- ColocationGroup group,
- Predicate<Row> filter,
- Function<Row, Row> rowTransformer,
- @Nullable ImmutableBitSet usedColumns
- ) {
- throw new AssertionError();
- }
-
- /** {@inheritDoc} */
@Override public RelDataType getRowType(RelDataTypeFactory typeFactory, ImmutableBitSet bitSet) {
RelDataType rowType = protoType.apply(typeFactory);
@@ -724,11 +710,6 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
}
/** {@inheritDoc} */
- @Override public ColocationGroup colocationGroup(PlanningContext ctx) {
- throw new AssertionError();
- }
-
- /** {@inheritDoc} */
@Override public ColumnStrategy generationStrategy(RelOptTable table, int iColumn) {
throw new AssertionError();
}
@@ -765,6 +746,11 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
}
/** {@inheritDoc} */
+ @Override public boolean field() {
+ return true;
+ }
+
+ /** {@inheritDoc} */
@Override public boolean key() {
return false;
}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/ExceptPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/ExceptPlannerTest.java
new file mode 100644
index 0000000..429ce93
--- /dev/null
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/ExceptPlannerTest.java
@@ -0,0 +1,329 @@
+/*
+ * 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.planner;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+
+/**
+ * Test to verify EXCEPT operator.
+ */
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public class ExceptPlannerTest extends AbstractPlannerTest {
+ /** Public schema. */
+ private IgniteSchema publicSchema;
+
+ /** */
+ @BeforeAll
+ public void setup() {
+ publicSchema = new IgniteSchema("PUBLIC");
+
+ IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
+
+ RelDataType type = new RelDataTypeFactory.Builder(f)
+ .add("ID", f.createJavaType(Integer.class))
+ .add("NAME", f.createJavaType(String.class))
+ .add("SALARY", f.createJavaType(Double.class))
+ .build();
+
+ createTable(publicSchema, "RANDOM_TBL1", type, IgniteDistributions.random());
+ createTable(publicSchema, "RANDOM_TBL2", type, IgniteDistributions.random());
+ createTable(publicSchema, "BROADCAST_TBL1", type, IgniteDistributions.broadcast());
+ createTable(publicSchema, "BROADCAST_TBL2", type, IgniteDistributions.broadcast());
+ createTable(publicSchema, "SINGLE_TBL1", type, IgniteDistributions.single());
+ createTable(publicSchema, "SINGLE_TBL2", type, IgniteDistributions.single());
+
+ createTable(publicSchema, "AFFINITY_TBL1", type,
+ IgniteDistributions.affinity(0, "Test1", "hash"));
+
+ createTable(publicSchema, "AFFINITY_TBL2", type,
+ IgniteDistributions.affinity(0, "Test2", "hash"));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptRandom() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM random_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> !n.all)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptAllRandom() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl1 " +
+ "EXCEPT ALL " +
+ "SELECT * FROM random_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> n.all)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptBroadcast() throws Exception {
+ String sql = "" +
+ "SELECT * FROM broadcast_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM broadcast_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, isTableScan("broadcast_tbl1")))
+ .and(input(1, isTableScan("broadcast_tbl2")))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptSingle() throws Exception {
+ String sql = "" +
+ "SELECT * FROM single_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM single_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, isTableScan("single_tbl1")))
+ .and(input(1, isTableScan("single_tbl2"))));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptSingleAndRandom() throws Exception {
+ String sql = "" +
+ "SELECT * FROM single_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM random_tbl1 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, isTableScan("single_tbl1")))
+ .and(input(1, hasChildThat(isTableScan("random_tbl1")))));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptSingleAndAffinity() throws Exception {
+ String sql = "" +
+ "SELECT * FROM single_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM affinity_tbl1 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, isTableScan("single_tbl1")))
+ .and(input(1, hasChildThat(isTableScan("affinity_tbl1")))));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptSingleAndBroadcast() throws Exception {
+ String sql = "" +
+ "SELECT * FROM single_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM broadcast_tbl1 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, isTableScan("single_tbl1")))
+ .and(input(1, isTableScan("broadcast_tbl1")))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptAffinity() throws Exception {
+ String sql = "" +
+ "SELECT * FROM affinity_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM affinity_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("affinity_tbl1")))
+ .and(input(1, isTableScan("affinity_tbl2")))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptBroadcastAndRandom() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM broadcast_tbl1 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, hasChildThat(isTableScan("random_tbl1"))))
+ .and(input(1, isTableScan("broadcast_tbl1")))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptRandomNested() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl2 EXCEPT (" +
+ " SELECT * FROM random_tbl1 " +
+ " EXCEPT " +
+ " SELECT * FROM random_tbl2" +
+ ")";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, hasChildThat(isTableScan("random_tbl2"))))
+ .and(input(1, isInstanceOf(IgniteReduceMinus.class)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ ))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptBroadcastAndRandomNested() throws Exception {
+ String sql = "" +
+ "SELECT * FROM broadcast_tbl1 EXCEPT (" +
+ " SELECT * FROM random_tbl1 " +
+ " EXCEPT " +
+ " SELECT * FROM random_tbl2" +
+ ")";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
+ .and(input(0, isTableScan("broadcast_tbl1")))
+ .and(input(1, isInstanceOf(IgniteReduceMinus.class)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ ))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptMerge() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM random_tbl2 " +
+ "EXCEPT " +
+ "SELECT * FROM affinity_tbl1 " +
+ "EXCEPT " +
+ "SELECT * FROM affinity_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ .and(input(2, isTableScan("affinity_tbl1")))
+ .and(input(3, isTableScan("affinity_tbl2")))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptAllMerge() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl1 " +
+ "EXCEPT ALL " +
+ "SELECT * FROM random_tbl2 " +
+ "EXCEPT ALL " +
+ "SELECT * FROM affinity_tbl1 " +
+ "EXCEPT ALL " +
+ "SELECT * FROM affinity_tbl2 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> n.all)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ .and(input(2, isTableScan("affinity_tbl1")))
+ .and(input(3, isTableScan("affinity_tbl2")))
+ ))
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testExceptAllWithExceptMerge() throws Exception {
+ String sql = "" +
+ "SELECT * FROM random_tbl1 " +
+ "EXCEPT ALL " +
+ "SELECT * FROM random_tbl2 " +
+ "EXCEPT " +
+ "SELECT * FROM affinity_tbl1 ";
+
+ assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> !n.all)
+ .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
+ .and(input(0, isTableScan("random_tbl1")))
+ .and(input(1, isTableScan("random_tbl2")))
+ .and(input(2, isTableScan("affinity_tbl1")))
+ ))
+ );
+ }
+}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java
index 1001446..a0115b4 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java
@@ -87,14 +87,14 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
*/
public class PlannerTest extends AbstractPlannerTest {
/** */
- private static List<String> NODES;
+ private static List<UUID> NODES;
@BeforeAll
public static void init() {
NODES = new ArrayList<>(4);
for (int i = 0; i < 4; i++)
- NODES.add(UUID.randomUUID().toString());
+ NODES.add(UUID.randomUUID());
}
/**
@@ -1840,7 +1840,7 @@ public class PlannerTest extends AbstractPlannerTest {
}
/** */
- private List<String> intermediateMapping(long topVer, boolean single,
+ private List<UUID> intermediateMapping(long topVer, boolean single,
@Nullable Predicate<ClusterNode> filter) {
return single ? select(NODES, 0) : select(NODES, 0, 1, 2, 3);
}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SetOpPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SetOpPlannerTest.java
deleted file mode 100644
index 3559492..0000000
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SetOpPlannerTest.java
+++ /dev/null
@@ -1,421 +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.planner;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapIntersect;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapSetOp;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceIntersect;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceSetOp;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleIntersect;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleSetOp;
-import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.TestInstance;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.EnumSource;
-
-/**
- * Test to verify set op (EXCEPT, INTERSECT).
- */
-@TestInstance(TestInstance.Lifecycle.PER_CLASS)
-public class SetOpPlannerTest extends AbstractPlannerTest {
- /** Public schema. */
- private IgniteSchema publicSchema;
-
- /** */
- @BeforeAll
- public void setup() {
- publicSchema = new IgniteSchema("PUBLIC");
-
- IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
-
- RelDataType type = new RelDataTypeFactory.Builder(f)
- .add("ID", f.createJavaType(Integer.class))
- .add("NAME", f.createJavaType(String.class))
- .add("SALARY", f.createJavaType(Double.class))
- .build();
-
- createTable(publicSchema, "RANDOM_TBL1", type, IgniteDistributions.random());
- createTable(publicSchema, "RANDOM_TBL2", type, IgniteDistributions.random());
- createTable(publicSchema, "BROADCAST_TBL1", type, IgniteDistributions.broadcast());
- createTable(publicSchema, "BROADCAST_TBL2", type, IgniteDistributions.broadcast());
- createTable(publicSchema, "SINGLE_TBL1", type, IgniteDistributions.single());
- createTable(publicSchema, "SINGLE_TBL2", type, IgniteDistributions.single());
-
- createTable(publicSchema, "AFFINITY_TBL1", type,
- IgniteDistributions.affinity(0, "Test1", "hash"));
-
- createTable(publicSchema, "AFFINITY_TBL2", type,
- IgniteDistributions.affinity(0, "Test2", "hash"));
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpRandom(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM random_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(n -> !n.all())
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- ))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpAllRandom(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl1 " +
- setOpAll(setOp) +
- "SELECT * FROM random_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(IgniteSetOp::all)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- ))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpBroadcast(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM broadcast_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM broadcast_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, isTableScan("broadcast_tbl1")))
- .and(input(1, isTableScan("broadcast_tbl2")))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpSingle(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM single_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM single_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, isTableScan("single_tbl1")))
- .and(input(1, isTableScan("single_tbl2"))));
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpSingleAndRandom(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM single_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM random_tbl1 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, isTableScan("single_tbl1")))
- .and(input(1, hasChildThat(isTableScan("random_tbl1")))));
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpSingleAndAffinity(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM single_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM affinity_tbl1 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, isTableScan("single_tbl1")))
- .and(input(1, hasChildThat(isTableScan("affinity_tbl1")))));
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpSingleAndBroadcast(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM single_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM broadcast_tbl1 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, isTableScan("single_tbl1")))
- .and(input(1, isTableScan("broadcast_tbl1")))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpAffinity(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM affinity_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM affinity_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("affinity_tbl1")))
- .and(input(1, isTableScan("affinity_tbl2")))
- ))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpBroadcastAndRandom(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM broadcast_tbl1 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, hasChildThat(isTableScan("random_tbl1"))))
- .and(input(1, isTableScan("broadcast_tbl1")))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpRandomNested(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl2 " + setOp(setOp) + "(" +
- " SELECT * FROM random_tbl1 " +
- setOp(setOp) +
- " SELECT * FROM random_tbl2" +
- ")";
-
- if (setOp == SetOp.EXCEPT) {
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, hasChildThat(isTableScan("random_tbl2"))))
- .and(input(1, isInstanceOf(setOp.reduce)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- ))
- ))
- );
- }
- else {
- // INTERSECT operator is commutative and can be merged.
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl2")))
- .and(input(1, isTableScan("random_tbl1")))
- .and(input(2, isTableScan("random_tbl2")))
- ))
- );
- }
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpBroadcastAndRandomNested(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM broadcast_tbl1 " + setOp(setOp) + "(" +
- " SELECT * FROM random_tbl1 " +
- setOp(setOp) +
- " SELECT * FROM random_tbl2" +
- ")";
-
- if (setOp == SetOp.EXCEPT) {
- assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
- .and(input(0, isTableScan("broadcast_tbl1")))
- .and(input(1, isInstanceOf(setOp.reduce)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- ))
- ))
- );
- }
- else {
- // INTERSECT operator is commutative and can be merged.
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, nodeOrAnyChild(isTableScan("broadcast_tbl1"))))
- .and(input(1, isTableScan("random_tbl1")))
- .and(input(2, isTableScan("random_tbl2")))
- ))
- );
- }
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpMerge(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM random_tbl2 " +
- setOp(setOp) +
- "SELECT * FROM affinity_tbl1 " +
- setOp(setOp) +
- "SELECT * FROM affinity_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- .and(input(2, isTableScan("affinity_tbl1")))
- .and(input(3, isTableScan("affinity_tbl2")))
- ))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpAllMerge(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl1 " +
- setOpAll(setOp) +
- "SELECT * FROM random_tbl2 " +
- setOpAll(setOp) +
- "SELECT * FROM affinity_tbl1 " +
- setOpAll(setOp) +
- "SELECT * FROM affinity_tbl2 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(IgniteSetOp::all)
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- .and(input(2, isTableScan("affinity_tbl1")))
- .and(input(3, isTableScan("affinity_tbl2")))
- ))
- );
- }
-
- /**
- * @throws Exception If failed.
- */
- @ParameterizedTest
- @EnumSource
- public void testSetOpAllWithExceptMerge(SetOp setOp) throws Exception {
- String sql =
- "SELECT * FROM random_tbl1 " +
- setOpAll(setOp) +
- "SELECT * FROM random_tbl2 " +
- setOp(setOp) +
- "SELECT * FROM affinity_tbl1 ";
-
- assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(n -> !n.all())
- .and(hasChildThat(isInstanceOf(setOp.map)
- .and(input(0, isTableScan("random_tbl1")))
- .and(input(1, isTableScan("random_tbl2")))
- .and(input(2, isTableScan("affinity_tbl1")))
- ))
- );
- }
-
- /** */
- private String setOp(SetOp setOp) {
- return setOp.name() + ' ';
- }
-
- /** */
- private String setOpAll(SetOp setOp) {
- return setOp.name() + " ALL ";
- }
-
- /** */
- enum SetOp {
- /** */
- EXCEPT(
- IgniteSingleMinus.class,
- IgniteMapMinus.class,
- IgniteReduceMinus.class
- ),
-
- /** */
- INTERSECT(
- IgniteSingleIntersect.class,
- IgniteMapIntersect.class,
- IgniteReduceIntersect.class
- );
-
- /** */
- public final Class<? extends IgniteSingleSetOp> single;
-
- /** */
- public final Class<? extends IgniteMapSetOp> map;
-
- /** */
- public final Class<? extends IgniteReduceSetOp> reduce;
-
- /** */
- SetOp(
- Class<? extends IgniteSingleSetOp> single,
- Class<? extends IgniteMapSetOp> map,
- Class<? extends IgniteReduceSetOp> reduce) {
- this.single = single;
- this.map = map;
- this.reduce = reduce;
- }
- }
-}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionTest.java
similarity index 98%
rename from modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionPlannerTest.java
rename to modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionTest.java
index c2c9dc3..e6f1b04 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionPlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionTest.java
@@ -35,7 +35,7 @@ import org.junit.jupiter.api.TestInstance;
* Test table functions.
*/
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
-public class TableFunctionPlannerTest extends AbstractPlannerTest {
+public class TableFunctionTest extends AbstractPlannerTest {
/** Public schema. */
private IgniteSchema publicSchema;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/Cancellable.java b/modules/core/src/main/java/org/apache/ignite/internal/util/Cancellable.java
deleted file mode 100644
index 84397cc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/Cancellable.java
+++ /dev/null
@@ -1,29 +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.util;
-
-/**
- * A {@code Cancellable} represents a process or an operation that can be
- * canceled.
- */
-public interface Cancellable {
- /**
- * Cancels the ongoing operation or process or do nothing if it has been
- * already cancelled.
- */
- void cancel();
-}
diff --git a/modules/runner/pom.xml b/modules/runner/pom.xml
index c0f82b8..711ec38 100644
--- a/modules/runner/pom.xml
+++ b/modules/runner/pom.xml
@@ -84,11 +84,6 @@
<artifactId>ignite-table</artifactId>
</dependency>
- <dependency>
- <groupId>org.apache.ignite</groupId>
- <artifactId>ignite-calcite</artifactId>
- </dependency>
-
<!-- 3rd party dependencies -->
<dependency>
<groupId>org.slf4j</groupId>
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 54a93b9..5f276d4 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.app;
import org.apache.ignite.app.Ignite;
import org.apache.ignite.internal.vault.VaultManager;
-import org.apache.ignite.internal.processors.query.calcite.SqlQueryProcessor;
import org.apache.ignite.table.manager.IgniteTables;
import org.apache.ignite.tx.IgniteTransactions;
@@ -33,17 +32,13 @@ public class IgniteImpl implements Ignite {
/** Vault manager */
private final VaultManager vaultManager;
- private final SqlQueryProcessor qryEngine;
-
/**
* @param tableManager Table manager.
* @param vaultManager Vault manager.
- * @param qryEngine Query processor.
*/
- IgniteImpl(IgniteTables tableManager, VaultManager vaultManager, SqlQueryProcessor qryEngine) {
+ IgniteImpl(IgniteTables tableManager, VaultManager vaultManager) {
this.distributedTableManager = tableManager;
this.vaultManager = vaultManager;
- this.qryEngine = qryEngine;
}
/** {@inheritDoc} */
@@ -51,10 +46,6 @@ public class IgniteImpl implements Ignite {
return distributedTableManager;
}
- public SqlQueryProcessor queryEngine() {
- return qryEngine;
- }
-
/** {@inheritDoc} */
@Override public IgniteTransactions transactions() {
return null;
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
index a3face6..bf6f300 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
@@ -28,7 +28,6 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
-
import org.apache.ignite.app.Ignite;
import org.apache.ignite.app.Ignition;
import org.apache.ignite.configuration.RootKey;
@@ -43,7 +42,6 @@ import org.apache.ignite.internal.baseline.BaselineManager;
import org.apache.ignite.internal.configuration.ConfigurationManager;
import org.apache.ignite.internal.configuration.storage.ConfigurationStorage;
import org.apache.ignite.internal.metastorage.MetaStorageManager;
-import org.apache.ignite.internal.processors.query.calcite.SqlQueryProcessor;
import org.apache.ignite.internal.raft.Loza;
import org.apache.ignite.internal.schema.SchemaManager;
import org.apache.ignite.internal.storage.DistributedConfigurationStorage;
@@ -60,6 +58,7 @@ import org.apache.ignite.network.ClusterService;
import org.apache.ignite.network.MessageSerializationRegistryImpl;
import org.apache.ignite.network.NetworkAddress;
import org.apache.ignite.network.scalecube.ScaleCubeClusterServiceFactory;
+import org.apache.ignite.table.manager.IgniteTables;
import org.apache.ignite.utils.IgniteProperties;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -221,7 +220,7 @@ public class IgnitionImpl implements Ignition {
SchemaManager schemaMgr = new SchemaManager(configurationMgr, metaStorageMgr, vaultMgr);
// Distributed table manager startup.
- TableManager distributedTblMgr = new TableManager(
+ IgniteTables distributedTblMgr = new TableManager(
configurationMgr,
metaStorageMgr,
schemaMgr,
@@ -230,11 +229,6 @@ public class IgnitionImpl implements Ignition {
vaultMgr
);
- SqlQueryProcessor qryProc = new SqlQueryProcessor(
- clusterNetSvc,
- distributedTblMgr
- );
-
// TODO IGNITE-14579 Start rest manager.
// Deploy all resisted watches cause all components are ready and have registered their listeners.
@@ -242,7 +236,7 @@ public class IgnitionImpl implements Ignition {
ackSuccessStart();
- return new IgniteImpl(distributedTblMgr, vaultMgr, qryProc);
+ return new IgniteImpl(distributedTblMgr, vaultMgr);
}
/**
diff --git a/modules/schema/README.md b/modules/schema/README.md
index f9f6742..7ec07c2 100644
--- a/modules/schema/README.md
+++ b/modules/schema/README.md
@@ -3,7 +3,7 @@
This module provides implementation for schema management components:
* Public API for schema definition and evolution
-* Schema manager component that implements necessary machinery to translate schema management commands to corresponding
+* Schema manager component that implements necessary machinary to translate schema management commands to corresponding
metastorage modifications, as well as schema modification event processing logic
* Necessary logic to build and upgrade rows of specific schema that encode user data in schema-defined format.
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java b/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
index e146f3c..e35068e 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
@@ -18,9 +18,9 @@
package org.apache.ignite.internal.table.event;
import java.util.UUID;
-
import org.apache.ignite.internal.manager.EventParameters;
import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.table.Table;
/**
* Table event parameters. There are properties which associate with a concrete table.
@@ -84,7 +84,7 @@ public class TableEventParameters implements EventParameters {
*
* @return Table.
*/
- public TableImpl table() {
+ public Table table() {
return table;
}
}
diff --git a/parent/pom.xml b/parent/pom.xml
index 0130ec1..ad90c1b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -150,12 +150,6 @@
<dependency>
<groupId>org.apache.ignite</groupId>
- <artifactId>ignite-calcite</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- <dependency>
- <groupId>org.apache.ignite</groupId>
<artifactId>ignite-cli-common</artifactId>
<version>${project.version}</version>
</dependency>