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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to