amogh-jahagirdar commented on code in PR #11180: URL: https://github.com/apache/iceberg/pull/11180#discussion_r1797052043
########## core/src/main/java/org/apache/iceberg/rest/requests/FetchScanTasksRequest.java: ########## @@ -0,0 +1,45 @@ +/* + * 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.requests; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.RESTRequest; + +public class FetchScanTasksRequest implements RESTRequest { + + private String planTask; + + public FetchScanTasksRequest() { Review Comment: Same as the other empty constructors, I don't think these are needed now with the explicit serializer/deserializer implementations ########## core/src/main/java/org/apache/iceberg/RESTPlanningMode.java: ########## @@ -0,0 +1,47 @@ +/* + * 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.Locale; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public enum RESTPlanningMode { + REQUIRED("required"), + SUPPORTED("supported"), + UNSUPPORTED("unsupported"); Review Comment: I'm still going through the proposed spec change but I'm skeptical do we really need an explicit `unsupported`? Isn't the lack of presence of supported or required in the config map enough for the client to make a decision? ########## core/src/main/java/org/apache/iceberg/rest/requests/FetchScanTasksRequest.java: ########## @@ -0,0 +1,45 @@ +/* + * 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.requests; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.RESTRequest; + +public class FetchScanTasksRequest implements RESTRequest { + + private String planTask; + + public FetchScanTasksRequest() { + // Needed for Jackson Deserialization. + } + + public FetchScanTasksRequest(String planTask) { + this.planTask = planTask; + validate(); + } + + public String planTask() { + return planTask; + } + + @Override + public void validate() { + Preconditions.checkArgument(planTask != null, "Invalid request: planTask null"); + } Review Comment: I feel like this should just be inlined in the constructor ########## core/src/main/java/org/apache/iceberg/ScanTasksIterable.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.ArrayList; +import java.util.List; +import java.util.Map; +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.rest.ErrorHandlers; +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; + +public class ScanTasksIterable implements CloseableIterable<FileScanTask> { + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private final List<String> planTasks; + private final List<FileScanTask> fileScanTasks; + + public ScanTasksIterable( + List<String> planTasks, + List<FileScanTask> fileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers) { + this.planTasks = planTasks; + this.fileScanTasks = fileScanTasks; + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + } + + @Override + public CloseableIterator<FileScanTask> iterator() { + return new ScanTasksIterator( + planTasks, fileScanTasks, client, resourcePaths, tableIdentifier, headers); + } + + @Override + public void close() throws IOException {} + + private static class ScanTasksIterator implements CloseableIterator<FileScanTask> { + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private List<String> planTasks; + private List<FileScanTask> fileScanTasks; + + ScanTasksIterator( + List<String> planTasks, + List<FileScanTask> fileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers) { + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + this.planTasks = planTasks != null ? planTasks : new ArrayList<String>(); + this.fileScanTasks = fileScanTasks != null ? fileScanTasks : new ArrayList<FileScanTask>(); + } + + @Override + public boolean hasNext() { + if (!fileScanTasks.isEmpty()) { + // Have file scan tasks so continue to consume + return true; + } + // Out of file scan tasks, so need to now fetch more from each planTask + // Service can send back more planTasks which acts as pagination + if (!planTasks.isEmpty()) { + executeFetchScanTasks(planTasks.remove(0)); + // Make another hasNext() call, as more planTasks and fileScanTasks have been fetched + return hasNext(); + } + // we have no file scan tasks left to consume, and planTasks are exhausted + // so means we are finished + return false; + } + + @Override + public FileScanTask next() { + return fileScanTasks.remove(0); + } + + private void executeFetchScanTasks(String planTask) { Review Comment: nit: I think could just be called fetchScanTasks ########## core/src/main/java/org/apache/iceberg/ScanTasksIterable.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.ArrayList; +import java.util.List; +import java.util.Map; +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.rest.ErrorHandlers; +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; + +public class ScanTasksIterable implements CloseableIterable<FileScanTask> { + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private final List<String> planTasks; + private final List<FileScanTask> fileScanTasks; + + public ScanTasksIterable( + List<String> planTasks, + List<FileScanTask> fileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers) { + this.planTasks = planTasks; + this.fileScanTasks = fileScanTasks; + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + } + + @Override + public CloseableIterator<FileScanTask> iterator() { + return new ScanTasksIterator( + planTasks, fileScanTasks, client, resourcePaths, tableIdentifier, headers); + } + + @Override + public void close() throws IOException {} + + private static class ScanTasksIterator implements CloseableIterator<FileScanTask> { + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private List<String> planTasks; + private List<FileScanTask> fileScanTasks; + + ScanTasksIterator( + List<String> planTasks, + List<FileScanTask> fileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers) { + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + this.planTasks = planTasks != null ? planTasks : new ArrayList<String>(); + this.fileScanTasks = fileScanTasks != null ? fileScanTasks : new ArrayList<FileScanTask>(); + } + + @Override + public boolean hasNext() { + if (!fileScanTasks.isEmpty()) { + // Have file scan tasks so continue to consume + return true; + } + // Out of file scan tasks, so need to now fetch more from each planTask + // Service can send back more planTasks which acts as pagination + if (!planTasks.isEmpty()) { + executeFetchScanTasks(planTasks.remove(0)); + // Make another hasNext() call, as more planTasks and fileScanTasks have been fetched + return hasNext(); Review Comment: I see what you're getting at but I think in the implementation, you'd recurse and fetch more scan tasks for plan tasks on the next `hasNext` so a single hasNext call keeps cycling for as long as there are plan tasks, which doesn't seem right for an iterator. I think you'd just want to return `!fileScanTasks.isEmpty() || !planTasks.isEmpty()` We could do a higher degree level of prefetching but that's probably a bit more complex than it's worth right now. ########## core/src/main/java/org/apache/iceberg/ScanTasksIterable.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.ArrayList; +import java.util.List; +import java.util.Map; +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.rest.ErrorHandlers; +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; + +public class ScanTasksIterable implements CloseableIterable<FileScanTask> { + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private final List<String> planTasks; + private final List<FileScanTask> fileScanTasks; + + public ScanTasksIterable( + List<String> planTasks, + List<FileScanTask> fileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers) { + this.planTasks = planTasks; + this.fileScanTasks = fileScanTasks; + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + } + + @Override + public CloseableIterator<FileScanTask> iterator() { + return new ScanTasksIterator( + planTasks, fileScanTasks, client, resourcePaths, tableIdentifier, headers); + } + + @Override + public void close() throws IOException {} + + private static class ScanTasksIterator implements CloseableIterator<FileScanTask> { + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Supplier<Map<String, String>> headers; + private List<String> planTasks; + private List<FileScanTask> fileScanTasks; + + ScanTasksIterator( + List<String> planTasks, + List<FileScanTask> fileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Supplier<Map<String, String>> headers) { + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + this.planTasks = planTasks != null ? planTasks : new ArrayList<String>(); + this.fileScanTasks = fileScanTasks != null ? fileScanTasks : new ArrayList<FileScanTask>(); Review Comment: Probably should be a `LinkedList` instead of an arraylist since we're always removing from the front ########## core/src/main/java/org/apache/iceberg/TableScanContext.java: ########## @@ -35,7 +35,7 @@ /** Context object with optional arguments for a TableScan. */ @Value.Immutable -abstract class TableScanContext { +public abstract class TableScanContext { Review Comment: This change shouldn't be needed, checking out the code locally -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org