luoyuxia commented on code in PR #1555: URL: https://github.com/apache/fluss/pull/1555#discussion_r2282416787
########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/writer/DeltaTaskWriter.java: ########## @@ -0,0 +1,112 @@ +/* + * 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 com.alibaba.fluss.lake.iceberg.tiering.writer; + +import com.alibaba.fluss.lake.iceberg.tiering.RecordWriter; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.record.LogRecord; +import com.alibaba.fluss.shaded.guava32.com.google.common.collect.Lists; +import com.alibaba.fluss.types.RowType; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.util.ArrayUtil; + +import javax.annotation.Nullable; + +import java.util.List; + +/** A {@link RecordWriter} to write to Iceberg's primary-key table. */ +public class DeltaTaskWriter extends RecordWriter { + + public DeltaTaskWriter( + Table icebergTable, + RowType flussRowType, + TableBucket tableBucket, + @Nullable String partition, + List<String> partitionKeys, + FileFormat format, + OutputFileFactory outputFileFactory, + long targetFileSize) { + super( + createTaskWriter(icebergTable, format, outputFileFactory, targetFileSize), + icebergTable.schema(), + flussRowType, + tableBucket, + partition, + partitionKeys); + } + + private static TaskWriter<Record> createTaskWriter( + Table icebergTable, + FileFormat format, + OutputFileFactory outputFileFactory, + long targetFileSize) { + + FileAppenderFactory<Record> appenderFactory = + new GenericAppenderFactory( + icebergTable.schema(), + icebergTable.spec(), + ArrayUtil.toIntArray( + Lists.newArrayList(icebergTable.schema().identifierFieldIds())), + icebergTable.schema(), + null); + + List<String> columns = Lists.newArrayList(); Review Comment: Can we avoid to using guava Lists? I'm afraid of there may be class not found issue if lacking guava in some times. ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/writer/GenericTaskDeltaWriter.java: ########## @@ -0,0 +1,85 @@ +/* + * 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 com.alibaba.fluss.lake.iceberg.tiering.writer; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +import java.io.IOException; + +/** A generic task equality delta writer. * */ +class GenericTaskDeltaWriter extends BaseTaskWriter<Record> { + private final GenericEqualityDeltaWriter deltaWriter; + + public GenericTaskDeltaWriter( + Schema schema, + Schema deleteSchema, + PartitionSpec spec, + FileFormat format, + FileAppenderFactory<Record> appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.deltaWriter = new GenericEqualityDeltaWriter(null, schema, deleteSchema); + } + + @Override + public void write(Record row) throws IOException { + deltaWriter.write(row); + } + + public void delete(Record row) throws IOException { + deltaWriter.delete(row); + } + + // The caller of this function is responsible for passing in a record with only the key fields + public void deleteKey(Record key) throws IOException { Review Comment: nit: not used, can be removed? ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java: ########## @@ -88,22 +95,46 @@ public long commit(IcebergCommittable committable, Map<String, String> snapshotP try { // Refresh table to get latest metadata icebergTable.refresh(); - // Simple append-only case: only data files, no delete files or compaction - AppendFiles appendFiles = icebergTable.newAppend(); - for (DataFile dataFile : committable.getDataFiles()) { - appendFiles.appendFile(dataFile); - } - if (!committable.getDeleteFiles().isEmpty()) { - throw new IllegalStateException( - "Delete files are not supported in append-only mode. " - + "Found " - + committable.getDeleteFiles().size() - + " delete files."); - } - addFlussProperties(appendFiles, snapshotProperties); + if (committable.getDeleteFiles().isEmpty()) { + // Simple append-only case: only data files, no delete files or compaction + AppendFiles appendFiles = icebergTable.newAppend(); + for (DataFile dataFile : committable.getDataFiles()) { + appendFiles.appendFile(dataFile); + } + if (!committable.getDeleteFiles().isEmpty()) { + throw new IllegalStateException( + "Delete files are not supported in append-only mode. " + + "Found " + + committable.getDeleteFiles().size() + + " delete files."); + } + + addFlussProperties(appendFiles, snapshotProperties); - appendFiles.commit(); + appendFiles.commit(); + } else { + // Row delta validations are not needed for streaming changes that write equality + // deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries + // may + // push deletes further in the future, but do not affect correctness. Position + // deletes + // committed to the table in this path are used only to delete rows from data files + // that are + // being added in this commit. There is no way for data files added along with the + // delete + // files to be concurrently removed, so there is no need to validate the files + // referenced by + // the position delete files that are being committed. + RowDelta rowDelta = icebergTable.newRowDelta(); + Arrays.stream(committable.getDataFiles().stream().toArray(DataFile[]::new)) + .forEach(rowDelta::addRows); + Arrays.stream(committable.getDeleteFiles().stream().toArray(DeleteFile[]::new)) + .forEach(rowDelta::addDeletes); + snapshotProperties.forEach(rowDelta::set); Review Comment: In method `abort`, please also delete `DeleteFiles`. ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java: ########## @@ -88,22 +95,46 @@ public long commit(IcebergCommittable committable, Map<String, String> snapshotP try { // Refresh table to get latest metadata icebergTable.refresh(); - // Simple append-only case: only data files, no delete files or compaction - AppendFiles appendFiles = icebergTable.newAppend(); - for (DataFile dataFile : committable.getDataFiles()) { - appendFiles.appendFile(dataFile); - } - if (!committable.getDeleteFiles().isEmpty()) { - throw new IllegalStateException( - "Delete files are not supported in append-only mode. " - + "Found " - + committable.getDeleteFiles().size() - + " delete files."); - } - addFlussProperties(appendFiles, snapshotProperties); + if (committable.getDeleteFiles().isEmpty()) { + // Simple append-only case: only data files, no delete files or compaction + AppendFiles appendFiles = icebergTable.newAppend(); + for (DataFile dataFile : committable.getDataFiles()) { + appendFiles.appendFile(dataFile); + } + if (!committable.getDeleteFiles().isEmpty()) { + throw new IllegalStateException( + "Delete files are not supported in append-only mode. " + + "Found " + + committable.getDeleteFiles().size() + + " delete files."); + } + + addFlussProperties(appendFiles, snapshotProperties); - appendFiles.commit(); + appendFiles.commit(); + } else { + // Row delta validations are not needed for streaming changes that write equality + // deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries + // may + // push deletes further in the future, but do not affect correctness. Position + // deletes + // committed to the table in this path are used only to delete rows from data files + // that are + // being added in this commit. There is no way for data files added along with the + // delete + // files to be concurrently removed, so there is no need to validate the files + // referenced by + // the position delete files that are being committed. + RowDelta rowDelta = icebergTable.newRowDelta(); + Arrays.stream(committable.getDataFiles().stream().toArray(DataFile[]::new)) + .forEach(rowDelta::addRows); + Arrays.stream(committable.getDeleteFiles().stream().toArray(DeleteFile[]::new)) + .forEach(rowDelta::addDeletes); + snapshotProperties.forEach(rowDelta::set); Review Comment: ```suggestion addFlussProperties(rowDelta, snapshotProperties); ``` Change method addFlussProperties to ``` private void addFlussProperties( SnapshotUpdate<?> snapshotUpdate, Map<String, String> snapshotProperties) { snapshotUpdate.set("commit-user", FLUSS_LAKE_TIERING_COMMIT_USER); for (Map.Entry<String, String> entry : snapshotProperties.entrySet()) { snapshotUpdate.set(entry.getKey(), entry.getValue()); } } ``` Otherwise, we'll miss setting `commit-user` property ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/writer/DeltaTaskWriter.java: ########## @@ -0,0 +1,112 @@ +/* + * 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 com.alibaba.fluss.lake.iceberg.tiering.writer; + +import com.alibaba.fluss.lake.iceberg.tiering.RecordWriter; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.record.LogRecord; +import com.alibaba.fluss.shaded.guava32.com.google.common.collect.Lists; +import com.alibaba.fluss.types.RowType; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.util.ArrayUtil; + +import javax.annotation.Nullable; + +import java.util.List; + +/** A {@link RecordWriter} to write to Iceberg's primary-key table. */ +public class DeltaTaskWriter extends RecordWriter { + + public DeltaTaskWriter( + Table icebergTable, + RowType flussRowType, + TableBucket tableBucket, + @Nullable String partition, + List<String> partitionKeys, + FileFormat format, + OutputFileFactory outputFileFactory, + long targetFileSize) { + super( + createTaskWriter(icebergTable, format, outputFileFactory, targetFileSize), + icebergTable.schema(), + flussRowType, + tableBucket, + partition, + partitionKeys); + } + + private static TaskWriter<Record> createTaskWriter( + Table icebergTable, + FileFormat format, + OutputFileFactory outputFileFactory, + long targetFileSize) { + + FileAppenderFactory<Record> appenderFactory = + new GenericAppenderFactory( + icebergTable.schema(), + icebergTable.spec(), + ArrayUtil.toIntArray( + Lists.newArrayList(icebergTable.schema().identifierFieldIds())), + icebergTable.schema(), + null); + + List<String> columns = Lists.newArrayList(); + for (Integer fieldId : icebergTable.schema().identifierFieldIds()) { + columns.add(icebergTable.schema().findField(fieldId).name()); + } + Schema deleteSchema = icebergTable.schema().select(columns); + return new GenericTaskDeltaWriter( + icebergTable.schema(), + deleteSchema, + icebergTable.spec(), + format, + appenderFactory, + outputFileFactory, + icebergTable.io(), + targetFileSize); + } + + @Override + public void write(LogRecord record) throws Exception { + GenericTaskDeltaWriter deltaWriter = (GenericTaskDeltaWriter) taskWriter; + flussRecordAsIcebergRecord.setFlussRecord(record); + switch (record.getChangeType()) { + case INSERT: + case UPDATE_AFTER: + deltaWriter.write(flussRecordAsIcebergRecord); + break; + case UPDATE_BEFORE: Review Comment: nit: ``` case UPDATE_BEFORE: case DELETE: deltaWriter.delete(flussRecordAsIcebergRecord); break; ``` ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/writer/GenericTaskDeltaWriter.java: ########## @@ -0,0 +1,85 @@ +/* + * 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 com.alibaba.fluss.lake.iceberg.tiering.writer; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +import java.io.IOException; + +/** A generic task equality delta writer. * */ +class GenericTaskDeltaWriter extends BaseTaskWriter<Record> { + private final GenericEqualityDeltaWriter deltaWriter; + + public GenericTaskDeltaWriter( + Schema schema, + Schema deleteSchema, + PartitionSpec spec, + FileFormat format, + FileAppenderFactory<Record> appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.deltaWriter = new GenericEqualityDeltaWriter(null, schema, deleteSchema); + } + + @Override + public void write(Record row) throws IOException { + deltaWriter.write(row); + } + + public void delete(Record row) throws IOException { + deltaWriter.delete(row); + } + + // The caller of this function is responsible for passing in a record with only the key fields + public void deleteKey(Record key) throws IOException { + deltaWriter.deleteKey(key); + } + + @Override + public void close() throws IOException { + deltaWriter.close(); + } + + private class GenericEqualityDeltaWriter extends BaseEqualityDeltaWriter { + private GenericEqualityDeltaWriter( + PartitionKey partition, Schema schema, Schema eqDeleteSchema) { + super(partition, schema, eqDeleteSchema); + } + + @Override + protected StructLike asStructLike(Record row) { + return row; + } + + @Override + protected StructLike asStructLikeKey(Record data) { Review Comment: nit: ```suggestion protected StructLike asStructLikeKey(Record record) { ``` ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java: ########## @@ -88,22 +95,46 @@ public long commit(IcebergCommittable committable, Map<String, String> snapshotP try { // Refresh table to get latest metadata icebergTable.refresh(); - // Simple append-only case: only data files, no delete files or compaction - AppendFiles appendFiles = icebergTable.newAppend(); - for (DataFile dataFile : committable.getDataFiles()) { - appendFiles.appendFile(dataFile); - } - if (!committable.getDeleteFiles().isEmpty()) { - throw new IllegalStateException( - "Delete files are not supported in append-only mode. " - + "Found " - + committable.getDeleteFiles().size() - + " delete files."); - } - addFlussProperties(appendFiles, snapshotProperties); + if (committable.getDeleteFiles().isEmpty()) { + // Simple append-only case: only data files, no delete files or compaction + AppendFiles appendFiles = icebergTable.newAppend(); + for (DataFile dataFile : committable.getDataFiles()) { + appendFiles.appendFile(dataFile); + } + if (!committable.getDeleteFiles().isEmpty()) { + throw new IllegalStateException( + "Delete files are not supported in append-only mode. " + + "Found " + + committable.getDeleteFiles().size() + + " delete files."); + } + + addFlussProperties(appendFiles, snapshotProperties); - appendFiles.commit(); + appendFiles.commit(); + } else { + // Row delta validations are not needed for streaming changes that write equality + // deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries + // may + // push deletes further in the future, but do not affect correctness. Position + // deletes + // committed to the table in this path are used only to delete rows from data files + // that are + // being added in this commit. There is no way for data files added along with the + // delete + // files to be concurrently removed, so there is no need to validate the files + // referenced by + // the position delete files that are being committed. + RowDelta rowDelta = icebergTable.newRowDelta(); Review Comment: nit: ``` Arrays.stream(committable.getDataFiles().toArray(DataFile[]::new)) .forEach(rowDelta::addRows); Arrays.stream(committable.getDeleteFiles().toArray(DeleteFile[]::new)) .forEach(rowDelta::addDeletes); ``` ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java: ########## @@ -88,22 +95,46 @@ public long commit(IcebergCommittable committable, Map<String, String> snapshotP try { // Refresh table to get latest metadata icebergTable.refresh(); - // Simple append-only case: only data files, no delete files or compaction - AppendFiles appendFiles = icebergTable.newAppend(); - for (DataFile dataFile : committable.getDataFiles()) { - appendFiles.appendFile(dataFile); - } - if (!committable.getDeleteFiles().isEmpty()) { - throw new IllegalStateException( - "Delete files are not supported in append-only mode. " - + "Found " - + committable.getDeleteFiles().size() - + " delete files."); - } - addFlussProperties(appendFiles, snapshotProperties); + if (committable.getDeleteFiles().isEmpty()) { + // Simple append-only case: only data files, no delete files or compaction + AppendFiles appendFiles = icebergTable.newAppend(); + for (DataFile dataFile : committable.getDataFiles()) { + appendFiles.appendFile(dataFile); + } + if (!committable.getDeleteFiles().isEmpty()) { + throw new IllegalStateException( + "Delete files are not supported in append-only mode. " + + "Found " + + committable.getDeleteFiles().size() + + " delete files."); + } + + addFlussProperties(appendFiles, snapshotProperties); - appendFiles.commit(); + appendFiles.commit(); + } else { + // Row delta validations are not needed for streaming changes that write equality Review Comment: nit: could you please adjust the comments to make it compact? Now, it has too many lines. ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/IcebergLakeCommitter.java: ########## @@ -88,22 +95,46 @@ public long commit(IcebergCommittable committable, Map<String, String> snapshotP try { // Refresh table to get latest metadata icebergTable.refresh(); - // Simple append-only case: only data files, no delete files or compaction - AppendFiles appendFiles = icebergTable.newAppend(); - for (DataFile dataFile : committable.getDataFiles()) { - appendFiles.appendFile(dataFile); - } - if (!committable.getDeleteFiles().isEmpty()) { - throw new IllegalStateException( - "Delete files are not supported in append-only mode. " - + "Found " - + committable.getDeleteFiles().size() - + " delete files."); - } - addFlussProperties(appendFiles, snapshotProperties); + if (committable.getDeleteFiles().isEmpty()) { + // Simple append-only case: only data files, no delete files or compaction + AppendFiles appendFiles = icebergTable.newAppend(); + for (DataFile dataFile : committable.getDataFiles()) { + appendFiles.appendFile(dataFile); + } + if (!committable.getDeleteFiles().isEmpty()) { Review Comment: nit: we don't need to check is empty or not again ########## fluss-lake/fluss-lake-iceberg/src/test/java/com/alibaba/fluss/lake/iceberg/IcebergTieringTest.java: ########## @@ -258,4 +351,34 @@ private void verifyLogTableRecords( .isEqualTo(expectRecord.timestamp()); } } + + private void verifyPrimaryKeyTableRecord( Review Comment: Could `verifyLogTableRecords` and `verifyPrimaryKeyTableRecord` be merged into just one method? ########## fluss-lake/fluss-lake-iceberg/src/main/java/com/alibaba/fluss/lake/iceberg/tiering/writer/GenericTaskDeltaWriter.java: ########## @@ -0,0 +1,85 @@ +/* + * 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 com.alibaba.fluss.lake.iceberg.tiering.writer; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +import java.io.IOException; + +/** A generic task equality delta writer. * */ +class GenericTaskDeltaWriter extends BaseTaskWriter<Record> { + private final GenericEqualityDeltaWriter deltaWriter; + + public GenericTaskDeltaWriter( + Schema schema, + Schema deleteSchema, + PartitionSpec spec, + FileFormat format, + FileAppenderFactory<Record> appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.deltaWriter = new GenericEqualityDeltaWriter(null, schema, deleteSchema); + } + + @Override + public void write(Record row) throws IOException { + deltaWriter.write(row); + } + + public void delete(Record row) throws IOException { + deltaWriter.delete(row); + } + + // The caller of this function is responsible for passing in a record with only the key fields + public void deleteKey(Record key) throws IOException { + deltaWriter.deleteKey(key); + } + + @Override + public void close() throws IOException { + deltaWriter.close(); + } + + private class GenericEqualityDeltaWriter extends BaseEqualityDeltaWriter { + private GenericEqualityDeltaWriter( + PartitionKey partition, Schema schema, Schema eqDeleteSchema) { + super(partition, schema, eqDeleteSchema); + } + + @Override + protected StructLike asStructLike(Record row) { Review Comment: nit: ```suggestion protected StructLike asStructLike(Record record) { ``` -- 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]
