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

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

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


##########
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:
   > Instead, you can always return a future from lamda for QUERY and DML with putting computation to another thread.
   
   This is the main problem - I can't do this, because we need to parse the query to understand the type of 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