You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/10/06 14:31:37 UTC

[GitHub] [ignite] tledkov-gridgain opened a new pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

tledkov-gridgain opened a new pull request #9476:
URL: https://github.com/apache/ignite/pull/9476


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737586972



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(

Review comment:
       For the test we use the query that is planned for a long time (contains a lot of joins).
   Also very long query is used to check query registry on EXECUTING phase.
   
   I see no way to insert barrier between planning and execution.
   I guess we can change the test's query if the planner will plan such queries very fast in future.
   WDYT?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735621700



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.Collection;
+import java.util.UUID;
+
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ * Registry of the running queries.
+ */
+public interface QueryRegistry<RowT> extends Service {
+    /**
+     * Register the query or return the exists query with the same identifier.
+     *
+     * @param qry Query to register.
+     * @return registered query.
+     */
+    Query<RowT> register(Query<RowT> qry);
+
+    /**
+     * Lookup query by identifier.
+     *
+     * @param id Query identified.
+     * @return registered query or {@code null} if the query with specified identifier isn't found.

Review comment:
       ```suggestion
        * @return Registered query or {@code null} if the query with specified identifier isn't found.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744467893



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -36,4 +39,10 @@
      */
     List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String schemaName, String qry, Object... params)
         throws IgniteSQLException;
+
+    /** */
+    Collection<? extends RunningQuery> runningQueries();

Review comment:
       If I'm not mistaken, the current approach to cancel query is to get instance of `RunningQuery` and invoke on it a `cancel` method. Unfortunately such an approach doesn't cover the case when current node doesn't run any fragment of the query of interest




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723926084



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +618,15 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qryNew = new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id()));
+
+            Query<Row> qry = qryReg.register(qryNew);

