mxm commented on code in PR #15996: URL: https://github.com/apache/iceberg/pull/15996#discussion_r3225144947
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/EqualityConvertReader.java: ########## @@ -0,0 +1,211 @@ +/* + * 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.flink.maintenance.operator; + +import java.util.List; +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.Accessor; +import org.apache.iceberg.AddedRowsScanTask; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +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.ContentFileUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Parallel reader that processes {@link ReadCommand}s from the planner and emits {@link + * IndexCommand}s. Dispatches on the wrapped {@link ContentScanTask} subtype: {@link + * AddedRowsScanTask} (data file) emits {@code ADD_DATA_ROW} per row, our equality-delete task emits + * {@code RESOLVE_DELETE} per row, {@link PositionDeletesScanTask} emits {@link DVPosition} records + * via side output. + */ +@Internal +public class EqualityConvertReader extends ProcessFunction<ReadCommand, IndexCommand> { + + private static final Logger LOG = LoggerFactory.getLogger(EqualityConvertReader.class); + + public static final OutputTag<DVPosition> DV_POSITION_STREAM = + new OutputTag<>("dv-position-stream") {}; + + private final TableLoader tableLoader; + + private transient Table table; + private transient EqualityFieldSerializer fieldSerializer; + private transient DeleteLoader deleteLoader; + + public EqualityConvertReader(TableLoader tableLoader) { + this.tableLoader = tableLoader; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + table = tableLoader.loadTable(); + fieldSerializer = new EqualityFieldSerializer(); + deleteLoader = new BaseDeleteLoader(deleteFile -> table.io().newInputFile(deleteFile)); + } + + @Override + public void processElement(ReadCommand cmd, Context ctx, Collector<IndexCommand> out) + throws Exception { + ContentScanTask<?> task = cmd.task(); + ContentFile<?> file = task.file(); + try { + if (task instanceof PositionDeletesScanTask) { + readPosDeleteFile((PositionDeletesScanTask) task, ctx); + return; + } + + Schema keySchema = findSchema(cmd, file); + boolean isDataFile = task instanceof AddedRowsScanTask; + Schema readSchema = isDataFile ? appendRowPosition(keySchema) : keySchema; + + PositionDeleteIndex existingDeletes = + isDataFile ? loadExistingDVs((AddedRowsScanTask) task, file.location()) : null; + + int specId = file.specId(); + StructLike partition = file.partition(); + Types.StructType partitionType = table.specs().get(specId).partitionType(); + + InputFile input = table.io().newInputFile(file.location()); + ReadBuilder<Record, Schema> builder = + FormatModelRegistry.readBuilder(file.format(), Record.class, input); + try (CloseableIterable<Record> records = builder.project(readSchema).build()) { + Accessor<StructLike> posAccessor = + isDataFile ? readSchema.accessorForField(MetadataColumns.ROW_POSITION.fieldId()) : null; + for (Record record : records) { + SerializedEqualityValues key = + fieldSerializer.serializeKey( + record, keySchema.asStruct(), specId, partition, partitionType); + if (isDataFile) { + long position = (long) posAccessor.get(record); + if (existingDeletes != null && existingDeletes.isDeleted(position)) { + continue; + } + + out.collect( + IndexCommand.addDataRow(cmd.mainSnapshotId(), key, file.location(), position)); + } else { + out.collect(IndexCommand.resolveDelete(cmd.mainSnapshotId(), key)); + } + } + } + } catch (Exception e) { + LOG.error("Reader failed to process command for file={}", file.location(), e); + ctx.output(TaskResultAggregator.ERROR_STREAM, e); + ctx.output(DV_POSITION_STREAM, DVPosition.abort()); + } + } + + @Override + public void close() throws Exception { + super.close(); + tableLoader.close(); + } + + private void readPosDeleteFile(PositionDeletesScanTask task, Context ctx) throws Exception { + Schema pathPosSchema = DeleteSchemaUtil.pathPosSchema(); + InputFile input = table.io().newInputFile(task.file().location()); + ReadBuilder<Record, Schema> builder = + FormatModelRegistry.readBuilder(task.file().format(), Record.class, input); + try (CloseableIterable<Record> records = builder.project(pathPosSchema).build()) { + for (Record record : records) { + String dataFilePath = (String) record.get(0); + long position = (Long) record.get(1); + ctx.output(DV_POSITION_STREAM, new DVPosition(dataFilePath, position)); + } + } + } + + private Schema appendRowPosition(Schema keySchema) { + List<Types.NestedField> columns = Lists.newArrayList(keySchema.columns()); + columns.add(MetadataColumns.ROW_POSITION); + return new Schema(columns); + } + + private PositionDeleteIndex loadExistingDVs(AddedRowsScanTask task, String dataFilePath) { + List<DeleteFile> dvs = Lists.newArrayList(); + for (DeleteFile deleteFile : task.deletes()) { + if (ContentFileUtil.isDV(deleteFile)) { + dvs.add(deleteFile); + } + } + + if (dvs.isEmpty()) { + return null; + } + + return deleteLoader.loadPositionDeletes(dvs, dataFilePath); + } + + private Schema findSchema(ReadCommand cmd, ContentFile<?> file) { + Set<Integer> requestedIds = Sets.newHashSet(cmd.equalityFieldIds()); + Schema keySchema = TypeUtil.select(table.schema(), requestedIds); + + if (!keySchema.columns().isEmpty()) { + return keySchema; + } Review Comment: This is no longer needed. Equality fields are fixed now, set at the builder level. -- 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]
