singhpk234 commented on code in PR #13400: URL: https://github.com/apache/iceberg/pull/13400#discussion_r2561371401
########## core/src/main/java/org/apache/iceberg/ScanTasksIterable.java: ########## @@ -0,0 +1,270 @@ +/* + * 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.iceberg; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.Endpoint; +import org.apache.iceberg.rest.ErrorHandlers; +import org.apache.iceberg.rest.ParserContext; +import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.ResourcePaths; +import org.apache.iceberg.rest.requests.FetchScanTasksRequest; +import org.apache.iceberg.rest.responses.FetchScanTasksResponse; +import org.apache.iceberg.util.ThreadPools; + +class ScanTasksIterable implements CloseableIterable<FileScanTask> { + + private static final int DEFAULT_TASK_QUEUE_CAPACITY = 1000; + + private static final long QUEUE_POLL_TIMEOUT_MS = 100; + + private static final int WORKER_POOL_SIZE = Math.max(1, ThreadPools.WORKER_THREAD_POOL_SIZE / 4); + + private final BlockingQueue<FileScanTask> taskQueue; + + private final ConcurrentLinkedQueue<String> planTasks; + + private final AtomicInteger activeWorkers = new AtomicInteger(0); + + private final AtomicBoolean shutdown = new AtomicBoolean(false); + + private final ExecutorService executorService; + + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private final Map<Integer, PartitionSpec> specsById; + private final boolean caseSensitive; + private final Supplier<Boolean> cancellationCallback; + private final Set<Endpoint> supportedEndpoints; + + ScanTasksIterable( + List<String> initialPlanTasks, + List<FileScanTask> initialFileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers, + ExecutorService executorService, + Map<Integer, PartitionSpec> specsById, + boolean caseSensitive, + Supplier<Boolean> cancellationCallback, + Set<Endpoint> supportedEndpoints) { + + this.taskQueue = new LinkedBlockingQueue<>(DEFAULT_TASK_QUEUE_CAPACITY); + this.planTasks = new ConcurrentLinkedQueue<>(); + + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + this.executorService = executorService; + this.specsById = specsById; + this.caseSensitive = caseSensitive; + this.cancellationCallback = cancellationCallback; + this.supportedEndpoints = supportedEndpoints; + + if (initialFileScanTasks != null && !initialFileScanTasks.isEmpty()) { + for (FileScanTask task : initialFileScanTasks) { + try { + taskQueue.put(task); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while adding initial tasks", e); + } + } + } + + if (initialPlanTasks != null && !initialPlanTasks.isEmpty()) { + planTasks.addAll(initialPlanTasks); + } + + submitFixedWorkers(); + } + + private void submitFixedWorkers() { + if (planTasks.isEmpty()) { + return; + } + + int numWorkers = Math.min(WORKER_POOL_SIZE, planTasks.size()); + + for (int i = 0; i < numWorkers; i++) { + executorService.execute(new PlanTaskWorker()); + } + } + + @Override + public CloseableIterator<FileScanTask> iterator() { + return new ScanTasksIterator(); + } + + @Override + public void close() throws IOException {} + + private class PlanTaskWorker implements Runnable { + + @Override + public void run() { + activeWorkers.incrementAndGet(); + + try { + while (true) { + if (shutdown.get()) { + return; + } + + String planTask = planTasks.poll(); + if (planTask == null) { + return; + } + + Preconditions.checkState( Review Comment: I missed completely the `Endpoint.check` utility :) ! > if it would make sense to add this check much earlier (in RESTTableScan) The issue is it may happen that server might not actually require to call this endpoint because it doesn't return plan-tasks at all, if we are ok with it happy to put this check my initial POV was https://github.com/apache/iceberg/pull/13400#discussion_r2540228084 on this -- 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]