Review comment:
       contract of register method shows that it can return query by uuid if such a query already registered, in such a case we can potentially obtain a problems here, do we need such contract ? I found that return state of register is used only here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724274855



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {
+    public static final CompletableFuture[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture[0];

Review comment:
       seems could be private




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737575048



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(

Review comment:
       Great issue. Thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737569515



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +622,13 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qry = qryReg.register(new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id())));

Review comment:
       Not catch an idea.
   Do you suggest the one additional lookup in the map?
   ```
   qry = qryReg.query(msg.queryId())?
   if (qry == null) {
       qry = qryReg.register(new Query()); // computeIfAbsent is replaced to putIfAbsent inside.
   }
   ```
   Please take a look at the @korlov42 [comment](https://github.com/apache/ignite/pull/9476#discussion_r735693816).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737556908



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.Collection;
+import java.util.UUID;
+
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ * Registry of the running queries.
+ */
+public interface QueryRegistry<RowT> extends Service {

Review comment:
       Great idea. Fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735693816



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistryImpl.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.Collection;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Registry of the running queries.
+ */
+public class QueryRegistryImpl<RowT> implements QueryRegistry<RowT> {
+    /** */
+    private final ConcurrentMap<UUID, Query<RowT>> runningQrys = new ConcurrentHashMap<>();
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    public QueryRegistryImpl(IgniteLogger log) {
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query<RowT> register(Query<RowT> qry) {
+        Query<RowT> old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;

Review comment:
       ```suggestion
           return runningQrys.computeIfAbsent(qry.id(), k -> qry);
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733516027



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(
+                qry,
+                plan
+            ));
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
+            try {
+                if (qryList.size() == 1) {

Review comment:
       I guess it is out of the scope of this patch. The main goal of the patch is refactoring query registration (introduce query registry) and query cancellation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r725052896



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {
+    public static final CompletableFuture[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture[0];
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Set<RunningFragment<Row>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INIT;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    public static BaseQueryContext createQueryContext(QueryContext ctx, SchemaPlus schema, IgniteLogger log) {
+        return BaseQueryContext.builder()
+            .parentContext(Commons.convert(ctx))
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        for (RunningFragment<Row> frag : fragments) {
+            CompletableFuture<?> f = frag.context().submit(frag.root()::close, frag.root()::onError);
+
+            CompletableFuture<?> fCancel = f.thenApply((u) -> frag.context().submit(frag.context()::cancel, frag.root()::onError));
+            futs.add(fCancel);
+        }
+
+        CompletableFuture.allOf(futs.toArray(COMPLETABLE_FUTURES_EMPTY_ARRAY))
+            .thenAccept((u) -> unregister.accept(this));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        synchronized (this) {
+            if (state == QueryState.CLOSED)

Review comment:
       you work with the state within synchronization in this class, but for RootQuery it is not so. Seems it's wrong. Let's imaging the situation when query is starting and the cancelation process is in parallel, then we can have state EXECUTION after the cancelation is completed. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723961903



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +375,36 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Query register(Query qry) {
+        Query old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query query(UUID id) {
+        return runningQrys.get(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(UUID id) {
+        runningQrys.remove(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Query> runningQueries() {
+        return runningQrys.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       runningQuery - returns interface RunningQuery, but runningQueries return implementation, looks like problem in API. Correct me if i`m wrong.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733520915



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {
+    public static final CompletableFuture[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture[0];
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Set<RunningFragment<Row>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INIT;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    public static BaseQueryContext createQueryContext(QueryContext ctx, SchemaPlus schema, IgniteLogger log) {
+        return BaseQueryContext.builder()
+            .parentContext(Commons.convert(ctx))
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        for (RunningFragment<Row> frag : fragments) {
+            CompletableFuture<?> f = frag.context().submit(frag.root()::close, frag.root()::onError);
+
+            CompletableFuture<?> fCancel = f.thenApply((u) -> frag.context().submit(frag.context()::cancel, frag.root()::onError));
+            futs.add(fCancel);
+        }
+
+        CompletableFuture.allOf(futs.toArray(COMPLETABLE_FUTURES_EMPTY_ARRAY))
+            .thenAccept((u) -> unregister.accept(this));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        synchronized (this) {
+            if (state == QueryState.CLOSED)

Review comment:
       fixed. All R+W operations with state is under synchronized section.  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733435591



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface QueryRegistry extends Service {

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735608726



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
##########
@@ -27,5 +27,5 @@
     /**
      * @return Schema.
      */
-    SchemaPlus schema();
+    SchemaPlus schema(String schema);

Review comment:
       let's mark the `schema` param as `@Nullable`. Also javadoc should be updated as well




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724215062



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),

Review comment:
       Let's calculate schema just once ( see at 303 lines)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r725056892



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {
+    public static final CompletableFuture[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture[0];

Review comment:
       +javadoc and generic wilcard for CompletableFuture




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723992726



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
##########
@@ -269,6 +269,11 @@ private static Object affinityIdentity(CacheConfiguration<?, ?> ccfg) {
         rebuild();
     }
 
+    /** {@inheritDoc} */

Review comment:
       What`s the purpose of such a functionality ? No java doc found ^(




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733671047



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (this) {

Review comment:
       is it possible that we will take a lock on a different objects? THIS for RootQuery and for Query could be not the same.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733674001



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<RowT> implements RunningQuery {
+    /** Completable futures empty array. */
+    private static final CompletableFuture<?>[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture<?>[0];
+
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Set<RunningFragment<RowT>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query<RowT>> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INIT;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query<RowT>> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    public static BaseQueryContext createQueryContext(QueryContext ctx, SchemaPlus schema, IgniteLogger log) {
+        return BaseQueryContext.builder()
+            .parentContext(Commons.convert(ctx))
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        for (RunningFragment<RowT> frag : fragments) {
+            CompletableFuture<?> f = frag.context().submit(frag.root()::close, frag.root()::onError);
+
+            CompletableFuture<?> fCancel = f.thenApply((u) -> frag.context().submit(frag.context()::cancel, frag.root()::onError));
+            futs.add(fCancel);
+        }
+
+        CompletableFuture.allOf(futs.toArray(COMPLETABLE_FUTURES_EMPTY_ARRAY))
+            .thenAccept((u) -> unregister.accept(this));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        synchronized (this) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.EXECUTION)
+                state = QueryState.CLOSED;
+        }
+
+        for (RunningFragment<RowT> frag : fragments) {
+            frag.context().execute(()-> {
+                frag.root().onError(new ExecutionCancelledException());
+            }, frag.root()::onError);
+        }
+
+        tryClose();
+    }
+
+    /** */
+    public void addFragment(RunningFragment f) {

Review comment:
       ```suggestion
       public void addFragment(RunningFragment<RowT> f) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733435139



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    public RootQuery childQuery(SchemaPlus schema) {
+        return new RootQuery(sql, schema, params, null, exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /** */
+    public void run(ExecutionContext<Row> ctx, MultiStepPlan plan, Node<Row> root) {
+        if (state == QueryState.CLOSED) {
+            throw new IgniteSQLException(
+                "The query was cancelled while executing.",
+                IgniteQueryErrorCode.QUERY_CANCELED
+            );
+        }
+
+        RootNode<Row> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+        rootNode.register(root);

Review comment:
       Compare and write to state will be moved under synchronize section




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] Berkof commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
Berkof commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737999397



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(
+                    qry,
+                    plan
+                ));
+            }
+                catch (Exception e) {
+                qryReg.unregister(qry.id());
+
+                if (qry.isCancelled())
+                    throw new IgniteSQLException("The query was cancelled while planning", IgniteQueryErrorCode.QUERY_CANCELED, e);

Review comment:
       Can we get some other exception here? For example - while query cancellation.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(
+                    qry,
+                    plan
+                ));
+            }
+                catch (Exception e) {
+                qryReg.unregister(qry.id());
+
+                if (qry.isCancelled())
+                    throw new IgniteSQLException("The query was cancelled while planning", IgniteQueryErrorCode.QUERY_CANCELED, e);
+                else
+                    throw e;
+            }
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        List<RootQuery<Object[]>> qrys = new ArrayList<>(qryList.size());
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.schema(schemaName), // Update schema for each query in multiple statements.
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qrys.add(qry);
+
+            qryReg.register(qry);
+            try {

Review comment:
       Empty line before try block.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RunningFragment.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.Objects;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/** */

Review comment:
       JavaDoc

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<RowT> implements RunningQuery {
+    /** Completable futures empty array. */
+    private static final CompletableFuture<?>[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture<?>[0];
+
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Object mux = new Object();
+
+    /** */
+    protected final Set<RunningFragment<RowT>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query<RowT>> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INITED;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query<RowT>> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        for (RunningFragment<RowT> frag : fragments) {
+            CompletableFuture<?> f = frag.context().submit(frag.root()::close, frag.root()::onError);
+
+            CompletableFuture<?> fCancel = f.thenApply((u) -> frag.context().submit(frag.context()::cancel, frag.root()::onError));
+            futs.add(fCancel);
+        }
+
+        CompletableFuture.allOf(futs.toArray(COMPLETABLE_FUTURES_EMPTY_ARRAY))
+            .thenAccept((u) -> unregister.accept(this));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.EXECUTING)
+                state = QueryState.CLOSED;
+        }
+
+        for (RunningFragment<RowT> frag : fragments)
+            frag.context().execute(() -> frag.root().onError(new ExecutionCancelledException()), frag.root()::onError);
+
+        tryClose();
+    }
+
+    /** */

Review comment:
       JavaDoc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/RunningQuery.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+public interface RunningQuery {
+    /** */
+    UUID id();

Review comment:
       Should we use public modifiers here?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +396,19 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       No usage found.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,

Review comment:
       Contains 2.5 copy of execution code:
   1) if there are cached plan
   2) if there is no cached plan for single statement qry (with caching)
   2.5) if there is no cached plan for... multistatement qry? (without caching)
   With same exception handling. Let's refactor it with some "getOrCreateQryPlan(qry)" and "executeQry(RootQuery)"

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {

Review comment:
       Parameters should be alined one by line:
   @Override public List<FieldsQueryCursor<List<?>>> query(
       @Nullable QueryContext qryCtx, 
       @Nullable String schemaName,
       String SQL, 
       Object... params
   ) throws IgniteSQLException {

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */

Review comment:
       JavaDoc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/RunningQuery.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+import java.util.UUID;
+
+/**
+ *

Review comment:
       JavaDoc

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<RowT> implements RunningQuery {
+    /** Completable futures empty array. */
+    private static final CompletableFuture<?>[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture<?>[0];
+
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Object mux = new Object();
+
+    /** */
+    protected final Set<RunningFragment<RowT>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query<RowT>> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INITED;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query<RowT>> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        for (RunningFragment<RowT> frag : fragments) {

Review comment:
       Empty line before for block.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +623,13 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qry = (Query<Row>)qryReg.register(new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id())));

Review comment:
       Too long line of code.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -36,4 +39,10 @@
      */
     List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String schemaName, String qry, Object... params)
         throws IgniteSQLException;
+
+    /** */
+    Collection<? extends RunningQuery> runningQueries();
+
+    /** */
+    RunningQuery runningQuery(UUID id);

Review comment:
       1) No usage found.
   2) Should it be public?
   

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistryImpl.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.Collection;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Registry of the running queries.
+ */
+public class QueryRegistryImpl implements QueryRegistry {
+    /** */
+    private final ConcurrentMap<UUID, RunningQuery> runningQrys = new ConcurrentHashMap<>();
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    public QueryRegistryImpl(IgniteLogger log) {
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery register(RunningQuery qry) {
+        return runningQrys.computeIfAbsent(qry.id(), k -> qry);

Review comment:
       Why not putIfAbsent?

##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> qry.state() == QueryState.PLANNING, TIMEOUT_IN_MS));
+
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(0), IgniteSQLException.class, "The query was cancelled while planning");
+    }
+
+    /** */
+    @Test
+    public void testCancelAtExecutionPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 6;
+
+        sql("CREATE TABLE person (id int, val varchar)");
+
+        String data = IntStream.range(0, 1000).mapToObj((i) -> "(" + i + "," + i + ")").collect(joining(", "));
+        String insertSql = "INSERT INTO person (id, val) VALUES " + data;
+
+        sql(insertSql);
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "person p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                Collection<? extends RunningQuery> queries = engine.runningQueries();
+
+                return !queries.isEmpty() && F.first(queries).state() == QueryState.EXECUTING;
+            },
+            TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals(1, running.size());
+
+        RunningQuery qry = F.first(running);
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(100), IgniteSQLException.class, "The query was cancelled while executing.");
+    }
+
+    /** */
+    @Test
+    public void testCancelByRemoteFragment() throws IgniteCheckedException {

Review comment:
       Test fails with an error:
   java.lang.AssertionError
   	at org.junit.Assert.fail(Assert.java:86)
   	at org.junit.Assert.assertTrue(Assert.java:41)
   	at org.junit.Assert.assertTrue(Assert.java:52)
   	at org.apache.ignite.internal.processors.query.calcite.integration.RunningQueriesIntegrationTest.testCancelByRemoteFragment(RunningQueriesIntegrationTest.java:149)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.apache.ignite.testframework.junits.GridAbstractTest$7.run(GridAbstractTest.java:2432)
   	at java.lang.Thread.run(Thread.java:748)

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<RowT> implements RunningQuery {
+    /** Completable futures empty array. */
+    private static final CompletableFuture<?>[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture<?>[0];
+
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Object mux = new Object();
+
+    /** */
+    protected final Set<RunningFragment<RowT>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query<RowT>> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INITED;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query<RowT>> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        for (RunningFragment<RowT> frag : fragments) {
+            CompletableFuture<?> f = frag.context().submit(frag.root()::close, frag.root()::onError);
+
+            CompletableFuture<?> fCancel = f.thenApply((u) -> frag.context().submit(frag.context()::cancel, frag.root()::onError));

Review comment:
       Line longer than 120 symbols.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RemoteFragmentKey.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.UUID;
+
+/** */
+final class RemoteFragmentKey {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private final long fragmentId;
+
+    /** */
+    RemoteFragmentKey(UUID nodeId, long fragmentId) {

Review comment:
       Not public?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RemoteFragmentKey.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.UUID;
+
+/** */

Review comment:
       JavaDoc

##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> qry.state() == QueryState.PLANNING, TIMEOUT_IN_MS));
+
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(0), IgniteSQLException.class, "The query was cancelled while planning");
+    }
+
+    /** */
+    @Test
+    public void testCancelAtExecutionPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 6;
+
+        sql("CREATE TABLE person (id int, val varchar)");
+
+        String data = IntStream.range(0, 1000).mapToObj((i) -> "(" + i + "," + i + ")").collect(joining(", "));
+        String insertSql = "INSERT INTO person (id, val) VALUES " + data;
+
+        sql(insertSql);
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "person p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                Collection<? extends RunningQuery> queries = engine.runningQueries();
+
+                return !queries.isEmpty() && F.first(queries).state() == QueryState.EXECUTING;
+            },
+            TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals(1, running.size());
+
+        RunningQuery qry = F.first(running);
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(100), IgniteSQLException.class, "The query was cancelled while executing.");
+    }
+
+    /** */

Review comment:
       JavaDoc

##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> qry.state() == QueryState.PLANNING, TIMEOUT_IN_MS));
+
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(0), IgniteSQLException.class, "The query was cancelled while planning");
+    }
+
+    /** */
+    @Test
+    public void testCancelAtExecutionPhase() throws IgniteCheckedException {

Review comment:
       Test fails with an error:
   org.apache.ignite.internal.processors.query.calcite.integration.RunningQueriesIntegrationTest.lambda$testCancelAtExecutionPhase$8(RunningQueriesIntegrationTest.java:107)
   	at org.apache.ignite.testframework.GridTestUtils.lambda$runAsync$3(GridTestUtils.java:1175)
   	at org.apache.ignite.testframework.GridTestUtils$7.call(GridTestUtils.java:1496)
   	at org.apache.ignite.testframework.GridTestThread.run(GridTestThread.java:88)
   Caused by: java.lang.NullPointerException
   	at org.apache.ignite.internal.processors.query.calcite.exec.ExecutionServiceImpl.executeFragment(ExecutionServiceImpl.java:604)
   	at org.apache.ignite.internal.processors.query.calcite.exec.ExecutionServiceImpl.onMessage(ExecutionServiceImpl.java:654)
   	at org.apache.ignite.internal.processors.query.calcite.exec.ExecutionServiceImpl.lambda$init$2(ExecutionServiceImpl.java:400)
   	at org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl.onMessageInternal(MessageServiceImpl.java:276)
   	at org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl.lam

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -375,46 +352,22 @@ public ClosableIteratorsHolder iteratorsHolder() {
         return iteratorsHolder;
     }
 
-    /** {@inheritDoc} */
-    @Override public List<FieldsQueryCursor<List<?>>> executeQuery(
-        @Nullable QueryContext ctx,
-        String schema,
-        String qry,
-        Object[] params
-    ) {
-        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(getDefaultSchema(schema).getName(), qry));
-        if (plan != null) {
-            PlanningContext pctx = createContext(ctx, schema, qry, params);
-
-            return Collections.singletonList(executePlan(UUID.randomUUID(), pctx, plan));
-        }
-
-        SqlNodeList qryList = Commons.parse(qry, FRAMEWORK_CONFIG.getParserConfig());
-        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
-
-        for (final SqlNode qry0: qryList) {
-            final PlanningContext pctx = createContext(ctx, schema, qry0.toString(), params);
-
-            if (qryList.size() == 1) {
-                plan = queryPlanCache().queryPlan(
-                    new CacheKey(pctx.schemaName(), pctx.query()),
-                    () -> prepareSingle(qry0, pctx));
-            }
-            else
-                plan = prepareSingle(qry0, pctx);
-
-            cursors.add(executePlan(UUID.randomUUID(), pctx, plan));
-        }
+    /** */
+    public QueryRegistry queryRegistry() {

Review comment:
       No usages found.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PrepareServiceImpl.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlDdl;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlSqlToCommandConverter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.internal.util.typedef.T2;
+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.prepare.PlannerHelper.optimize;
+
+/**
+ *
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class PrepareServiceImpl extends AbstractService implements PrepareService {
+    /** */
+    private final DdlSqlToCommandConverter ddlConverter;
+
+    /**
+     * @param ctx Kernal.
+     */
+    public PrepareServiceImpl(GridKernalContext ctx) {
+        super(ctx);
+
+        ddlConverter = new DdlSqlToCommandConverter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        super.onStart(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) {
+        try {
+            assert single(sqlNode);
+
+            ctx.planner().reset();
+
+            if (SqlKind.DDL.contains(sqlNode.getKind()))
+                return prepareDdl(sqlNode, ctx);
+
+            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);
+
+                default:
+                    throw new IgniteSQLException("Unsupported operation [" +
+                        "sqlNodeKind=" + sqlNode.getKind() + "; " +
+                        "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
+        }
+        catch (ValidationException | CalciteContextException e) {
+            throw new IgniteSQLException("Failed to validate query. " + e.getMessage(), IgniteQueryErrorCode.PARSING, e);
+        }
+    }
+
+    /**
+     *
+     */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        return new DdlPlan(ddlConverter.convert((SqlDdl)sqlNode, 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 boolean single(SqlNode sqlNode) {
+        return !(sqlNode instanceof SqlNodeList);
+    }
+
+    /** */
+    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(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(fragments);
+
+        return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
+    }
+
+    /** */
+    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);

Review comment:
       New line before return.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735610335



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryState.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/** */
+public enum QueryState {
+    /** */
+    INIT,
+
+    /** */
+    PLANNING,
+
+    /** */
+    MAPPING,
+
+    /** */
+    EXECUTION,

Review comment:
       ```suggestion
       EXECUTING,
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735617862



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
##########
@@ -143,16 +158,17 @@ public MessageService messageService() {
         taskExecutor(proc.taskExecutor());
         mailboxRegistry(proc.mailboxRegistry());
         messageService(proc.messageService());
+        queryRegistry(proc.queryRegistry());
 
         init();
     }
 
     /** {@inheritDoc} */
     @Override public void init() {
         messageService().register((n, m) -> onMessage(n, (InboxCloseMessage) m), MessageType.QUERY_INBOX_CANCEL_MESSAGE);
-        messageService().register((n, m) -> onMessage(n, (OutboxCloseMessage) m), MessageType.QUERY_OUTBOX_CANCEL_MESSAGE);

Review comment:
       looks like OutboxCloseMessage is not used anywhere. It's better to remove the class




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] alex-plekhanov commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737580193



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +622,13 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qry = qryReg.register(new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id())));

Review comment:
       Yes, I've suggested something like that. But I miss additional lookup, perhaps it's not worth it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724218538



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(

Review comment:
       Let's write good javadoc for all new code ( maybe after review)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724218538



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(

Review comment:
       Let's write good javadoc for whole class




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain merged pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain merged pull request #9476:
URL: https://github.com/apache/ignite/pull/9476


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733494056



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryState.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/** */
+public enum QueryState {
+    /** */
+    INIT,
+
+    /** */
+    PLANNING,
+
+    /** */
+    MAPPING,

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733524379



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733527708



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PrepareServiceImpl.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlDdl;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlSqlToCommandConverter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.internal.util.typedef.T2;
+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;
+
+/**
+ *
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class PrepareServiceImpl extends AbstractService {
+    /** */
+    private static final int MAX_PREPARE_THREADS = 4;

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735610086



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryState.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/** */
+public enum QueryState {
+    /** */
+    INIT,

Review comment:
       ```suggestion
       INITED,
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r725063861



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -71,12 +84,13 @@
 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.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.processors.query.calcite.util.LifecycleAware;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
 /** */
-public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
+public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine, QueryRegistry {

Review comment:
       ```suggestion
   public class CalciteQueryProcessor<RowT> extends GridProcessorAdapter implements QueryEngine, QueryRegistry<Query<RowT>> {
   ```
   Seems need to check all generics




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r725868417



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    public RootQuery childQuery(SchemaPlus schema) {
+        return new RootQuery(sql, schema, params, null, exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /** */
+    public void run(ExecutionContext<Row> ctx, MultiStepPlan plan, Node<Row> root) {
+        if (state == QueryState.CLOSED) {
+            throw new IgniteSQLException(
+                "The query was cancelled while executing.",
+                IgniteQueryErrorCode.QUERY_CANCELED
+            );
+        }
+
+        RootNode<Row> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+        rootNode.register(root);
+
+        addFragment(new RunningFragment<>(F.first(plan.fragments()).root(), rootNode, ctx));
+
+        this.root = rootNode;
+
+        for (int i = 1; i < plan.fragments().size(); i++) {
+            Fragment fragment = plan.fragments().get(i);
+            List<UUID> nodes = plan.mapping(fragment).nodeIds();
+
+            remotes.addAll(nodes);
+
+            for (UUID node : nodes)
+                waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+        }
+
+        state = QueryState.EXECUTION;
+    }
+
+    /**
+     * Can be called multiple times after receive each error
+     * at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+     */
+    @Override protected void tryClose() {
+        QueryState state0 = null;
+
+        synchronized (this) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.PLANNING) {
+                state = QueryState.CLOSED;
+
+                return;
+            }
+
+            if (state == QueryState.EXECUTION)
+                state0 = state = QueryState.CLOSING;
+
+            root.closeInternal();
+
+            if (state == QueryState.CLOSING && waiting.isEmpty())
+                state0 = state = QueryState.CLOSED;
+        }
+
+        if (state0 == QueryState.CLOSED) {
+            try {
+                IgniteException wrpEx = null;
+
+                for (UUID nodeId : remotes) {
+                    try {
+                        if (!nodeId.equals(root.context().localNodeId()))
+                            exch.closeQuery(nodeId, id());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (wrpEx == null)
+                            wrpEx = new IgniteException("Failed to send cancel message. [nodeId=" + nodeId + ']', e);
+                        else
+                            wrpEx.addSuppressed(e);
+                    }
+                }
+
+                if (wrpEx != null)
+                    throw wrpEx;
+            }
+            finally {
+                super.tryClose();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        cancel.cancel();
+
+        tryClose();
+    }
+
+    /** */
+    public PlanningContext planningContext() {
+        if (pctx == null) {
+            state = QueryState.PLANNING;
+
+            pctx = PlanningContext.builder()
+                .parentContext(ctx)
+                .query(sql)
+                .parameters(params)
+                .build();
+
+            try {
+                cancel.add(() -> pctx.unwrap(CancelFlag.class).requestCancel());
+            }
+            catch (QueryCancelledException e) {
+                throw new IgniteSQLException(e.getMessage(), IgniteQueryErrorCode.QUERY_CANCELED, e);
+            }
+        }
+
+        return pctx;
+    }
+
+    /** */
+    public Iterator<Row> iterator() {
+        return root;
+    }
+
+    /** */
+    public void onNodeLeft(UUID nodeId) {
+        List<RemoteFragmentKey> fragments = null;
+
+        synchronized (this) {
+            for (RemoteFragmentKey fragment : waiting) {
+                if (!fragment.nodeId().equals(nodeId))
+                    continue;
+
+                if (fragments == null)
+                    fragments = new ArrayList<>();
+
+                fragments.add(fragment);
+            }

Review comment:
       ```suggestion
   fragments = waiting.stream().filter(f -> f.nodeId().equals(nodeId)).collect(Collectors.toList());```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723893224



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    public RootQuery childQuery(SchemaPlus schema) {
+        return new RootQuery(sql, schema, params, null, exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /** */
+    public void run(ExecutionContext<Row> ctx, MultiStepPlan plan, Node<Row> root) {
+        if (state == QueryState.CLOSED) {
+            throw new IgniteSQLException(
+                "The query was cancelled while executing.",
+                IgniteQueryErrorCode.QUERY_CANCELED
+            );
+        }
+
+        RootNode<Row> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+        rootNode.register(root);

Review comment:
       can we obtain the situation when one thread T1 executes here while T2 tries to call tryClose() ? Logic with volatile and syncronized on _state_ is complicated, can we rewrite all these stuff with AtomicRef ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733437476



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +618,15 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qryNew = new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id()));
+
+            Query<Row> qry = qryReg.register(qryNew);

Review comment:
       Because RootQuery cannot be register twice with the similar ID,




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733490716



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +375,36 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Query register(Query qry) {
+        Query old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query query(UUID id) {
+        return runningQrys.get(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(UUID id) {
+        runningQrys.remove(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Query> runningQueries() {
+        return runningQrys.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       The CalciteQueryProcessor and QueryRegistry are separated with introduce `QueryRegistryImpl`. I guess it partially  solves the ugly code at this place.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735630881



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RunningFragment.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Downstream;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/** */
+public class RunningFragment<Row> {
+    /** */
+    private final IgniteRel rootRel;
+
+    /** */
+    private final AbstractNode<Row> root;
+
+    /** */
+    private final ExecutionContext<Row> ectx;
+
+    /** */
+    public RunningFragment(
+        IgniteRel rootRel,

Review comment:
       seems rootRel could be safely deleted




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r738135801



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(
+                    qry,
+                    plan
+                ));
+            }
+                catch (Exception e) {
+                qryReg.unregister(qry.id());
+
+                if (qry.isCancelled())
+                    throw new IgniteSQLException("The query was cancelled while planning", IgniteQueryErrorCode.QUERY_CANCELED, e);

Review comment:
       Not catch an idea. Now other exception is rethrown here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744706123



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(
+            UUID.randomUUID(),
+            null,
+            qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null,
+            exch,
+            unregister,
+            log
+        );
+
+        this.sql = sql;
+        this.params = params;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            state = QueryState.MAPPING;
+        }
+    }
+
+    /**
+     * Starts execution phase for the query and setup remote fragments.
+     */
+    public void run(ExecutionContext<RowT> ctx, MultiStepPlan plan, Node<RowT> root) {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            RootNode<RowT> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+            rootNode.register(root);
+
+            addFragment(new RunningFragment<>(F.first(plan.fragments()).root(), rootNode, ctx));
+
+            this.root = rootNode;
+
+            for (int i = 1; i < plan.fragments().size(); i++) {
+                Fragment fragment = plan.fragments().get(i);
+                List<UUID> nodes = plan.mapping(fragment).nodeIds();
+
+                remotes.addAll(nodes);
+
+                for (UUID node : nodes)
+                    waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+            }
+
+            state = QueryState.EXECUTING;
+        }
+    }
+
+    /**
+     * Can be called multiple times after receive each error
+     * at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+     */
+    @Override protected void tryClose() {
+        QueryState state0 = null;
+
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.INITED || state == QueryState.PLANNING) {
+                state = QueryState.CLOSED;
+
+                return;
+            }
+
+            if (state == QueryState.EXECUTING)
+                state0 = state = QueryState.CLOSING;
+
+            root.closeInternal();
+
+            if (state == QueryState.CLOSING && waiting.isEmpty())
+                state0 = state = QueryState.CLOSED;
+        }
+
+        if (state0 == QueryState.CLOSED) {
+            try {
+                IgniteException wrpEx = null;
+
+                for (UUID nodeId : remotes) {
+                    try {
+                        if (!nodeId.equals(root.context().localNodeId()))
+                            exch.closeQuery(nodeId, id());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (wrpEx == null)
+                            wrpEx = new IgniteException("Failed to send cancel message. [nodeId=" + nodeId + ']', e);
+                        else
+                            wrpEx.addSuppressed(e);
+                    }
+                }
+
+                if (wrpEx != null)
+                    throw wrpEx;
+            }
+            finally {
+                super.tryClose();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        cancel.cancel();
+
+        tryClose();
+    }
+
+    /** */
+    public PlanningContext planningContext() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            if (pctx == null) {
+                state = QueryState.PLANNING;

Review comment:
       Flow is fixed here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723893224



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    public RootQuery childQuery(SchemaPlus schema) {
+        return new RootQuery(sql, schema, params, null, exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /** */
+    public void run(ExecutionContext<Row> ctx, MultiStepPlan plan, Node<Row> root) {
+        if (state == QueryState.CLOSED) {
+            throw new IgniteSQLException(
+                "The query was cancelled while executing.",
+                IgniteQueryErrorCode.QUERY_CANCELED
+            );
+        }
+
+        RootNode<Row> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+        rootNode.register(root);

Review comment:
       can we obtain the situation when one thread T1 executes here while T2 tries to call tryClose() ? Logic with volatile and syncronized on _state_ is complicated, can we rewrite all these stuff with AtomicRef ?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface QueryRegistry extends Service {

Review comment:
       we must specify type : public interface QueryRegistry<Row> 
   Query register(Query<Row> qry); and so on .. isn`t it ?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface QueryRegistry extends Service {

Review comment:
       we must specify type : public interface QueryRegistry\<Row\> 
   Query register(Query\<Row\> qry); and so on .. isn`t it ?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +618,15 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qryNew = new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id()));
+
+            Query<Row> qry = qryReg.register(qryNew);

Review comment:
       contract of register method shows that it can return query by uuid if such a query already registered, in such a case we can potentially obtain a problems here, do we need such contract ? I found that return state of register is used only here.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LocalQueryLazyTest.java
##########
@@ -145,6 +145,24 @@ public void testTableUnlockOnNotFinishedQuery() throws Exception {
         }
     }
 
+    /** */
+    @Test
+    public void testDbg() throws Exception {

Review comment:
       If we still need some Dbg for local usage, it`s ok for me to remove @Test annotation and rename for most information.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +375,36 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Query register(Query qry) {
+        Query old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query query(UUID id) {
+        return runningQrys.get(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(UUID id) {
+        runningQrys.remove(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Query> runningQueries() {
+        return runningQrys.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       runningQuery - returns interface RunningQuery, but runningQueries return implementation, looks like problem in API. Correct me if i`m wrong.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryState.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/** */
+public enum QueryState {
+    /** */
+    INIT,
+
+    /** */
+    PLANNING,
+
+    /** */
+    MAPPING,

Review comment:
       I found no usage of such a state.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +375,36 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Query register(Query qry) {
+        Query old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query query(UUID id) {
+        return runningQrys.get(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(UUID id) {
+        runningQrys.remove(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Query> runningQueries() {
+        return runningQrys.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       But public interface QueryEngine still have :
   Collection<? extends RunningQuery> runningQueries();

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -17,7 +17,10 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.util.Collection;
+import java.util.Collections;

Review comment:
       unused

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
##########
@@ -269,6 +269,11 @@ private static Object affinityIdentity(CacheConfiguration<?, ?> ccfg) {
         rebuild();
     }
 
+    /** {@inheritDoc} */

Review comment:
       What`s the purpose of such a functionality ? No java doc found ^(




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724215062



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),

Review comment:
       Let's calculate schema just once ( see at 303 and 330 lines)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723977228



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +375,36 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Query register(Query qry) {
+        Query old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query query(UUID id) {
+        return runningQrys.get(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(UUID id) {
+        runningQrys.remove(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Query> runningQueries() {
+        return runningQrys.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       But public interface QueryEngine still have :
   Collection<? extends RunningQuery> runningQueries();




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r725945504



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PrepareServiceImpl.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.SqlDdl;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlSqlToCommandConverter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.internal.util.typedef.T2;
+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;
+
+/**
+ *
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class PrepareServiceImpl extends AbstractService {
+    /** */
+    private static final int MAX_PREPARE_THREADS = 4;

Review comment:
       no any usages




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724250920



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(
+                qry,
+                plan
+            ));
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
+            try {
+                if (qryList.size() == 1) {

Review comment:
       why we cache only one-statement queries?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723957414



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LocalQueryLazyTest.java
##########
@@ -145,6 +145,24 @@ public void testTableUnlockOnNotFinishedQuery() throws Exception {
         }
     }
 
+    /** */
+    @Test
+    public void testDbg() throws Exception {

Review comment:
       If we still need some Dbg for local usage, it`s ok for me to remove @Test annotation and rename for most information.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723901987



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface QueryRegistry extends Service {

Review comment:
       we must specify type : public interface QueryRegistry\<Row\> 
   Query register(Query\<Row\> qry); and so on .. isn`t it ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737547805



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
##########
@@ -354,4 +347,21 @@ public boolean cancel() {
     public boolean isCancelled() {
         return cancelFlag.get();
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        ExecutionContext<?> context = (ExecutionContext<?>)o;
+
+        return qryId.equals(context.qryId) && fragmentDesc.fragmentId() == context.fragmentDesc.fragmentId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {

Review comment:
       RunningFragment is compared by ExecutionContext. 
   `Query#fragments` is a Set.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744559363



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/** */
+public class Query<RowT> implements RunningQuery {
+    /** Completable futures empty array. */
+    private static final CompletableFuture<?>[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture<?>[0];
+
+    /** */
+    private final UUID initNodeId;
+
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Object mux = new Object();
+
+    /** */
+    protected final Set<RunningFragment<RowT>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query<RowT>> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INITED;
+
+    /** */
+    protected final ExchangeService exch;
+
+    /** Logger. */
+    protected final IgniteLogger log;
+
+    /** */
+    public Query(
+        UUID id,
+        UUID initNodeId,
+        GridQueryCancel cancel,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        this.id = id;
+        this.unregister = unregister;
+        this.initNodeId = initNodeId;
+        this.exch = exch;
+        this.log = log;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    public UUID initiatorNodeId() {
+        return initNodeId;
+    }
+
+    /** */
+    protected void tryClose() {
+        List<CompletableFuture<?>> futs = new ArrayList<>();
+        
+        for (RunningFragment<RowT> frag : fragments) {
+            CompletableFuture<?> f = frag.context().submit(frag.root()::close, frag.root()::onError);
+
+            CompletableFuture<?> fCancel = f.thenApply((u) -> frag.context().submit(frag.context()::cancel, frag.root()::onError));
+            futs.add(fCancel);
+        }
+
+        CompletableFuture.allOf(futs.toArray(COMPLETABLE_FUTURES_EMPTY_ARRAY))
+            .thenAccept((u) -> unregister.accept(this));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+            
+            if (state == QueryState.INITED) {
+                state = QueryState.CLOSING;
+
+                try {
+                    exch.closeQuery(initNodeId, id);
+
+                    return;
+                }
+                catch (IgniteCheckedException e) {
+                    log.warning("Cannot send cancel request to query initiator", e);
+                }
+            }
+
+            if (state == QueryState.EXECUTING || state == QueryState.CLOSING)
+                state = QueryState.CLOSED;
+        }
+
+        for (RunningFragment<RowT> frag : fragments)
+            frag.context().execute(() -> frag.root().onError(new ExecutionCancelledException()), frag.root()::onError);
+
+        tryClose();
+    }
+
+    /** */
+    public void addFragment(RunningFragment<RowT> f) {
+        if (state == QueryState.INITED) {
+            synchronized (mux) {
+                if (state == QueryState.INITED)
+                    state = QueryState.EXECUTING;
+
+                if (state == QueryState.CLOSING || state == QueryState.CLOSED) {
+                    throw new IgniteSQLException(
+                        "The query was cancelled",
+                        IgniteQueryErrorCode.QUERY_CANCELED,
+                        new ExecutionCancelledException()
+                    );
+                }
+            }
+        }
+
+        fragments.add(f);

Review comment:
       is it OK if new fragment will be added in `CLOSING` or `CLOSED` state? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735616491



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
##########
@@ -19,31 +19,22 @@
 
 import java.util.List;
 import java.util.UUID;
-
 import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.calcite.RootQuery;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlan;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
-import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
-public interface ExecutionService extends Service {
-    /**
-     * Executes a query.
-     *
-     * @param ctx Query external context, contains flags and connection settings like a locale or a timezone.
-     * @param schema Schema name.
-     * @param query Query.
-     * @param params Query parameters.
-     * @return Query cursor.
-     */
-    List<FieldsQueryCursor<List<?>>> executeQuery(@Nullable QueryContext ctx, String schema, String query, Object[] params);
-
+public interface ExecutionService<Row> extends Service {
     /**
      * Cancels a running query.
      *
      * @param queryId Query ID.
      */
     void cancelQuery(UUID queryId);
-}
+
+    /** */
+    FieldsQueryCursor<List<?>> executePlan(RootQuery<Row> qry, QueryPlan plan);
+}

Review comment:
       new line is required at the end




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735601113



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -744,51 +556,54 @@ private FieldsMetadata explainFieldsMetadata(PlanningContext ctx) {
             Throwable ex = null;
             for (UUID nodeId : fragmentDesc.nodeIds()) {
                 if (ex != null)
-                    info.onResponse(nodeId, fragment.fragmentId(), ex);
+                    qry.onResponse(nodeId, fragment.fragmentId(), ex);
                 else {
                     try {
                         QueryStartRequest req = new QueryStartRequest(
-                            qryId,
-                            qctx.schemaName(),
+                            qry.id(),
+                            qry.context().schemaName(),
                             fragment.serialized(),
                             ectx.topologyVersion(),
                             fragmentDesc,
-                            params);
+                            qry.parameters());
 
                         messageService().send(nodeId, req);
                     }
                     catch (Throwable e) {
-                        info.onResponse(nodeId, fragment.fragmentId(), ex = e);
+                        qry.onResponse(nodeId, fragment.fragmentId(), ex = e);
                     }
                 }
             }
         }
 
-        return new ListFieldsQueryCursor<>(plan, info.iterator(), ectx);
+        return new ListFieldsQueryCursor<>(plan, iteratorsHolder().iterator(qry.iterator()), ectx);
     }
 
     /** */
-    private FieldsQueryCursor<List<?>> executeExplain(ExplainPlan plan, PlanningContext pctx) {
+    private FieldsQueryCursor<List<?>> executeExplain(ExplainPlan plan) {
         QueryCursorImpl<List<?>> cur = new QueryCursorImpl<>(singletonList(singletonList(plan.plan())));
-        cur.fieldsMeta(plan.fieldsMeta().queryFieldsMetadata(pctx.typeFactory()));
+        cur.fieldsMeta(plan.fieldsMeta().queryFieldsMetadata(Commons.typeFactory()));
 
         return cur;
     }
 
     /** */
-    private void executeFragment(UUID qryId, FragmentPlan plan, ExecutionContext<Row> ectx) {
+    private void executeFragment(Query<Row> qry, FragmentPlan plan, ExecutionContext<Row> ectx) {
         UUID origNodeId = ectx.originatingNodeId();
 
         Outbox<Row> node = new LogicalRelImplementor<>(
-                ectx,
-                partitionService(),
-                mailboxRegistry(),
-                exchangeService(),
-                failureProcessor())
-                .go(plan.root());
+            ectx,
+            partitionService(),
+            mailboxRegistry(),
+            exchangeService(),
+            failureProcessor())
+            .go(plan.root()
+        );

Review comment:
       ```suggestion
               failureProcessor()
           )
               .go(plan.root());
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744512065



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -36,4 +39,10 @@
      */
     List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String schemaName, String qry, Object... params)
         throws IgniteSQLException;
+
+    /** */
+    Collection<? extends RunningQuery> runningQueries();

Review comment:
       I see no problem with it. Looks like the described case is in the scope of the `KILL <query>` command implementation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723977798



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -17,7 +17,10 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.util.Collection;
+import java.util.Collections;

Review comment:
       unused




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733671047



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (this) {

Review comment:
       is it possible that we will take a lock on a different objects? THIS for RootQuery and for Query could be not the same.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733522447



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -71,12 +84,13 @@
 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.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.processors.query.calcite.util.LifecycleAware;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
 /** */
-public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
+public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine, QueryRegistry {

Review comment:
       Not catch an idea. It is a GridComponent, how it will work with generics?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735784271



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RunningFragment.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Downstream;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/** */
+public class RunningFragment<Row> {
+    /** */
+    private final IgniteRel rootRel;
+
+    /** */
+    private final AbstractNode<Row> root;
+
+    /** */
+    private final ExecutionContext<Row> ectx;
+
+    /** */
+    public RunningFragment(
+        IgniteRel rootRel,

Review comment:
       I guess it will be very helpful to print info about running queries.
   But if you insist, then I'll remove it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724215062



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),

Review comment:
       Let's calculate schema just once ( see at 303 lines)

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(

Review comment:
       Let's write good javadoc for whole class

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(

Review comment:
       Let's write good javadoc for whole class ( maybe after review)

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),

Review comment:
       Let's calculate schema just once ( see at 303 and 330 lines)

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(
+                qry,
+                plan
+            ));
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
+            try {
+                if (qryList.size() == 1) {

Review comment:
       why we cache only one-statement queries?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(
+                qry,
+                plan
+            ));
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
+            try {
+                if (qryList.size() == 1) {

Review comment:
       Also, we can check the cache for each statement in multistatement queries

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {

Review comment:
       I propose to use 'T' at the end of the name of all generic parameters, to have fewer issues  during merge it to 3.0

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {
+    public static final CompletableFuture[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture[0];

Review comment:
       seems could be private




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] alex-plekhanov commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r738118723



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(

Review comment:
       I think we can inject some mock objects to make barriers. For example, get schema via `CalciteQueryProcessor.schemaHolder()` and mock table `toRel` or `scan` methods. 
   For EXECUTING phase barrier also can be inserted via own thread pool `(CalciteQueryProcessor.taskExecutor().stripedThreadPoolExecutor(...))`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744532759



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -36,4 +39,10 @@
      */
     List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String schemaName, String qry, Object... params)
         throws IgniteSQLException;
+
+    /** */
+    Collection<? extends RunningQuery> runningQueries();

Review comment:
       I guess internal API should represent the modules abilities.
   We can introduce the method `QueryEngine#kill(UUID queryId)` when this functionality will be available.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744520712



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -36,4 +39,10 @@
      */
     List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String schemaName, String qry, Object... params)
         throws IgniteSQLException;
+
+    /** */
+    Collection<? extends RunningQuery> runningQueries();

Review comment:
       then why do we need a limited API at all?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] alex-plekhanov commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r737200735



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> qry.state() == QueryState.PLANNING, TIMEOUT_IN_MS));
+
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(0), IgniteSQLException.class, "The query was cancelled while planning");
+    }
+
+    /** */
+    @Test
+    public void testCancelAtExecutionPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 6;
+
+        sql("CREATE TABLE person (id int, val varchar)");
+
+        String data = IntStream.range(0, 1000).mapToObj((i) -> "(" + i + "," + i + ")").collect(joining(", "));
+        String insertSql = "INSERT INTO person (id, val) VALUES " + data;
+
+        sql(insertSql);
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "person p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                Collection<? extends RunningQuery> queries = engine.runningQueries();
+
+                return !queries.isEmpty() && F.first(queries).state() == QueryState.EXECUTING;
+            },
+            TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals(1, running.size());
+
+        RunningQuery qry = F.first(running);
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(100), IgniteSQLException.class, "The query was cancelled while executing.");
+    }
+
+    /** */
+    @Test
+    public void testCancelByRemoteFragment() throws IgniteCheckedException {
+        QueryEngine clientEngine = queryProcessor(client);
+        QueryEngine serverEngine = queryProcessor(srv);
+        int cnt = 6;
+
+        sql("CREATE TABLE t (id int, val varchar)");
+
+        String data = IntStream.range(0, 10000).mapToObj((i) -> "(" + i + ",'" + i + "')").collect(joining(", "));
+        String insertSql = "INSERT INTO t (id, val) VALUES " + data;
+
+        sql(insertSql);
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "t t" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                Collection<? extends RunningQuery> queries = clientEngine.runningQueries();
+
+                return !queries.isEmpty() && F.first(queries).state() == QueryState.EXECUTING;
+            },
+            TIMEOUT_IN_MS));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(() -> !serverEngine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = serverEngine.runningQueries();
+        RunningQuery qry = F.first(running);
+
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                log.info("+++ " + F.first(clientEngine.runningQueries()));

Review comment:
       looks redundant

##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> qry.state() == QueryState.PLANNING, TIMEOUT_IN_MS));
+
+        qry.cancel();
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> engine.runningQueries().isEmpty(), TIMEOUT_IN_MS));
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(0), IgniteSQLException.class, "The query was cancelled while planning");
+    }
+
+    /** */
+    @Test
+    public void testCancelAtExecutionPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 6;
+
+        sql("CREATE TABLE person (id int, val varchar)");
+
+        String data = IntStream.range(0, 1000).mapToObj((i) -> "(" + i + "," + i + ")").collect(joining(", "));
+        String insertSql = "INSERT INTO person (id, val) VALUES " + data;
+
+        sql(insertSql);
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "person p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> {
+                Collection<? extends RunningQuery> queries = engine.runningQueries();
+
+                return !queries.isEmpty() && F.first(queries).state() == QueryState.EXECUTING;

Review comment:
       The same as for `testCancelAtPlanningPhase`, no guarantees to catch the execution phase.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            state = QueryState.MAPPING;
+        }
+    }
+
+    /**
+     * Starts execution phase for the query and setup remote fragments.
+     */
+    public void run(ExecutionContext<RowT> ctx, MultiStepPlan plan, Node<RowT> root) {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            RootNode<RowT> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+            rootNode.register(root);
+
+            addFragment(new RunningFragment<>(F.first(plan.fragments()).root(), rootNode, ctx));
+
+            this.root = rootNode;
+
+            for (int i = 1; i < plan.fragments().size(); i++) {
+                Fragment fragment = plan.fragments().get(i);
+                List<UUID> nodes = plan.mapping(fragment).nodeIds();
+
+                remotes.addAll(nodes);
+
+                for (UUID node : nodes)
+                    waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+            }
+
+            state = QueryState.EXECUTING;
+        }
+    }
+
+    /**
+     * Can be called multiple times after receive each error
+     * at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+     */
+    @Override protected void tryClose() {
+        QueryState state0 = null;
+
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.INITED || state == QueryState.PLANNING) {
+                state = QueryState.CLOSED;
+
+                return;
+            }
+
+            if (state == QueryState.EXECUTING)
+                state0 = state = QueryState.CLOSING;
+
+            root.closeInternal();
+
+            if (state == QueryState.CLOSING && waiting.isEmpty())
+                state0 = state = QueryState.CLOSED;
+        }
+
+        if (state0 == QueryState.CLOSED) {
+            try {
+                IgniteException wrpEx = null;
+
+                for (UUID nodeId : remotes) {
+                    try {
+                        if (!nodeId.equals(root.context().localNodeId()))
+                            exch.closeQuery(nodeId, id());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (wrpEx == null)
+                            wrpEx = new IgniteException("Failed to send cancel message. [nodeId=" + nodeId + ']', e);
+                        else
+                            wrpEx.addSuppressed(e);
+                    }
+                }
+
+                if (wrpEx != null)
+                    throw wrpEx;
+            }
+            finally {
+                super.tryClose();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        cancel.cancel();
+
+        tryClose();
+    }
+
+    /** */
+    public PlanningContext planningContext() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            if (pctx == null) {
+                state = QueryState.PLANNING;
+
+                pctx = PlanningContext.builder()
+                    .parentContext(ctx)
+                    .query(sql)
+                    .parameters(params)
+                    .build();
+
+                try {
+                    cancel.add(() -> pctx.unwrap(CancelFlag.class).requestCancel());
+                }
+                catch (QueryCancelledException e) {
+                    throw new IgniteSQLException(e.getMessage(), IgniteQueryErrorCode.QUERY_CANCELED, e);
+                }
+            }
+
+            return pctx;
+        }
+    }
+
+    /** */
+    public Iterator<RowT> iterator() {
+        return root;
+    }
+
+    /** */
+    public void onNodeLeft(UUID nodeId) {
+        List<RemoteFragmentKey> fragments = null;
+
+        synchronized (mux) {
+            fragments = waiting.stream().filter(f -> f.nodeId().equals(nodeId)).collect(Collectors.toList());
+        }
+
+        if (!F.isEmpty(fragments)) {
+            ClusterTopologyCheckedException ex = new ClusterTopologyCheckedException(
+                "Failed to start query, node left. nodeId=" + nodeId);
+
+            for (RemoteFragmentKey fragment : fragments)
+                onResponse(fragment, ex);
+        }
+    }
+
+    /** */
+    public void onResponse(UUID nodeId, long fragmentId, Throwable error) {
+        onResponse(new RemoteFragmentKey(nodeId, fragmentId), error);
+    }
+
+    /** */
+    private void onResponse(RemoteFragmentKey fragment, Throwable error) {
+        QueryState state;
+        synchronized (mux) {
+            waiting.remove(fragment);
+
+            state = this.state;
+        }
+
+        if (error != null)
+            onError(error);
+        else if (state == QueryState.CLOSING)
+            tryClose();
+    }
+
+    /** */
+    public void onError(Throwable error) {
+        root.onError(error);
+
+        tryClose();
+    }
+
+    /** */
+    public static PlanningContext createPlanningContext(BaseQueryContext ctx, String qry, Object[] params) {

Review comment:
       There are no usages of this method

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(

Review comment:
       Let's add `<Object[]>` to `executionSvc` declaration, to avoid warnings here 

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<RowT> implements RunningQuery {
+    /** Completable futures empty array. */
+    private static final CompletableFuture<?>[] COMPLETABLE_FUTURES_EMPTY_ARRAY = new CompletableFuture<?>[0];
+
+    /** */
+    private final UUID id;
+
+    /** */
+    protected final Object mux = new Object();
+
+    /** */
+    protected final Set<RunningFragment<RowT>> fragments;
+
+    /** */
+    protected final GridQueryCancel cancel;
+
+    /** */
+    protected final Consumer<Query<RowT>> unregister;
+
+    /** */
+    protected volatile QueryState state = QueryState.INITED;
+
+    /** */
+    public Query(UUID id, GridQueryCancel cancel, Consumer<Query<RowT>> unregister) {
+        this.id = id;
+        this.unregister = unregister;
+
+        this.cancel = cancel != null ? cancel : new GridQueryCancel();
+
+        fragments = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    /** */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** */
+    @Override public QueryState state() {
+        return state;
+    }
+
+    /** */
+    public static BaseQueryContext createQueryContext(QueryContext ctx, SchemaPlus schema, IgniteLogger log) {

Review comment:
       Not used

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -828,16 +622,13 @@ private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlT
         return new FieldsMetadataImpl(resultType, origins);
     }
 
-    /** */
-    private boolean single(SqlNode sqlNode) {
-        return !(sqlNode instanceof SqlNodeList);
-    }
-
     /** */
     private void onMessage(UUID nodeId, final QueryStartRequest msg) {
         assert nodeId != null && msg != null;
 
         try {
+            Query<Row> qry = qryReg.register(new Query<>(msg.queryId(), null, (q) -> qryReg.unregister(q.id())));

Review comment:
       A little optimization is possible here. An additional instance of `Query` is created each time. Perhaps it's better to check `qryReg.query(msg.queryId())` first, and if instance not exists create new one and register. In this case, register should not return anything and `computeIfAbsent` in `register` method (with lambda creation on each call) can be replaced with `putIfAbstent` or `put`

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java
##########
@@ -45,10 +45,10 @@
     QUERY_INBOX_CANCEL_MESSAGE(305, InboxCloseMessage::new),
 
     /** */
-    QUERY_OUTBOX_CANCEL_MESSAGE(306, OutboxCloseMessage::new),
+    GENERIC_VALUE_MESSAGE(307, GenericValueMessage::new),
 
     /** */
-    GENERIC_VALUE_MESSAGE(307, GenericValueMessage::new),
+    QUERY_CLOSE_MESSAGE(308, QueryCloseMessage::new),

Review comment:
       Why not reuse 306 code?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.Collection;
+import java.util.UUID;
+
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ * Registry of the running queries.
+ */
+public interface QueryRegistry<RowT> extends Service {

Review comment:
       Perhaps it's better to work in this interface with interface `RunningQuery` instead of class `Query<RowT>`. In this case you do not require `<RowT>` generic in the `QueryRegistry` interface. All current code do not require additional casting of `RunningQuery` to `Query` except one call to the `register` method.  WDYT?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            state = QueryState.MAPPING;
+        }
+    }
+
+    /**
+     * Starts execution phase for the query and setup remote fragments.
+     */
+    public void run(ExecutionContext<RowT> ctx, MultiStepPlan plan, Node<RowT> root) {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            RootNode<RowT> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+            rootNode.register(root);
+
+            addFragment(new RunningFragment<>(F.first(plan.fragments()).root(), rootNode, ctx));
+
+            this.root = rootNode;
+
+            for (int i = 1; i < plan.fragments().size(); i++) {
+                Fragment fragment = plan.fragments().get(i);
+                List<UUID> nodes = plan.mapping(fragment).nodeIds();
+
+                remotes.addAll(nodes);
+
+                for (UUID node : nodes)
+                    waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+            }
+
+            state = QueryState.EXECUTING;
+        }
+    }
+
+    /**
+     * Can be called multiple times after receive each error
+     * at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+     */
+    @Override protected void tryClose() {
+        QueryState state0 = null;
+
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.INITED || state == QueryState.PLANNING) {
+                state = QueryState.CLOSED;
+
+                return;
+            }
+
+            if (state == QueryState.EXECUTING)
+                state0 = state = QueryState.CLOSING;
+
+            root.closeInternal();

Review comment:
       `tryClose` method can be executed several times, why do we closing the root node on each node response (it creates task to executor each time)? It's not enough to close the root node once?   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
##########
@@ -90,15 +90,13 @@ public SqlCommandProcessor(GridKernalContext ctx, GridQuerySchemaManager schemaM
      *
      * @param sql SQL.
      * @param cmdNative Native command.
-     * @param cliCtx Client context.
      * @return Result.
      */
-    @Nullable public FieldsQueryCursor<List<?>> runCommand(String sql, SqlCommand cmdNative,
-        @Nullable SqlClientContext cliCtx) {

Review comment:
       Import should be removed too

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
##########
@@ -90,15 +90,13 @@ public SqlCommandProcessor(GridKernalContext ctx, GridQuerySchemaManager schemaM
      *
      * @param sql SQL.

Review comment:
       Parameter was removed

##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.integration;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+
+/**
+ *
+ */
+public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static IgniteEx srv;
+
+    /** Timeout in ms for async operations. */
+    private static final long TIMEOUT_IN_MS = 10_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        srv = grid(0);
+    }
+
+    /** */
+    @Test
+    public void testCancelAtPlanningPhase() throws IgniteCheckedException {
+        QueryEngine engine = queryProcessor(client);
+        int cnt = 9;
+
+        for (int i = 0; i < cnt; i++)
+            sql("CREATE TABLE test_tbl" + i + " (id int, val varchar)");
+
+        String bigJoin = IntStream.range(0, cnt).mapToObj((i) -> "test_tbl" + i + " p" + i).collect(joining(", "));
+        String sql = "SELECT * FROM " + bigJoin;
+
+        IgniteInternalFuture<List<List<?>>> fut = GridTestUtils.runAsync(() -> sql(sql));
+
+        Assert.assertTrue(GridTestUtils.waitForCondition(
+            () -> !engine.runningQueries().isEmpty() || fut.isDone(), TIMEOUT_IN_MS));
+
+        Collection<? extends RunningQuery> running = engine.runningQueries();
+
+        assertEquals("Running: " + running, 1, running.size());
+
+        RunningQuery qry = F.first(running);
+
+        // Waits for planning.
+        Assert.assertTrue(GridTestUtils.waitForCondition(

Review comment:
       There are no barriers between planning and execution, there are no guarantees that the planning phase will be caught. The test can become flaky.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -634,61 +453,57 @@ private FieldsMetadata explainFieldsMetadata(PlanningContext ctx) {
 
             case QUERY:
                 return mapAndExecutePlan(
-                    qryId,
-                    (MultiStepPlan)plan,
-                    pctx.unwrap(BaseQueryContext.class),
-                    pctx.parameters()
+                    qry,
+                    (MultiStepPlan)plan
                 );
 
             case EXPLAIN:
-                return executeExplain((ExplainPlan)plan, pctx);
+                return executeExplain((ExplainPlan)plan);
 
             case DDL:
-                return executeDdl(qryId, (DdlPlan)plan, pctx);
+                return executeDdl(qry, (DdlPlan)plan);
 
             default:
                 throw new AssertionError("Unexpected plan type: " + plan);
         }
     }
 
     /** */
-    private FieldsQueryCursor<List<?>> executeDdl(UUID qryId, DdlPlan plan, PlanningContext pctx) {
+    private FieldsQueryCursor<List<?>> executeDdl(RootQuery qry, DdlPlan plan) {

Review comment:
       `RootQuery` -> `RootQuery<Row>`

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(
+                    qry,
+                    plan
+                ));
+            }
+                catch (Exception e) {

Review comment:
       Wrong indent

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,85 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(
+                    qry,
+                    plan
+                ));
+            }
+                catch (Exception e) {
+                qryReg.unregister(qry.id());
+
+                if (qry.isCancelled())
+                    throw new IgniteSQLException("The query was cancelled while planning", IgniteQueryErrorCode.QUERY_CANCELED, e);
+                else
+                    throw e;
+            }
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        List<RootQuery<Object[]>> qrys = new ArrayList<>(qryList.size());
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.schema(schemaName), // Update schema for each query in multiple statements.
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qrys.add(qry);
+
+            qryReg.register(qry);
+            try {
+                if (qryList.size() == 1) {
+                    plan = queryPlanCache().queryPlan(
+                        new CacheKey(schemaName, qry.sql()),
+                        () -> prepareSvc.prepareSingle(sqlNode, qry.planningContext()));
+                }
+                else
+                    plan = prepareSvc.prepareSingle(sqlNode, qry.planningContext());
+
+                cursors.add(executionSvc.executePlan(qry, plan));
+            }
+            catch (Exception e) {
+                boolean isCanceled = qry.isCancelled();
+
+                qrys.forEach(RootQuery::cancel);

Review comment:
       `RootQuery.cancel` can throw an exception, in this case query will not be unregistered and will be leaked in `qryReg`. Also, I think a test should be added for multistatement queries.

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RunningFragment.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.Objects;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/** */
+public class RunningFragment<Row> {
+    /** */
+    private final IgniteRel rootRel;
+
+    /** */
+    private final AbstractNode<Row> root;
+
+    /** */
+    private final ExecutionContext<Row> ectx;
+
+    /** */
+    public RunningFragment(
+        IgniteRel rootRel,
+        AbstractNode<Row> root,
+        ExecutionContext<Row> ectx) {
+        this.rootRel = rootRel;
+        this.root = root;
+        this.ectx = ectx;
+    }
+
+    /** */
+    public ExecutionContext<Row> context() {
+        return ectx;
+    }
+
+    /** */
+    public IgniteRel rootRel() {
+        return rootRel;
+    }
+
+    /** */
+    public AbstractNode<Row> root() {
+        return root;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        RunningFragment fragment = (RunningFragment)o;

Review comment:
       `RunningFragment<Row> fragment = (RunningFragment<Row>)o;` to avoid warnings

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(UUID.randomUUID(), qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null, unregister);
+
+        this.sql = sql;
+        this.params = params;
+        this.exch = exch;
+        this.log = log;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            state = QueryState.MAPPING;
+        }
+    }
+
+    /**
+     * Starts execution phase for the query and setup remote fragments.
+     */
+    public void run(ExecutionContext<RowT> ctx, MultiStepPlan plan, Node<RowT> root) {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            RootNode<RowT> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+            rootNode.register(root);
+
+            addFragment(new RunningFragment<>(F.first(plan.fragments()).root(), rootNode, ctx));
+
+            this.root = rootNode;
+
+            for (int i = 1; i < plan.fragments().size(); i++) {
+                Fragment fragment = plan.fragments().get(i);
+                List<UUID> nodes = plan.mapping(fragment).nodeIds();
+
+                remotes.addAll(nodes);
+
+                for (UUID node : nodes)
+                    waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+            }
+
+            state = QueryState.EXECUTING;
+        }
+    }
+
+    /**
+     * Can be called multiple times after receive each error
+     * at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+     */
+    @Override protected void tryClose() {
+        QueryState state0 = null;
+
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.INITED || state == QueryState.PLANNING) {

Review comment:
       What about MAPPING state?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
##########
@@ -354,4 +347,21 @@ public boolean cancel() {
     public boolean isCancelled() {
         return cancelFlag.get();
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        ExecutionContext<?> context = (ExecutionContext<?>)o;
+
+        return qryId.equals(context.qryId) && fragmentDesc.fragmentId() == context.fragmentDesc.fragmentId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {

Review comment:
       Why do we need it? I see no maps or sets with ExecutionContext key (but perhaps I miss something)

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractQueryContext.java
##########
@@ -38,4 +38,9 @@ public AbstractQueryContext(Context parentCtx) {
 
         return parentCtx.unwrap(aCls);
     }
+
+    /** */
+    public Context parent() {

Review comment:
       Not used

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;

Review comment:
       qury -> query

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -375,46 +351,22 @@ public ClosableIteratorsHolder iteratorsHolder() {
         return iteratorsHolder;
     }
 
-    /** {@inheritDoc} */
-    @Override public List<FieldsQueryCursor<List<?>>> executeQuery(
-        @Nullable QueryContext ctx,
-        String schema,
-        String qry,
-        Object[] params
-    ) {
-        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(getDefaultSchema(schema).getName(), qry));
-        if (plan != null) {
-            PlanningContext pctx = createContext(ctx, schema, qry, params);
-
-            return Collections.singletonList(executePlan(UUID.randomUUID(), pctx, plan));
-        }
-
-        SqlNodeList qryList = Commons.parse(qry, FRAMEWORK_CONFIG.getParserConfig());
-        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
-
-        for (final SqlNode qry0: qryList) {
-            final PlanningContext pctx = createContext(ctx, schema, qry0.toString(), params);
-
-            if (qryList.size() == 1) {
-                plan = queryPlanCache().queryPlan(
-                    new CacheKey(pctx.schemaName(), pctx.query()),
-                    () -> prepareSingle(qry0, pctx));
-            }
-            else
-                plan = prepareSingle(qry0, pctx);
-
-            cursors.add(executePlan(UUID.randomUUID(), pctx, plan));
-        }
+    /** */
+    public QueryRegistry<Row> queryRegistry() {

Review comment:
       Method never used

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
##########
@@ -108,6 +113,16 @@ public MessageService messageService() {
         return msgSvc;
     }
 
+    /** */
+    public QueryRegistry queryRegistry() {

Review comment:
       Not used

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -634,61 +453,57 @@ private FieldsMetadata explainFieldsMetadata(PlanningContext ctx) {
 
             case QUERY:
                 return mapAndExecutePlan(
-                    qryId,
-                    (MultiStepPlan)plan,
-                    pctx.unwrap(BaseQueryContext.class),
-                    pctx.parameters()
+                    qry,
+                    (MultiStepPlan)plan
                 );
 
             case EXPLAIN:
-                return executeExplain((ExplainPlan)plan, pctx);
+                return executeExplain((ExplainPlan)plan);
 
             case DDL:
-                return executeDdl(qryId, (DdlPlan)plan, pctx);
+                return executeDdl(qry, (DdlPlan)plan);
 
             default:
                 throw new AssertionError("Unexpected plan type: " + plan);
         }
     }
 
     /** */
-    private FieldsQueryCursor<List<?>> executeDdl(UUID qryId, DdlPlan plan, PlanningContext pctx) {
+    private FieldsQueryCursor<List<?>> executeDdl(RootQuery qry, DdlPlan plan) {
         try {
-            ddlCmdHnd.handle(qryId, plan.command(), pctx);
+            ddlCmdHnd.handle(qry.id(), plan.command());
         }
         catch (IgniteCheckedException e) {
-            throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + pctx.query() +
+            throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + qry.sql() +
                 ", err=" + e.getMessage() + ']', e);
         }
+        finally {
+            qryReg.unregister(qry.id());
+        }
 
-        if (plan.command() instanceof CreateTableCommand && ((CreateTableCommand)plan.command()).insertStatement() != null) {
-            SqlInsert insertStmt = ((CreateTableCommand)plan.command()).insertStatement();
+        if (plan.command() instanceof CreateTableCommand
+            && ((CreateTableCommand)plan.command()).insertStatement() != null) {
+            RootQuery<Row> insQry = qry.childQuery(schemaHolder.schema(qry.context().schemaName()));

Review comment:
       This query will not be registered in query registry (at least in the planning phase) and we can't kill it. Is it ok?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RunningFragment.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Downstream;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/** */
+public class RunningFragment<Row> {
+    /** */
+    private final IgniteRel rootRel;
+
+    /** */
+    private final AbstractNode<Row> root;
+
+    /** */
+    private final ExecutionContext<Row> ectx;
+
+    /** */
+    public RunningFragment(
+        IgniteRel rootRel,

Review comment:
       Getter method is not required then




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r725095443



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(

Review comment:
       I see different approaches for execute cached singlestatement and for not cached plans. For cached singlestatement you do register and execute plan. For not cahed you added try/catch block with invoke unregister method.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723975081



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryState.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/** */
+public enum QueryState {
+    /** */
+    INIT,
+
+    /** */
+    PLANNING,
+
+    /** */
+    MAPPING,

Review comment:
       I found no usage of such a state.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724218538



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class RootQuery<Row> extends Query<Row> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<Row> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    private final ExchangeService exch;
+
+    /** */
+    public RootQuery(

Review comment:
       Let's write good javadoc for whole class ( maybe after review)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724253132



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -267,13 +293,71 @@ public FailureProcessor failureProcessor() {
             qryPlanCache,
             exchangeSvc
         );
+
+        runningQrys.clear();
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schemaHolder.getDefaultSchema(schemaName).getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+            return Collections.singletonList(executionSvc.executePlan(
+                qry,
+                plan
+            ));
+        }
+
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.getDefaultSchema(schemaName),
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> unregister(q.id()),
+                log
+            );
+
+            register(qry);
+            try {
+                if (qryList.size() == 1) {

Review comment:
       Also, we can check the cache for each statement in multistatement queries




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r724270595



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Query.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+import org.apache.ignite.internal.processors.query.RunningQuery;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/** */
+public class Query<Row> implements RunningQuery {

Review comment:
       I propose to use 'T' at the end of the name of all generic parameters, to have fewer issues  during merge it to 3.0




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r723901987



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface QueryRegistry extends Service {

Review comment:
       we must specify type : public interface QueryRegistry<Row> 
   Query register(Query<Row> qry); and so on .. isn`t it ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744512065



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
##########
@@ -36,4 +39,10 @@
      */
     List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String schemaName, String qry, Object... params)
         throws IgniteSQLException;
+
+    /** */
+    Collection<? extends RunningQuery> runningQueries();

Review comment:
       I see no problem with it. Looks like the described case is in the scope of `the KILL <query>` command implementation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r744631173



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/RootQuery.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.CancelFlag;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryState;
+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.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ * The RootQuery is created on the qury initiator (originator) node as the first step of a query run;
+ * It contains the information about query state, contexts, remote fragments;
+ * It provides 'cancel' functionality for running query like a base query class.
+ */
+public class RootQuery<RowT> extends Query<RowT> {
+    /** SQL query. */
+    private final String sql;
+
+    /** Parameters. */
+    private final Object[] params;
+
+    /** remote nodes */
+    private final Set<UUID> remotes;
+
+    /** node to fragment */
+    private final Set<RemoteFragmentKey> waiting;
+
+    /** */
+    private volatile RootNode<RowT> root;
+
+    /** */
+    private volatile PlanningContext pctx;
+
+    /** */
+    private final BaseQueryContext ctx;
+
+    /** */
+    public RootQuery(
+        String sql,
+        SchemaPlus schema,
+        Object[] params,
+        QueryContext qryCtx,
+        ExchangeService exch,
+        Consumer<Query<RowT>> unregister,
+        IgniteLogger log
+    ) {
+        super(
+            UUID.randomUUID(),
+            null,
+            qryCtx != null ? qryCtx.unwrap(GridQueryCancel.class) : null,
+            exch,
+            unregister,
+            log
+        );
+
+        this.sql = sql;
+        this.params = params;
+
+        remotes = new HashSet<>();
+        waiting = new HashSet<>();
+
+        Context parent = Commons.convert(qryCtx);
+
+        ctx = BaseQueryContext.builder()
+            .parentContext(parent)
+            .frameworkConfig(
+                Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                    .defaultSchema(schema)
+                    .build()
+            )
+            .logger(log)
+            .build();
+    }
+
+    /**
+     * Creates the new root that inherits the query parameters from {@code this} query.
+     * Is used to execute DML query immediately after (inside) DDL.
+     * e.g.:
+     *      CREATE TABLE MY_TABLE AS SELECT ... FROM ...;
+     *
+     * @param schema new schema.
+     */
+    public RootQuery<RowT> childQuery(SchemaPlus schema) {
+        return new RootQuery<>(sql, schema, params, QueryContext.of(cancel), exch, unregister, log);
+    }
+
+    /** */
+    public BaseQueryContext context() {
+        return ctx;
+    }
+
+    /** */
+    public String sql() {
+        return sql;
+    }
+
+    /** */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /**
+     * Starts maping phase for the query.
+     */
+    public void mapping() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            state = QueryState.MAPPING;
+        }
+    }
+
+    /**
+     * Starts execution phase for the query and setup remote fragments.
+     */
+    public void run(ExecutionContext<RowT> ctx, MultiStepPlan plan, Node<RowT> root) {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            RootNode<RowT> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+            rootNode.register(root);
+
+            addFragment(new RunningFragment<>(F.first(plan.fragments()).root(), rootNode, ctx));
+
+            this.root = rootNode;
+
+            for (int i = 1; i < plan.fragments().size(); i++) {
+                Fragment fragment = plan.fragments().get(i);
+                List<UUID> nodes = plan.mapping(fragment).nodeIds();
+
+                remotes.addAll(nodes);
+
+                for (UUID node : nodes)
+                    waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+            }
+
+            state = QueryState.EXECUTING;
+        }
+    }
+
+    /**
+     * Can be called multiple times after receive each error
+     * at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+     */
+    @Override protected void tryClose() {
+        QueryState state0 = null;
+
+        synchronized (mux) {
+            if (state == QueryState.CLOSED)
+                return;
+
+            if (state == QueryState.INITED || state == QueryState.PLANNING) {
+                state = QueryState.CLOSED;
+
+                return;
+            }
+
+            if (state == QueryState.EXECUTING)
+                state0 = state = QueryState.CLOSING;
+
+            root.closeInternal();
+
+            if (state == QueryState.CLOSING && waiting.isEmpty())
+                state0 = state = QueryState.CLOSED;
+        }
+
+        if (state0 == QueryState.CLOSED) {
+            try {
+                IgniteException wrpEx = null;
+
+                for (UUID nodeId : remotes) {
+                    try {
+                        if (!nodeId.equals(root.context().localNodeId()))
+                            exch.closeQuery(nodeId, id());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (wrpEx == null)
+                            wrpEx = new IgniteException("Failed to send cancel message. [nodeId=" + nodeId + ']', e);
+                        else
+                            wrpEx.addSuppressed(e);
+                    }
+                }
+
+                if (wrpEx != null)
+                    throw wrpEx;
+            }
+            finally {
+                super.tryClose();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        cancel.cancel();
+
+        tryClose();
+    }
+
+    /** */
+    public PlanningContext planningContext() {
+        synchronized (mux) {
+            if (state == QueryState.CLOSED) {
+                throw new IgniteSQLException(
+                    "The query was cancelled while executing.",
+                    IgniteQueryErrorCode.QUERY_CANCELED
+                );
+            }
+
+            if (pctx == null) {
+                state = QueryState.PLANNING;

Review comment:
       I'm wondering, what happens if there will be transition CLOSING --> PLANNING




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735634262



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -271,9 +300,77 @@ public FailureProcessor failureProcessor() {
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
-        String qry, Object... params) throws IgniteSQLException {
+        String sql, Object... params) throws IgniteSQLException {
+        SchemaPlus schema = schemaHolder.schema(schemaName);
+
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(schema.getName(), sql));
+
+        if (plan != null) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sql,
+                schema,
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+
+            try {
+                return Collections.singletonList(executionSvc.executePlan(
+                    qry,
+                    plan
+                ));
+            }
+                catch (Exception e) {
+                qryReg.unregister(qry.id());
+
+                if (qry.isCancelled())
+                    throw new IgniteSQLException("The query was cancelled while planning", IgniteQueryErrorCode.QUERY_CANCELED, e);
+                else
+                    throw e;
+            }
+        }
 
-        return executionSvc.executeQuery(qryCtx, schemaName, qry, params);
+        SqlNodeList qryList = Commons.parse(sql, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
+
+        for (final SqlNode sqlNode: qryList) {
+            RootQuery<Object[]> qry = new RootQuery<>(
+                sqlNode.toString(),
+                schemaHolder.schema(schemaName), // Update schema for each query in multiple statements.
+                params,
+                qryCtx,
+                exchangeSvc,
+                (q) -> qryReg.unregister(q.id()),
+                log
+            );
+
+            qryReg.register(qry);
+            try {
+                if (qryList.size() == 1) {
+                    plan = queryPlanCache().queryPlan(
+                        new CacheKey(schemaName, qry.sql()),
+                        () -> prepareSvc.prepareSingle(sqlNode, qry.planningContext()));
+                }
+                else
+                    plan = prepareSvc.prepareSingle(sqlNode, qry.planningContext());
+
+                cursors.add(executionSvc.executePlan(qry, plan));
+            }
+            catch (Exception e) {
+                qryReg.unregister(qry.id());
+

Review comment:
       let's close all opened cursors




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r735620767



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/QueryRegistry.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.Collection;
+import java.util.UUID;
+
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ * Registry of the running queries.
+ */
+public interface QueryRegistry<RowT> extends Service {
+    /**
+     * Register the query or return the exists query with the same identifier.
+     *
+     * @param qry Query to register.
+     * @return registered query.

Review comment:
       ```suggestion
        * Register the query or return the existing one with the same identifier.
        *
        * @param qry Query to register.
        * @return Registered query.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733490716



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -291,4 +375,36 @@ private void onStop(Service... services) {
                 ((LifecycleAware) service).onStop();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public Query register(Query qry) {
+        Query old = runningQrys.putIfAbsent(qry.id(), qry);
+
+        return old != null ? old : qry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Query query(UUID id) {
+        return runningQrys.get(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(UUID id) {
+        runningQrys.remove(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Query> runningQueries() {
+        return runningQrys.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RunningQuery runningQuery(UUID id) {

Review comment:
       The CalciteQueryProcessor and QueryRegistry are separated with introduce `QueryRegistryImpl`. I guess it solves partially the ugly code at this place.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9476: IGNITE-12991 Calcite integration. Introduce running query registry & cancellation refactoring

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on a change in pull request #9476:
URL: https://github.com/apache/ignite/pull/9476#discussion_r733659837



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
##########
@@ -71,12 +84,13 @@
 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.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.processors.query.calcite.util.LifecycleAware;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
 /** */
-public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
+public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine, QueryRegistry {

Review comment:
       you rewrite this code already, so not actual.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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