szehon-ho commented on code in PR #5077:
URL: https://github.com/apache/iceberg/pull/5077#discussion_r903116326
##########
api/src/main/java/org/apache/iceberg/Scan.java:
##########
@@ -29,7 +29,7 @@
* Scan objects are immutable and can be shared between threads. Refinement
methods, like
* {@link #select(Collection)} and {@link #filter(Expression)}, create new
TableScan instances.
*/
-public interface Scan<T extends Scan<T>> {
+public interface Scan<ThisT, T extends ScanTask, S extends ScanTaskGroup<T>> {
Review Comment:
Nit: maybe better type names than S/T, also javadoc for the types.
##########
api/src/main/java/org/apache/iceberg/BaseScanTaskGroup.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class BaseScanTaskGroup<T extends ScanTask> implements ScanTaskGroup<T>
{
+ private final List<T> tasks;
+
+ public BaseScanTaskGroup(List<T> tasks) {
+ Preconditions.checkNotNull(tasks, "tasks cannot be null");
+ this.tasks = Lists.newArrayList(tasks);
Review Comment:
Just use ImmutableList.of() and then return it directly?
##########
api/src/main/java/org/apache/iceberg/Scan.java:
##########
@@ -113,26 +113,23 @@
Schema schema();
/**
- * Plan the {@link FileScanTask files} that will be read by this scan.
+ * Plan tasks for this scan without trying to balance the work.
Review Comment:
Maybe just me but 'balance the work' seems a bit ambigious at first.
Is it right to just say, 'Plan tasks each reading a single file' ? (It may
not be)
##########
api/src/main/java/org/apache/iceberg/Scan.java:
##########
@@ -113,26 +113,23 @@
Schema schema();
/**
- * Plan the {@link FileScanTask files} that will be read by this scan.
+ * Plan tasks for this scan without trying to balance the work.
* <p>
- * Each file has a residual expression that should be applied to filter the
file's rows.
- * <p>
- * This simple plan returns file scans for each file from position 0 to the
file's length. For
- * planning that will combine small files, split large files, and attempt to
balance work, use
- * {@link #planTasks()} instead.
+ * Use {@link #planTasks()} for planning that will attempt to balance the
work
+ * by combining small or splitting large files.
Review Comment:
On similar note, how about 'for planning balanced tasks. Each task can read
either a single file, a part of a file, or multiple files'.
##########
api/src/main/java/org/apache/iceberg/BaseScanTaskGroup.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class BaseScanTaskGroup<T extends ScanTask> implements ScanTaskGroup<T>
{
+ private final List<T> tasks;
+
+ public BaseScanTaskGroup(List<T> tasks) {
Review Comment:
This class is a bit asymmetrical, takes a List but returns Iterable, can it
be normalized?
##########
api/src/main/java/org/apache/iceberg/CombinedScanTask.java:
##########
@@ -24,13 +24,18 @@
/**
* A scan task made of several ranges from files.
*/
-public interface CombinedScanTask extends ScanTask {
+public interface CombinedScanTask extends ScanTaskGroup<FileScanTask> {
/**
* Return the {@link FileScanTask tasks} in this combined task.
* @return a Collection of FileScanTask instances.
*/
Collection<FileScanTask> files();
Review Comment:
Is the plan to deprecate 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]