CalvinKirs commented on code in PR #46398:
URL: https://github.com/apache/doris/pull/46398#discussion_r1917811649
##########
fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java:
##########
@@ -184,14 +194,45 @@ private void setIcebergParams(TFileRangeDesc rangeDesc,
IcebergSplit icebergSpli
@Override
public List<Split> getSplits(int numBackends) throws UserException {
try {
- return
source.getCatalog().getPreExecutionAuthenticator().execute(() ->
doGetSplits(numBackends));
+ return preExecutionAuthenticator.execute(() ->
doGetSplits(numBackends));
} catch (Exception e) {
throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e),
e);
}
+ }
+ @Override
+ public void startSplit(int numBackends) throws UserException {
+ try {
+ preExecutionAuthenticator.execute(() -> {
+ doStartSplit();
+ return null;
+ });
+ } catch (Exception e) {
+ throw new UserException(e.getMessage(), e);
+ }
}
- private List<Split> doGetSplits(int numBackends) throws UserException {
+ public void doStartSplit() throws UserException {
+ TableScan scan = createTableScan();
+ CompletableFuture.runAsync(() -> {
+ try {
+ CloseableIterable<FileScanTask> fileScanTasks =
planFileScanTask(scan);
+ // 1. this task should stop when all splits are assigned
+ // 2. if we want to stop this plan, we can close the
fileScanTasks to stop
+ splitAssignment.addCloseable(fileScanTasks);
+
+ fileScanTasks.forEach(fileScanTask -> {
+
splitAssignment.addToQueue(Lists.newArrayList(createIcebergSplit(fileScanTask)));
+ });
+
+ splitAssignment.finishSchedule();
+ } catch (Exception e) {
+ splitAssignment.setException(new UserException(e.getMessage(),
e));
+ }
+ });
+ }
+
Review Comment:
This implementation starts a new thread, but since Subject cannot be
accessed across threads, Kerberos authentication will fail to function
correctly.
--
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]