rdblue commented on a change in pull request #1352:
URL: https://github.com/apache/iceberg/pull/1352#discussion_r473202172



##########
File path: data/src/main/java/org/apache/iceberg/data/GenericReader.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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.data;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.orc.GenericOrcReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.Deletes;
+import org.apache.iceberg.expressions.Evaluator;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.ProjectStructLike;
+import org.apache.iceberg.util.StructLikeSet;
+
+class GenericReader implements Serializable {
+  private static final Schema POS_DELETE_SCHEMA = new Schema(
+      MetadataColumns.DELETE_FILE_PATH,
+      MetadataColumns.DELETE_FILE_POS);
+
+  private final FileIO io;
+  private final Schema projection;
+  private final boolean caseSensitive;
+  private final boolean reuseContainers;
+
+  GenericReader(TableScan scan, boolean reuseContainers) {
+    this.io = scan.table().io();
+    this.projection = scan.schema();
+    this.caseSensitive = scan.isCaseSensitive();
+    this.reuseContainers = reuseContainers;
+  }
+
+  CloseableIterator<Record> open(CloseableIterable<CombinedScanTask> tasks) {
+    Iterable<FileScanTask> fileTasks = 
Iterables.concat(Iterables.transform(tasks, CombinedScanTask::files));
+    return CloseableIterable.concat(Iterables.transform(fileTasks, 
this::open)).iterator();
+  }
+
+  public CloseableIterable<Record> open(CombinedScanTask task) {
+    return new CombinedTaskIterable(task);
+  }
+
+  public CloseableIterable<Record> open(FileScanTask task) {
+    List<DeleteFile> posDeletes = Lists.newArrayList();
+    List<DeleteFile> eqDeletes = Lists.newArrayList();
+    for (DeleteFile delete : task.deletes()) {
+      switch (delete.content()) {
+        case POSITION_DELETES:
+          posDeletes.add(delete);
+          break;
+        case EQUALITY_DELETES:
+          eqDeletes.add(delete);
+          break;
+        default:
+          throw new UnsupportedOperationException("Unknown delete file 
content: " + delete.content());
+      }
+    }
+
+    Schema fileProjection = fileProjection(!posDeletes.isEmpty());
+
+    CloseableIterable<Record> records = openFile(task, fileProjection);
+    records = applyPosDeletes(records, fileProjection, task.file().path(), 
posDeletes, task.file());
+    records = applyEqDeletes(records, fileProjection, eqDeletes, task.file());
+    records = applyResidual(records, fileProjection, task.residual());
+
+    return records;
+  }
+
+  private Schema fileProjection(boolean hasPosDeletes) {
+    if (hasPosDeletes) {
+      List<Types.NestedField> columns = 
Lists.newArrayList(projection.columns());
+      columns.add(MetadataColumns.ROW_POSITION);
+      return new Schema(columns);
+    }
+
+    return projection;
+  }
+
+  private CloseableIterable<Record> applyResidual(CloseableIterable<Record> 
records, Schema recordSchema,
+                                                  Expression residual) {
+    if (residual != null && residual != Expressions.alwaysTrue()) {
+      InternalRecordWrapper wrapper = new 
InternalRecordWrapper(recordSchema.asStruct());
+      Evaluator filter = new Evaluator(recordSchema.asStruct(), residual, 
caseSensitive);
+      return CloseableIterable.filter(records, record -> 
filter.eval(wrapper.wrap(record)));
+    }
+
+    return records;
+  }
+
+  private CloseableIterable<Record> applyEqDeletes(CloseableIterable<Record> 
records, Schema recordSchema,
+                                                   List<DeleteFile> eqDeletes, 
DataFile dataFile) {
+    if (eqDeletes.isEmpty()) {
+      return records;
+    }
+
+    Multimap<Set<Integer>, DeleteFile> filesByDeleteIds = 
Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList);
+    for (DeleteFile delete : eqDeletes) {
+      filesByDeleteIds.put(Sets.newHashSet(delete.equalityFieldIds()), delete);
+    }
+
+    CloseableIterable<Record> filteredRecords = records;
+    for (Map.Entry<Set<Integer>, Collection<DeleteFile>> entry : 
filesByDeleteIds.asMap().entrySet()) {
+      Set<Integer> ids = entry.getKey();
+      Iterable<DeleteFile> deletes = entry.getValue();
+
+      Schema deleteSchema = TypeUtil.select(recordSchema, ids);
+      int[] orderedIds = 
deleteSchema.columns().stream().mapToInt(Types.NestedField::fieldId).toArray();

Review comment:
       I intended to support nested fields, but I think there is still a 
problem here.
   
   For example, schema `1: id bigint, 2: location struct<3: city, 4: postcode>` 
and delete `postcode='HG1 2AD'`. The delete id set is `{4}`, but the delete 
schema from the previous line is `2: location struct<4: postcode>`. The 
projection created from `orderedIds` should use ID 2 because that's the 
top-level field that needs to be pulled from a full record.
   
   The problem is that the nested struct also needs a projection, which I 
missed. I think we probably need to build the projection using a visitor and 
correctly wrap nested structs.




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to