You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "AMashenkov (via GitHub)" <gi...@apache.org> on 2023/06/12 13:37:34 UTC

[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2164: IGNITE-17765 Sql. Avoid parsing queries that already have cached plans.

AMashenkov commented on code in PR #2164:
URL: https://github.com/apache/ignite-3/pull/2164#discussion_r1226680971


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java:
##########
@@ -407,70 +423,83 @@ private CompletableFuture<AsyncSqlCursor<List<Object>>> querySingle0(
 
         CompletableFuture<Void> start = new CompletableFuture<>();
 
+        boolean implicitTxRequired = outerTx == null;
         AtomicReference<InternalTransaction> tx = new AtomicReference<>();
 
         CompletableFuture<AsyncSqlCursor<List<Object>>> stage = start
-                .thenApply(v -> {
-                    StatementParseResult parseResult = IgniteSqlParser.parse(sql, StatementParseResult.MODE);
-                    SqlNode sqlNode = parseResult.statement();
-
-                    validateParsedStatement(context, outerTx, parseResult, sqlNode, params);
-
-                    return sqlNode;
-                })
-                .thenCompose(sqlNode -> {
-                    boolean rwOp = dataModificationOp(sqlNode);
-
-                    boolean implicitTxRequired = outerTx == null;
-
-                    tx.set(implicitTxRequired ? txManager.begin(!rwOp) : outerTx);
-
-                    SchemaPlus schema = sqlSchemaManager.schema(schemaName);
-
-                    if (schema == null) {
-                        return CompletableFuture.failedFuture(new SchemaNotFoundException(schemaName));
-                    }
-
-                    BaseQueryContext ctx = BaseQueryContext.builder()
-                            .frameworkConfig(
-                                    Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
-                                            .defaultSchema(schema)
-                                            .build()
-                            )
+                .thenCompose(v -> {
+                    Builder contextBuilder = BaseQueryContext.builder()
                             .logger(LOG)
                             .cancel(queryCancel)
                             .parameters(params)
-                            .plannerTimeout(PLANNER_TIMEOUT)
-                            .build();
-
-                    return prepareSvc.prepareAsync(sqlNode, ctx)
-                            .thenApply(plan -> {
-                                var dataCursor = executionSrvc.executePlan(tx.get(), plan, ctx);
-
-                                SqlQueryType queryType = plan.type();
-                                assert queryType != null : "Expected a full plan but got a fragment: " + plan;
-
-                                return new AsyncSqlCursorImpl<>(
-                                        queryType,
-                                        plan.metadata(),
-                                        implicitTxRequired ? tx.get() : null,
-                                        new AsyncCursor<List<Object>>() {
-                                            @Override
-                                            public CompletableFuture<BatchedResult<List<Object>>> requestNextAsync(int rows) {
-                                                session.touch();
-
-                                                return dataCursor.requestNextAsync(rows);
-                                            }
-
-                                            @Override
-                                            public CompletableFuture<Void> closeAsync() {
-                                                session.touch();
-
-                                                return dataCursor.closeAsync();
-                                            }
-                                        }
-                                );
-                            });
+                            .plannerTimeout(PLANNER_TIMEOUT);
+
+                    CompletableFuture<PlanWithContext>[] newPlanHolder = new CompletableFuture[1];
+
+                    CompletableFuture<QueryPlan> cachedPlan = queryCache.computeIfAbsent(new CacheKey(schemaName, sql, params), (k) -> {

Review Comment:
   The lamda that is passed to `queryCache.computeIfAbsent` is run inside the synchronized block. For queries, which shouldn't be cached (like DDL), this maybe overkill.
   The lamda that is passed to `queryCache.computeIfAbsent` is run inside the synchronized block. For queries, which shouldn't be cached (like DDL), this maybe overkill.
   
   `queryCache.computeIfAbsent` already returns the future. Thus, there is no need to do any long operation inside the synchronized block.
   
   Instead, you can always return a future from lamda for QUERY and DML and put computation to another thread. For non-QUERY or non-DML just return null and do computation in the current thread. Or maybe avoid using compute 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