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


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java:
##########
@@ -459,63 +532,95 @@ private CompletableFuture<QueryPlan> prepareQuery(
                 // Try to produce a fast plan, if successful, then return that 
plan w/o caching it.
                 QueryPlan fastPlan = tryOptimizeFast(stmt, ctx);
                 if (fastPlan != null) {
-                    return CompletableFuture.completedFuture(fastPlan);
+                    return CompletableFuture.completedFuture(new 
PlanInfo(fastPlan));
                 }
             }
 
-            // Use parameter metadata to compute a cache key.
-            CacheKey key = 
createCacheKeyFromParameterMetadata(stmt.parsedResult, ctx, 
stmt.parameterMetadata);
+            if (prepareWithCache) {
+                // Use parameter metadata to compute a cache key.
+                CacheKey cacheKey = 
createCacheKeyFromParameterMetadata(stmt.parsedResult, ctx, 
stmt.parameterMetadata);
 
-            CompletableFuture<QueryPlan> planFut = cache.get(key, k -> 
CompletableFuture.supplyAsync(() -> {
-                IgnitePlanner planner = ctx.planner();
+                return cache.get(cacheKey, k -> 
CompletableFuture.supplyAsync(() -> buildQueryPlan(stmt, ctx,
+                        () -> cache.invalidate(cacheKey)), planningPool));
+            } else {
+                return CompletableFuture.supplyAsync(() -> 
buildQueryPlan(stmt, ctx, () -> {}), planningPool);
+            }
+        });
+    }
 
-                ValidationResult validated = stmt.value;
-                ParameterMetadata parameterMetadata = stmt.parameterMetadata;
+    private PlanInfo buildQueryPlan(ValidStatement<ValidationResult> stmt, 
PlanningContext ctx, Runnable onTimeoutAction) {
+        IgnitePlanner planner = ctx.planner();
 
-                SqlNode validatedNode = validated.sqlNode();
+        ValidationResult validated = stmt.value;
+        ParameterMetadata parameterMetadata = stmt.parameterMetadata;
 
-                RelWithMetadata relWithMetadata = doOptimize(ctx, 
validatedNode, planner, () -> cache.invalidate(key));
-                IgniteRel optimizedRel = relWithMetadata.rel;
-                QueryPlan fastPlan = tryOptimizeFast(stmt, ctx);
+        SqlNode validatedNode = validated.sqlNode();
 
-                ResultSetMetadata resultSetMetadata = 
resultSetMetadata(validated.dataType(), validated.origins(), 
validated.aliases());
+        RelWithMetadata relWithMetadata = doOptimize(ctx, validatedNode, 
planner, onTimeoutAction);
+        IgniteRel optimizedRel = relWithMetadata.rel;
+        QueryPlan fastPlan = tryOptimizeFast(stmt, ctx);
 
-                int catalogVersion = ctx.catalogVersion();
+        ResultSetMetadata resultSetMetadata = 
resultSetMetadata(validated.dataType(), validated.origins(), 
validated.aliases());
 
-                if (optimizedRel instanceof IgniteKeyValueGet) {
-                    IgniteKeyValueGet kvGet = (IgniteKeyValueGet) optimizedRel;
+        int catalogVersion = ctx.catalogVersion();
 
-                    return new KeyValueGetPlan(
-                            nextPlanId(),
-                            catalogVersion,
-                            kvGet,
-                            resultSetMetadata,
-                            parameterMetadata,
-                            relWithMetadata.paMetadata,
-                            relWithMetadata.ppMetadata
-                    );
-                }
+        if (optimizedRel instanceof IgniteKeyValueGet) {
+            IgniteKeyValueGet kvGet = (IgniteKeyValueGet) optimizedRel;
 
-                var plan = new MultiStepPlan(
-                        nextPlanId(),
-                        SqlQueryType.QUERY,
-                        optimizedRel,
-                        resultSetMetadata,
-                        parameterMetadata,
-                        catalogVersion,
-                        relWithMetadata.numSources,
-                        fastPlan,
-                        relWithMetadata.paMetadata,
-                        relWithMetadata.ppMetadata
-                );
-
-                logPlan(parsedResult.originalQuery(), plan);
-
-                return plan;
-            }, planningPool));
-
-            return planFut;
-        });
+            var plan = new KeyValueGetPlan(
+                    nextPlanId(),
+                    catalogVersion,
+                    kvGet,
+                    resultSetMetadata,
+                    parameterMetadata,
+                    relWithMetadata.paMetadata,
+                    relWithMetadata.ppMetadata
+            );
+
+            return new PlanInfo(plan);
+        }
+
+        var plan = new MultiStepPlan(
+                nextPlanId(),
+                SqlQueryType.QUERY,
+                optimizedRel,
+                resultSetMetadata,
+                parameterMetadata,
+                catalogVersion,
+                relWithMetadata.numSources,
+                fastPlan,
+                relWithMetadata.paMetadata,
+                relWithMetadata.ppMetadata
+        );
+
+        logPlan(stmt.parsedResult().originalQuery(), plan);
+
+        int currentCatalogVersion = directCatalogVersion();
+
+        if (currentCatalogVersion == catalogVersion) {
+            Set<Integer> sources = resolveSources(plan.getRel());
+
+            return new PlanInfo(plan, stmt, ctx, sources);
+        }
+
+        return new PlanInfo(plan);
+    }
+
+    private CompletableFuture<PlanInfo> preparePlan(SqlQueryType queryType, 
ParsedResult parsedRes, PlanningContext ctx, CacheKey key) {

Review Comment:
   Calls inside the method can be also renamed if it will better reflect their 
purposes.



-- 
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