szehon-ho commented on code in PR #6365:
URL: https://github.com/apache/iceberg/pull/6365#discussion_r1080709554


##########
core/src/main/java/org/apache/iceberg/PositionDeletesTable.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 static 
org.apache.iceberg.MetadataColumns.POSITION_DELETE_TABLE_FILE_PATH_ID;
+import static 
org.apache.iceberg.MetadataColumns.POSITION_DELETE_TABLE_PARTITION_FIELD_ID;
+import static org.apache.iceberg.MetadataColumns.POSITION_DELETE_TABLE_SPEC_ID;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.ResidualEvaluator;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ParallelIterable;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+public class PositionDeletesTable extends BaseMetadataTable {
+
+  private final Schema schema;
+
+  PositionDeletesTable(TableOperations ops, Table table) {
+    super(ops, table, table.name() + ".position_deletes");
+    this.schema = calculateSchema();
+  }
+
+  PositionDeletesTable(TableOperations ops, Table table, String name) {
+    super(ops, table, name);
+    this.schema = calculateSchema();
+  }
+
+  @Override
+  MetadataTableType metadataTableType() {
+    return MetadataTableType.POSITION_DELETES;
+  }
+
+  @Override
+  public TableScan newScan() {
+    throw new UnsupportedOperationException(
+        "Cannot create TableScan from table of type POSITION_DELETES");
+  }
+
+  @Override
+  public BatchScan newBatchScan() {
+    return new PositionDeletesBatchScan(operations(), table(), schema());
+  }
+
+  @Override
+  public Schema schema() {
+    return schema;
+  }
+
+  private Schema calculateSchema() {
+    Types.StructType partitionType = Partitioning.partitionType(table());
+    Schema result =
+        new Schema(
+            MetadataColumns.DELETE_FILE_PATH,
+            MetadataColumns.DELETE_FILE_POS,
+            Types.NestedField.optional(
+                MetadataColumns.DELETE_FILE_ROW_FIELD_ID,
+                "row",
+                table().schema().asStruct(),
+                MetadataColumns.DELETE_FILE_ROW_DOC),
+            Types.NestedField.required(
+                POSITION_DELETE_TABLE_PARTITION_FIELD_ID,
+                "partition",
+                partitionType,
+                "Partition that position delete row belongs to"),
+            Types.NestedField.required(
+                POSITION_DELETE_TABLE_SPEC_ID,
+                "spec_id",
+                Types.IntegerType.get(),
+                "Spec ID of the file that the position delete row belongs to"),
+            Types.NestedField.required(
+                POSITION_DELETE_TABLE_FILE_PATH_ID,
+                "delete_file_path",
+                Types.StringType.get(),
+                "Path of the delete file that the position delete row belongs 
to"));
+
+    if (partitionType.fields().size() > 0) {
+      return result;
+    } else {
+      // avoid returning an empty struct, which is not always supported.
+      // instead, drop the partition field
+      return TypeUtil.selectNot(result, 
Sets.newHashSet(POSITION_DELETE_TABLE_PARTITION_FIELD_ID));
+    }
+  }
+
+  public static class PositionDeletesBatchScan
+      extends SnapshotScan<BatchScan, ScanTask, ScanTaskGroup<ScanTask>> 
implements BatchScan {
+
+    protected PositionDeletesBatchScan(TableOperations ops, Table table, 
Schema schema) {
+      super(ops, table, schema, new TableScanContext());
+    }
+
+    protected PositionDeletesBatchScan(
+        TableOperations ops, Table table, Schema schema, TableScanContext 
context) {
+      super(ops, table, schema, context);
+    }
+
+    @Override
+    protected PositionDeletesBatchScan newRefinedScan(
+        TableOperations newOps, Table newTable, Schema newSchema, 
TableScanContext newContext) {
+      return new PositionDeletesBatchScan(newOps, newTable, newSchema, 
newContext);
+    }
+
+    @Override
+    public CloseableIterable<ScanTaskGroup<ScanTask>> planTasks() {
+      return TableScanUtil.planTaskGroups(
+          planFiles(), targetSplitSize(), splitLookback(), 
splitOpenFileCost());
+    }
+
+    protected List<String> scanColumns() {
+      return context().returnColumnStats() ? DELETE_SCAN_WITH_STATS_COLUMNS : 
DELETE_SCAN_COLUMNS;
+    }
+
+    @Override
+    protected CloseableIterable<ScanTask> doPlanFiles() {
+      Expression rowFilter = filter();
+      String schemaString = SchemaParser.toJson(tableSchema());
+
+      // prepare transformed partition specs and caches
+      Map<Integer, PartitionSpec> transformedSpecs =
+          table().specs().values().stream()
+              .map(spec -> transformSpec(tableSchema(), spec))
+              .collect(Collectors.toMap(PartitionSpec::specId, spec -> spec));
+
+      LoadingCache<Integer, ResidualEvaluator> residualCache =
+          Caffeine.newBuilder()
+              .build(
+                  specId -> {
+                    PartitionSpec spec = transformedSpecs.get(specId);
+                    Expression filter =
+                        context().ignoreResiduals() ? Expressions.alwaysTrue() 
: filter();
+                    return ResidualEvaluator.of(spec, filter, 
isCaseSensitive());
+                  });
+      LoadingCache<Integer, String> specStringCache =
+          Caffeine.newBuilder()
+              .build(
+                  specId -> {
+                    PartitionSpec spec = transformedSpecs.get(specId);
+                    return PartitionSpecParser.toJson(spec);
+                  });
+
+      // iterate through delete manifests
+      CloseableIterable<ManifestFile> deleteManifests =
+          
CloseableIterable.withNoopClose(snapshot().deleteManifests(tableOps().io()));
+      CloseableIterable<CloseableIterable<ScanTask>> results =
+          CloseableIterable.transform(
+              deleteManifests,
+              manifest -> {
+                // Filter partitions
+                CloseableIterable<ManifestEntry<DeleteFile>> deleteFileEntries 
=
+                    ManifestFiles.readDeleteManifest(manifest, 
tableOps().io(), transformedSpecs)
+                        .caseSensitive(isCaseSensitive())
+                        .select(scanColumns())
+                        .filterRows(rowFilter)
+                        .liveEntries();
+
+                // Filter delete file type
+                CloseableIterable<ManifestEntry<DeleteFile>> 
positionDeleteEntries =
+                    CloseableIterable.filter(
+                        deleteFileEntries,
+                        entry -> 
entry.file().content().equals(FileContent.POSITION_DELETES));
+
+                Types.StructType partitionType = 
Partitioning.partitionType(table());
+
+                return CloseableIterable.transform(
+                    positionDeleteEntries,
+                    entry -> {
+                      int specId = entry.file().specId();
+                      return new PositionDeleteScanTaskImpl(
+                          entry.file().copyWithoutStats(),
+                          schemaString,
+                          specStringCache.get(specId),
+                          residualCache.get(specId),
+                          partitionType);
+                    });
+              });
+
+      return new ParallelIterable<>(results, planExecutor());
+    }
+  }
+
+  /** Scan task for position delete files */
+  public interface PositionDeleteScanTask extends ContentScanTask<DeleteFile> {
+
+    /**
+     * Utility method to get constant values of rows to be scanned by this 
task. The following
+     * columns are constants of each PositionDeletesFileScanTask: * spec_id * 
partition *
+     * delete_file_path
+     *
+     * @return a map of column id to constant values returned by this task type
+     */
+    Map<Integer, ?> constantsMap();

Review Comment:
   Yea didn't want to duplicate the code.  Took your earlier suggestion and 
just used the existing MetadataColumn ids so we can re-use PartitionUtil as is 
when we implement the spark read side.  However, this table still declares them 
not not as metadata columns per-se, but as declared.



-- 
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

Reply via email to