amogh-jahagirdar commented on code in PR #13400:
URL: https://github.com/apache/iceberg/pull/13400#discussion_r2356878678
##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -465,6 +479,26 @@ public Table loadTable(SessionContext context,
TableIdentifier identifier) {
return table;
}
+ private RESTTable tableSupportsRemoteScanPlanning(
+ TableOperations ops, TableIdentifier finalIdentifier, RESTClient
restClient) {
+ if
(ops.current().properties().containsKey(REST_TABLE_SCAN_PLANNING_PROPERTY)) {
+ boolean tableSupportsRemotePlanning =
+ ops.current().propertyAsBoolean(REST_TABLE_SCAN_PLANNING_PROPERTY,
false);
+ if (tableSupportsRemotePlanning && restServerPlanningEnabled) {
Review Comment:
Whether or not server side planning is supported is a matter of checking the
config response no? I don't think it's something that would be defined per
table so I don't think table properties is right. I think the right conditions
is client side config + server config response contains scan-planning.
I do agree we'll also need something from a server to indicate that a client
*must* do client side planning but I think that's something we can take in a
follow on. Once the spec is defined for that then I think we'd be able to come
back here and add a check (at some point?) that will first check if the table
requires scan planning based on the response from the server.
##########
core/src/main/java/org/apache/iceberg/RESTTableScan.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.ParserContext;
+import org.apache.iceberg.rest.PlanStatus;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.ResourcePaths;
+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.apache.iceberg.util.ParallelIterable;
+
+class RESTTableScan extends DataTableScan {
+ private final RESTClient client;
+ private final String path;
+ private final Supplier<Map<String, String>> headers;
+ private final TableOperations operations;
+ private final Table table;
+ private final ResourcePaths resourcePaths;
+ private final TableIdentifier tableIdentifier;
+
+ // TODO revisit if this property should be configurable
+ private static final int FETCH_PLANNING_SLEEP_DURATION_MS = 1000;
+
+ RESTTableScan(
+ Table table,
+ Schema schema,
+ TableScanContext context,
+ RESTClient client,
+ String path,
+ Supplier<Map<String, String>> headers,
+ TableOperations operations,
+ TableIdentifier tableIdentifier,
+ ResourcePaths resourcePaths) {
+ super(table, schema, context);
+ this.table = table;
+ this.client = client;
+ this.headers = headers;
+ this.path = path;
+ this.operations = operations;
+ this.tableIdentifier = tableIdentifier;
+ this.resourcePaths = resourcePaths;
+ }
+
+ @Override
+ protected TableScan newRefinedScan(
+ Table refinedTable, Schema refinedSchema, TableScanContext
refinedContext) {
+ return new RESTTableScan(
+ refinedTable,
+ refinedSchema,
+ refinedContext,
+ client,
+ path,
+ headers,
+ operations,
+ tableIdentifier,
+ resourcePaths);
+ }
+
+ @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 planTableScanRequestBuilder =
+ new PlanTableScanRequest.Builder()
+ .withSelect(selectedColumns)
+ .withFilter(filter())
+ .withCaseSensitive(isCaseSensitive())
+ .withStatsFields(statsFields);
+
+ if (startSnapshotId != null && endSnapshotId != null) {
+ planTableScanRequestBuilder
+ .withStartSnapshotId(startSnapshotId)
+ .withEndSnapshotId(endSnapshotId)
+ .withUseSnapshotSchema(true);
+
+ } else if (snapshotId != null) {
+ boolean useSnapShotSchema = snapshotId !=
table.currentSnapshot().snapshotId();
+ planTableScanRequestBuilder
+ .withSnapshotId(snapshotId)
+ .withUseSnapshotSchema(useSnapShotSchema);
+
+ } else {
+
planTableScanRequestBuilder.withSnapshotId(table().currentSnapshot().snapshotId());
+ }
+
+ return planTableScan(planTableScanRequestBuilder.build());
+ }
+
+ private CloseableIterable<FileScanTask> planTableScan(PlanTableScanRequest
planTableScanRequest) {
+ ParserContext context =
+ ParserContext.builder()
+ .add("specsById", table.specs())
+ .add("caseSensitive", context().caseSensitive())
+ .build();
+
+ PlanTableScanResponse response =
+ client.post(
+ resourcePaths.planTableScan(tableIdentifier),
+ planTableScanRequest,
+ PlanTableScanResponse.class,
+ headers.get(),
+ ErrorHandlers.defaultErrorHandler(),
+ stringStringMap -> {},
+ context);
+
+ PlanStatus planStatus = response.planStatus();
+ switch (planStatus) {
+ case COMPLETED:
+ return getScanTasksIterable(response.planTasks(),
response.fileScanTasks());
+ case SUBMITTED:
+ return fetchPlanningResult(response.planId());
+ case FAILED:
+ throw new IllegalStateException(
+ "Received \"failed\" status from service when planning a table
scan");
+ case CANCELLED:
+ throw new IllegalStateException(
+ "Received \"cancelled\" status from service when planning a table
scan");
+ default:
+ throw new RuntimeException(
+ String.format("Invalid planStatus during planTableScan: %s",
planStatus));
+ }
+ }
+
+ private CloseableIterable<FileScanTask> fetchPlanningResult(String planId) {
+ // TODO need to introduce a max wait time for this loop potentially
+ boolean planningFinished = false;
+ // we need to inject specById map here and also the caseSensitive
+ ParserContext context =
+ ParserContext.builder()
+ .add("specsById", table.specs())
+ .add("caseSensitive", context().caseSensitive())
+ .build();
+ while (!planningFinished) {
Review Comment:
Is this `planningFinished` state ever updated anywhere? It looks like in
every possible branch we'll either exit due to a success or throw. The one
exception is if it's repeatedly in submitted. But effectively as its written,
this is just a while(true)...
--
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]