geruh commented on code in PR #13400: URL: https://github.com/apache/iceberg/pull/13400#discussion_r2597603088
########## core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java: ########## @@ -0,0 +1,260 @@ +/* + * 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.rest; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +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 org.apache.iceberg.FileScanTask; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.rest.requests.FetchScanTasksRequest; +import org.apache.iceberg.rest.responses.FetchScanTasksResponse; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class ScanTaskIterable implements CloseableIterable<FileScanTask> { + + private static final Logger LOG = LoggerFactory.getLogger(ScanTaskIterable.class); + 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<FileScanTask> initialFileScanTasks; + 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 Map<String, String> headers; + private final ParserContext parserContext; + + ScanTaskIterable( + List<String> initialPlanTasks, + List<FileScanTask> initialFileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Map<String, String> headers, + ExecutorService executorService, + ParserContext parserContext) { + + this.taskQueue = new LinkedBlockingQueue<>(DEFAULT_TASK_QUEUE_CAPACITY); + this.planTasks = new ConcurrentLinkedQueue<>(); + // Initialize initial file scan tasks queue so that multiple workers can poll produce from it. + this.initialFileScanTasks = new ConcurrentLinkedQueue<>(initialFileScanTasks); Review Comment: Will this npe if initialFileScanTasks is null. ########## core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java: ########## @@ -0,0 +1,282 @@ +/* + * 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.rest; + +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DataTableScan; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.TableScanContext; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.requests.PlanTableScanRequest; +import org.apache.iceberg.rest.responses.FetchPlanningResultResponse; +import org.apache.iceberg.rest.responses.PlanTableScanResponse; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class RESTTableScan extends DataTableScan { + private static final Logger LOG = LoggerFactory.getLogger(RESTTableScan.class); + private static final long MIN_SLEEP_MS = 1000; // Initial delay + private static final long MAX_SLEEP_MS = 60 * 1000; // Max backoff delay (1 minute) + private static final int MAX_ATTEMPTS = 10; // Max number of poll checks + private static final long MAX_WAIT_TIME_MS = 5 * 60 * 1000; // Total maximum duration (5 minutes) + private static final double SCALE_FACTOR = 2.0; // Exponential scale factor + + private final RESTClient client; + private final Map<String, String> headers; + private final TableOperations operations; + private final Table table; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Set<Endpoint> supportedEndpoints; + private final ParserContext parserContext; + private String currentPlanId = null; + + RESTTableScan( + Table table, + Schema schema, + TableScanContext context, + RESTClient client, + Map<String, String> headers, + TableOperations operations, + TableIdentifier tableIdentifier, + ResourcePaths resourcePaths, + Set<Endpoint> supportedEndpoints) { + super(table, schema, context); + this.table = table; + this.client = client; + this.headers = headers; + this.operations = operations; + this.tableIdentifier = tableIdentifier; + this.resourcePaths = resourcePaths; + this.supportedEndpoints = supportedEndpoints; + this.parserContext = + ParserContext.builder() + .add("specsById", table.specs()) + .add("caseSensitive", context().caseSensitive()) + .build(); + } + + @Override + protected TableScan newRefinedScan( + Table refinedTable, Schema refinedSchema, TableScanContext refinedContext) { + return new RESTTableScan( + refinedTable, + refinedSchema, + refinedContext, + client, + headers, + operations, + tableIdentifier, + resourcePaths, + supportedEndpoints); + } + + @Override + public CloseableIterable<FileScanTask> planFiles() { + Long startSnapshotId = context().fromSnapshotId(); Review Comment: In the case of multiple `planFile` calls what do you think cancelling the previous plan? ``` TableScan scan = table.newScan(); scan.planFiles(); // Plan created scan.planFiles(); // Plan created ``` ########## core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java: ########## @@ -133,4 +133,38 @@ public String view(TableIdentifier ident) { public String renameView() { return SLASH.join("v1", prefix, "views", "rename"); } + + public String planTableScan(TableIdentifier ident) { + return SLASH.join( + "v1", + prefix, + "namespaces", + RESTUtil.encodeNamespace(ident.namespace()), + "tables", + RESTUtil.encodeString(ident.name()), + "plan"); + } + + public String plan(TableIdentifier ident, String planId) { + return SLASH.join( + "v1", + prefix, + "namespaces", + RESTUtil.encodeNamespace(ident.namespace()), + "tables", + RESTUtil.encodeString(ident.name()), + "plan", + planId); Review Comment: Should we encode the planId? ########## core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java: ########## @@ -0,0 +1,282 @@ +/* + * 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.rest; + +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DataTableScan; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.TableScanContext; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.requests.PlanTableScanRequest; +import org.apache.iceberg.rest.responses.FetchPlanningResultResponse; +import org.apache.iceberg.rest.responses.PlanTableScanResponse; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class RESTTableScan extends DataTableScan { + private static final Logger LOG = LoggerFactory.getLogger(RESTTableScan.class); + private static final long MIN_SLEEP_MS = 1000; // Initial delay + private static final long MAX_SLEEP_MS = 60 * 1000; // Max backoff delay (1 minute) + private static final int MAX_ATTEMPTS = 10; // Max number of poll checks + private static final long MAX_WAIT_TIME_MS = 5 * 60 * 1000; // Total maximum duration (5 minutes) + private static final double SCALE_FACTOR = 2.0; // Exponential scale factor + + private final RESTClient client; + private final Map<String, String> headers; + private final TableOperations operations; + private final Table table; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Set<Endpoint> supportedEndpoints; + private final ParserContext parserContext; + private String currentPlanId = null; + + RESTTableScan( + Table table, + Schema schema, + TableScanContext context, + RESTClient client, + Map<String, String> headers, + TableOperations operations, + TableIdentifier tableIdentifier, + ResourcePaths resourcePaths, + Set<Endpoint> supportedEndpoints) { + super(table, schema, context); + this.table = table; + this.client = client; + this.headers = headers; + this.operations = operations; + this.tableIdentifier = tableIdentifier; + this.resourcePaths = resourcePaths; + this.supportedEndpoints = supportedEndpoints; + this.parserContext = + ParserContext.builder() + .add("specsById", table.specs()) + .add("caseSensitive", context().caseSensitive()) + .build(); + } + + @Override + protected TableScan newRefinedScan( + Table refinedTable, Schema refinedSchema, TableScanContext refinedContext) { + return new RESTTableScan( + refinedTable, + refinedSchema, + refinedContext, + client, + headers, + operations, + tableIdentifier, + resourcePaths, + supportedEndpoints); + } + + @Override + public CloseableIterable<FileScanTask> planFiles() { + Long startSnapshotId = context().fromSnapshotId(); + Long endSnapshotId = context().toSnapshotId(); + Long snapshotId = snapshotId(); + List<String> selectedColumns = + schema().columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); + + List<String> statsFields = null; + if (columnsToKeepStats() != null) { + statsFields = + columnsToKeepStats().stream() + .map(columnId -> schema().findColumnName(columnId)) + .collect(Collectors.toList()); + } + + PlanTableScanRequest.Builder builder = + PlanTableScanRequest.builder() + .withSelect(selectedColumns) + .withFilter(filter()) + .withCaseSensitive(isCaseSensitive()) + .withStatsFields(statsFields); + + if (startSnapshotId != null && endSnapshotId != null) { + builder + .withStartSnapshotId(startSnapshotId) + .withEndSnapshotId(endSnapshotId) + .withUseSnapshotSchema(true); + } else if (snapshotId != null) { + boolean useSnapShotSchema = snapshotId != table.currentSnapshot().snapshotId(); + builder.withSnapshotId(snapshotId).withUseSnapshotSchema(useSnapShotSchema); + } + + return planTableScan(builder.build()); + } + + private CloseableIterable<FileScanTask> planTableScan(PlanTableScanRequest planTableScanRequest) { + PlanTableScanResponse response = + client.post( + resourcePaths.planTableScan(tableIdentifier), + planTableScanRequest, + PlanTableScanResponse.class, + headers, + ErrorHandlers.defaultErrorHandler(), + stringStringMap -> {}, + parserContext); + + PlanStatus planStatus = response.planStatus(); + switch (planStatus) { + case COMPLETED: + currentPlanId = response.planId(); + return scanTasksIterable(response.planTasks(), response.fileScanTasks()); + case SUBMITTED: + Endpoint.check(supportedEndpoints, Endpoint.V1_FETCH_TABLE_SCAN_PLAN); + return fetchPlanningResult(response.planId()); + case FAILED: + throw new IllegalStateException( + String.format( + "Received status: %s for planId: %s", PlanStatus.FAILED, response.planId())); + case CANCELLED: + throw new IllegalStateException( + String.format( + "Received status: %s for planId: %s", PlanStatus.CANCELLED, response.planId())); + default: + throw new IllegalStateException( + String.format("Invalid planStatus: %s for planId: %s", planStatus, response.planId())); + } + } + + private CloseableIterable<FileScanTask> fetchPlanningResult(String planId) { + currentPlanId = planId; + + RetryPolicy<FetchPlanningResultResponse> retryPolicy = + RetryPolicy.<FetchPlanningResultResponse>builder() + .handleResultIf(response -> response.planStatus() == PlanStatus.SUBMITTED) + .withBackoff( + Duration.ofMillis(MIN_SLEEP_MS), Duration.ofMillis(MAX_SLEEP_MS), SCALE_FACTOR) + .withJitter(0.1) // Add jitter up to 10% of the calculated delay + .withMaxAttempts(MAX_ATTEMPTS) + .withMaxDuration(Duration.ofMillis(MAX_WAIT_TIME_MS)) + .onFailedAttempt( + e -> { + // Log when a retry occurs + LOG.debug( + "Plan {} still SUBMITTED (Attempt {}/{}). Previous attempt took {} ms.", + planId, + e.getAttemptCount(), + MAX_ATTEMPTS, + e.getElapsedAttemptTime().toMillis()); + }) + .onFailure( + e -> { + LOG.warn( + "Polling for plan {} failed due to: {}", + planId, + e.getException().getMessage()); + cancelPlan(); + }) + .build(); + + try { + FetchPlanningResultResponse response = + Failsafe.with(retryPolicy) + .get( + () -> + client.get( + resourcePaths.plan(tableIdentifier, planId), + headers, + FetchPlanningResultResponse.class, + headers, + ErrorHandlers.defaultErrorHandler(), Review Comment: This could return a NoSuchPlanTaskException. Maybe we can make an error handler for all these calls ########## core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java: ########## @@ -0,0 +1,282 @@ +/* + * 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.rest; + +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DataTableScan; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.TableScanContext; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.requests.PlanTableScanRequest; +import org.apache.iceberg.rest.responses.FetchPlanningResultResponse; +import org.apache.iceberg.rest.responses.PlanTableScanResponse; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class RESTTableScan extends DataTableScan { + private static final Logger LOG = LoggerFactory.getLogger(RESTTableScan.class); + private static final long MIN_SLEEP_MS = 1000; // Initial delay + private static final long MAX_SLEEP_MS = 60 * 1000; // Max backoff delay (1 minute) + private static final int MAX_ATTEMPTS = 10; // Max number of poll checks + private static final long MAX_WAIT_TIME_MS = 5 * 60 * 1000; // Total maximum duration (5 minutes) + private static final double SCALE_FACTOR = 2.0; // Exponential scale factor + + private final RESTClient client; + private final Map<String, String> headers; + private final TableOperations operations; + private final Table table; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Set<Endpoint> supportedEndpoints; + private final ParserContext parserContext; + private String currentPlanId = null; + + RESTTableScan( + Table table, + Schema schema, + TableScanContext context, + RESTClient client, + Map<String, String> headers, + TableOperations operations, + TableIdentifier tableIdentifier, + ResourcePaths resourcePaths, + Set<Endpoint> supportedEndpoints) { + super(table, schema, context); + this.table = table; + this.client = client; + this.headers = headers; + this.operations = operations; + this.tableIdentifier = tableIdentifier; + this.resourcePaths = resourcePaths; + this.supportedEndpoints = supportedEndpoints; + this.parserContext = + ParserContext.builder() + .add("specsById", table.specs()) + .add("caseSensitive", context().caseSensitive()) + .build(); + } + + @Override + protected TableScan newRefinedScan( + Table refinedTable, Schema refinedSchema, TableScanContext refinedContext) { + return new RESTTableScan( + refinedTable, + refinedSchema, + refinedContext, + client, + headers, + operations, + tableIdentifier, + resourcePaths, + supportedEndpoints); + } + + @Override + public CloseableIterable<FileScanTask> planFiles() { + Long startSnapshotId = context().fromSnapshotId(); + Long endSnapshotId = context().toSnapshotId(); + Long snapshotId = snapshotId(); + List<String> selectedColumns = + schema().columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); + + List<String> statsFields = null; + if (columnsToKeepStats() != null) { + statsFields = + columnsToKeepStats().stream() + .map(columnId -> schema().findColumnName(columnId)) + .collect(Collectors.toList()); + } + + PlanTableScanRequest.Builder builder = + PlanTableScanRequest.builder() + .withSelect(selectedColumns) + .withFilter(filter()) + .withCaseSensitive(isCaseSensitive()) + .withStatsFields(statsFields); + + if (startSnapshotId != null && endSnapshotId != null) { + builder + .withStartSnapshotId(startSnapshotId) + .withEndSnapshotId(endSnapshotId) + .withUseSnapshotSchema(true); + } else if (snapshotId != null) { + boolean useSnapShotSchema = snapshotId != table.currentSnapshot().snapshotId(); + builder.withSnapshotId(snapshotId).withUseSnapshotSchema(useSnapShotSchema); + } + + return planTableScan(builder.build()); + } + + private CloseableIterable<FileScanTask> planTableScan(PlanTableScanRequest planTableScanRequest) { + PlanTableScanResponse response = + client.post( + resourcePaths.planTableScan(tableIdentifier), + planTableScanRequest, + PlanTableScanResponse.class, + headers, + ErrorHandlers.defaultErrorHandler(), Review Comment: This error handler would be closer to the `ErrorHandlers.tableErrorHandler()` -- 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]
