korlov42 commented on code in PR #5143:
URL: https://github.com/apache/ignite-3/pull/5143#discussion_r1937318978


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutor.java:
##########
@@ -241,6 +254,229 @@ CompletableFuture<AsyncSqlCursor<InternalSqlRow>> 
executeChildQuery(
                 });
     }
 
+    CompletableFuture<AsyncSqlCursor<InternalSqlRow>> executeChildBatch(
+            Query parent,
+            QueryTransactionContext scriptTxContext,
+            int batchOffset,
+            List<ParsedResultWithNextCursorFuture> batch
+    ) {
+        if (IgniteUtils.assertionsEnabled()) {
+            int offsetInBatch = 0;
+            for (ParsedResultWithNextCursorFuture item : batch) {
+                assert item.parsedQuery.queryType() == SqlQueryType.DDL 
+                        : item.parsedQuery.queryType() + " at statement #" + 
(batchOffset + offsetInBatch);
+
+                offsetInBatch++;
+            }
+        }
+
+        if (!busyLock.enterBusy()) {
+            return failedFuture(new NodeStoppingException());
+        }
+
+        List<Query> queries = new ArrayList<>(batch.size());
+
+        try {
+            int offsetInBatch = 0;
+            for (ParsedResultWithNextCursorFuture item : batch) {
+                Query query = new Query(
+                        Instant.ofEpochMilli(clockService.now().getPhysical()),
+                        parent,
+                        item.parsedQuery,
+                        batchOffset + offsetInBatch,
+                        idGenerator.next(),
+                        scriptTxContext,
+                        ArrayUtils.OBJECT_EMPTY_ARRAY,
+                        item.nextCursorFuture
+                );
+
+                offsetInBatch++;
+
+                trackQuery(query, null);
+
+                queries.add(query);
+
+                parent.cancel.attach(query.cancel);
+            }
+        } catch (QueryCancelledException ex) {
+            queries.forEach(query -> query.onError(ex));
+
+            return failedFuture(ex);
+        } finally {
+            busyLock.leaveBusy();
+        }
+
+        List<CompletableFuture<?>> preparedQueryFutures = new 
ArrayList<>(batch.size());
+        for (Query query : queries) {
+            
preparedQueryFutures.add(Programs.SCRIPT_ITEM_PREPARATION.run(query));
+        }
+
+        return CompletableFutures.allOf(preparedQueryFutures)
+                .handle((none, ignored) -> {
+                    List<DdlPlan> dllPlans = new ArrayList<>();
+                    for (Query query : queries) {
+                        QueryPlan plan = query.plan;
+                        if (plan == null) {
+                            assert query.error.get() != null;
+
+                            break;
+                        }
+
+                        dllPlans.add((DdlPlan) plan);
+                    }
+
+                    return dllPlans;
+                })
+                .thenCompose(dllPlans -> {
+                    parent.cancel.throwIfCancelled();
+
+                    // First-statement-error case.
+                    if (dllPlans.isEmpty()) {
+                        Iterator<Query> it = queries.iterator();
+
+                        Throwable th = it.next().error.get();
+
+                        assert th != null;
+
+                        while (it.hasNext()) {
+                            it.next().onError(th);
+                        }
+
+                        return failedFuture(th);
+                    }
+
+                    return executionService.executeDdlBatch(dllPlans, 
scriptTxContext::updateObservableTime)
+                            .handle((dataCursors, error) -> {
+                                if (error != null) {
+                                    // Fallback to statement-by-statement 
execution
+                                    return executeSequentially(queries);
+                                }
+
+                                AsyncSqlCursor<InternalSqlRow> firstCursor = 
null;
+                                
CompletableFuture<AsyncSqlCursor<InternalSqlRow>> cursorRef = null;
+                                Iterator<AsyncDataCursor<InternalSqlRow>> 
dataCursorIterator = dataCursors.iterator();
+                                Throwable th = null;
+                                for (Query query : queries) {
+                                    if (th != null) {
+                                        query.onError(th);
+
+                                        continue;
+                                    }
+
+                                    QueryPlan plan = query.plan;
+
+                                    if (plan == null) {
+                                        th = query.error.get();
+
+                                        assert th != null;
+
+                                        // First-statement-error case is 
covered before we call executionService,
+                                        // and after first iteration cursorRef 
must not be null.
+                                        assert cursorRef != null;
+
+                                        cursorRef.completeExceptionally(th);
+
+                                        continue;
+                                    }
+
+                                    query.moveTo(ExecutionPhase.EXECUTING);
+
+                                    AsyncDataCursor<InternalSqlRow> dataCursor 
= dataCursorIterator.next();
+                                    AsyncSqlCursor<InternalSqlRow> 
currentCursor = createAndSaveSqlCursor(query, dataCursor); 
+
+                                    if (cursorRef != null) {
+                                        cursorRef.complete(currentCursor);
+                                    }
+
+                                    cursorRef = query.nextCursorFuture;
+
+                                    if (firstCursor == null) {
+                                        firstCursor = currentCursor;
+                                    }
+
+                                    currentCursor.onClose().thenRun(() -> 
query.moveTo(ExecutionPhase.TERMINATED));
+                                }
+
+                                return completedFuture(firstCursor);
+                            })
+                            .thenCompose(Function.identity());
+                });
+    }
+
+    private CompletableFuture<AsyncSqlCursor<InternalSqlRow>> 
executeSequentially(List<Query> queries) {
+        CompletableFuture<AsyncSqlCursor<InternalSqlRow>> firstCursor = null;
+        CompletableFuture<AsyncSqlCursor<InternalSqlRow>> cursorRef = null;
+        CompletableFuture<Void> lastStep = nullCompletedFuture();
+        for (Query query : queries) {
+            query.reset();
+
+            CompletableFuture<AsyncSqlCursor<InternalSqlRow>> cursorRef0 = 
cursorRef;
+            CompletableFuture<AsyncSqlCursor<InternalSqlRow>> cursorFuture = 
lastStep
+                    .whenComplete((none, ex) -> {
+                        if (ex != null) {
+                            query.onError(ex);
+                        }
+                    })
+                    .thenCompose(none -> 
Programs.SCRIPT_ITEM_EXECUTION.run(query)
+                            .whenComplete((cursor, ex) -> {
+                                if (cursorRef0 != null) {
+                                    if (cursor != null) {
+                                        cursorRef0.complete(cursor);
+                                    } else {
+                                        cursorRef0.completeExceptionally(ex);
+                                    }
+                                }
+
+                                if (cursor != null) {
+                                    cursor.onClose().thenRun(() -> 
query.moveTo(ExecutionPhase.TERMINATED));
+                                }
+                            }));
+
+            lastStep = 
cursorFuture.thenCompose(AsyncDataCursor::onFirstPageReady);
+            cursorRef = query.nextCursorFuture;
+
+            if (firstCursor == null) {
+                firstCursor = cursorFuture;
+            }
+        }
+
+        return firstCursor;
+    }
+
+    AsyncSqlCursor<InternalSqlRow> createAndSaveSqlCursor(Query query, 
AsyncDataCursor<InternalSqlRow> dataCursor) {

Review Comment:
   I would not do this. Query is supposed to be a single container for storing 
results of execution of particular phase + a few handy methods to cover 
state-machine-related part. And this is used to be part of phase handler. which 
were moved here because now part of the phase handler is implemented in 
QueryExecutor



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