github-actions[bot] commented on code in PR #61825:
URL: https://github.com/apache/doris/pull/61825#discussion_r3018645786


##########
fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/LocalParallelPlanningSplitProducer.java:
##########
@@ -0,0 +1,150 @@
+// 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.doris.datasource.iceberg.source;
+
+import org.apache.doris.common.UserException;
+import org.apache.doris.common.security.authentication.ExecutionAuthenticator;
+import org.apache.doris.nereids.exceptions.NotSupportedException;
+import org.apache.doris.spi.Split;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Local FE async planning producer that reuses current Iceberg planning flow.
+ */
+public class LocalParallelPlanningSplitProducer implements 
PlanningSplitProducer {
+    private static final Logger LOG = 
LogManager.getLogger(LocalParallelPlanningSplitProducer.class);
+
+    /**
+     * Runtime context required by local split planning.
+     */
+    public interface PlanningContext {
+        boolean isBatchMode();
+
+        int numApproximateSplits();
+
+        TableScan createTableScan() throws UserException;
+
+        CloseableIterable<FileScanTask> planFileScanTask(TableScan scan);
+
+        Split createSplit(FileScanTask task);
+
+        void recordManifestCacheProfile();
+
+        Optional<NotSupportedException> checkNotSupportedException(Exception 
e);
+
+        ExecutionAuthenticator getExecutionAuthenticator();
+
+        Executor getScheduleExecutor();
+    }
+
+    private final PlanningContext context;
+    private final AtomicBoolean stopped = new AtomicBoolean(false);
+    private final AtomicReference<CompletableFuture<Void>> runningTask = new 
AtomicReference<>();
+
+    public LocalParallelPlanningSplitProducer(PlanningContext context) {
+        this.context = Preconditions.checkNotNull(context, "planningContext is 
null");
+    }
+
+    @Override
+    public boolean isBatchMode() {
+        return context.isBatchMode();
+    }
+
+    @Override
+    public int numApproximateSplits() {
+        return context.numApproximateSplits();
+    }
+
+    @Override
+    public void start(int numBackends, SplitSink sink) throws UserException {
+        stopped.set(false);
+        final TableScan scan;
+        try {
+            scan = context.getExecutionAuthenticator().execute(() -> 
context.createTableScan());
+        } catch (Exception e) {
+            throw new UserException(e.getMessage(), e);
+        }
+        CompletableFuture<Void> future = CompletableFuture.runAsync(() -> {

Review Comment:
   `CompletableFuture.runAsync(..., context.getScheduleExecutor())` can throw 
synchronously when the scheduling pool rejects the task. Before this refactor, 
that path was caught by `IcebergScanNode.startSplit()` and converted to 
`UserException`; now it escapes as an unchecked exception before 
`sink.fail(...)` or `runningTask.set(...)` happens. That changes the contract 
seen by `SplitAssignment.init()` and makes Iceberg inconsistent with the other 
batch producers (`HiveScanNode`, `HudiScanNode`, `MaxComputeScanNode`), which 
explicitly convert submission failures into normal split-assignment errors. 
Please wrap the async submission itself and translate rejection into the same 
failure path.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to