amogh-jahagirdar commented on code in PR #13400: URL: https://github.com/apache/iceberg/pull/13400#discussion_r2604136112
########## 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: I don't think this is really something we need to worry about. If we put aside remote scan planning, and just take the client side planning case, these are 2 fundamentally different issuances of scanning work (e.g. in the client side case, we'd read manifests, sure on the second time maybe manifests are cached so we avoid the I/O but it's still doing the work and do the work of figuring out the tasks.). I think we just work from that semantic of the API. -- 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]
