Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Core: Fix leak of DeleteFile streams #8132

Merged
merged 3 commits into from Aug 7, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
64 changes: 52 additions & 12 deletions core/src/main/java/org/apache/iceberg/deletes/Deletes.java
Expand Up @@ -39,8 +39,13 @@
import org.apache.iceberg.util.Filter;
import org.apache.iceberg.util.SortedMerge;
import org.apache.iceberg.util.StructLikeSet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class Deletes {

private static final Logger LOG = LoggerFactory.getLogger(Deletes.class);

private static final Schema POSITION_DELETE_SCHEMA =
new Schema(MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS);

Expand Down Expand Up @@ -219,7 +224,7 @@ public CloseableIterator<T> iterator() {
CloseableIterator<T> iter;
if (deletePosIterator.hasNext()) {
nextDeletePos = deletePosIterator.next();
iter = applyDelete(rows.iterator());
iter = applyDelete(rows.iterator(), deletePosIterator);
} else {
iter = rows.iterator();
}
Expand Down Expand Up @@ -249,7 +254,8 @@ boolean isDeleted(T row) {
return isDeleted;
}

protected abstract CloseableIterator<T> applyDelete(CloseableIterator<T> items);
protected abstract CloseableIterator<T> applyDelete(
CloseableIterator<T> items, CloseableIterator<Long> deletePositions);
}

private static class PositionStreamDeleteFilter<T> extends PositionStreamDeleteIterable<T> {
Expand All @@ -265,7 +271,8 @@ private static class PositionStreamDeleteFilter<T> extends PositionStreamDeleteI
}

@Override
protected CloseableIterator<T> applyDelete(CloseableIterator<T> items) {
protected CloseableIterator<T> applyDelete(
CloseableIterator<T> items, CloseableIterator<Long> deletePositions) {
return new FilterIterator<T>(items) {
@Override
protected boolean shouldKeep(T item) {
Expand All @@ -276,6 +283,16 @@ protected boolean shouldKeep(T item) {

return !deleted;
}

@Override
public void close() {
try {
deletePositions.close();
} catch (IOException e) {
LOG.warn("Error closing delete file", e);
}
super.close();
}
};
}
}
Expand All @@ -293,15 +310,38 @@ private static class PositionStreamDeleteMarker<T> extends PositionStreamDeleteI
}

@Override
protected CloseableIterator<T> applyDelete(CloseableIterator<T> items) {
return CloseableIterator.transform(
items,
row -> {
if (isDeleted(row)) {
markDeleted.accept(row);
}
return row;
});
protected CloseableIterator<T> applyDelete(
CloseableIterator<T> items, CloseableIterator<Long> deletePositions) {

return new CloseableIterator<T>() {
@Override
public void close() {
try {
deletePositions.close();
} catch (IOException e) {
LOG.warn("Error closing delete file", e);
}
try {
items.close();
} catch (IOException e) {
LOG.warn("Error closing data file", e);
}
}

@Override
public boolean hasNext() {
return items.hasNext();
}

@Override
public T next() {
T row = items.next();
if (isDeleted(row)) {
markDeleted.accept(row);
}
return row;
}
};
}
}

Expand Down
12 changes: 11 additions & 1 deletion data/src/main/java/org/apache/iceberg/data/DeleteFilter.java
Expand Up @@ -82,7 +82,17 @@ protected DeleteFilter(
Schema tableSchema,
Schema requestedSchema,
DeleteCounter counter) {
this.setFilterThreshold = DEFAULT_SET_FILTER_THRESHOLD;
this(filePath, deletes, tableSchema, requestedSchema, counter, DEFAULT_SET_FILTER_THRESHOLD);
}

protected DeleteFilter(
String filePath,
List<DeleteFile> deletes,
Schema tableSchema,
Schema requestedSchema,
DeleteCounter counter,
long setFilterThreshold) {
this.setFilterThreshold = setFilterThreshold;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor: is this change needed? I assume this change will make test easier by setting a customized threshold. However, there is no such test case yet.

this.filePath = filePath;
this.counter = counter;

Expand Down
223 changes: 223 additions & 0 deletions data/src/test/java/org/apache/iceberg/data/TestDeleteFilter.java
@@ -0,0 +1,223 @@
/*
* 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.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.Files;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.Schema;
import org.apache.iceberg.TableTestBase;
import org.apache.iceberg.TestHelpers;
import org.apache.iceberg.deletes.DeleteCounter;
import org.apache.iceberg.deletes.Deletes;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.util.CharSequenceSet;
import org.apache.iceberg.util.Pair;
import org.junit.Assert;
import org.junit.Test;

public class TestDeleteFilter extends TableTestBase {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can put these tests into class TestPositionFilter. What we do here is to check if both data file iterator and delete file iterator are closed, would it be easier to modify test case like TestPositionFilter::testPositionStreamRowFilter for that purpose?


public TestDeleteFilter() {
super(2);
}

@Test
public void testClosePositionStreamRowDeleteMarker() throws Exception {
// Add a data file
DataFile dataFile =
DataFiles.builder(SPEC)
.withPath("/path/to/data-a.parquet")
.withFileSizeInBytes(10)
.withPartitionPath("data_bucket=0")
.withRecordCount(1)
.build();
table.newFastAppend().appendFile(dataFile).commit();

// Add a delete file
List<Pair<CharSequence, Long>> deletes = Lists.newArrayList();
deletes.add(Pair.of(dataFile.path(), 1L));
deletes.add(Pair.of(dataFile.path(), 2L));

Pair<DeleteFile, CharSequenceSet> posDeletes =
FileHelpers.writeDeleteFile(
table, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes);
table
.newRowDelta()
.addDeletes(posDeletes.first())
.validateDataFilesExist(posDeletes.second())
.commit();

// mock records
List<Record> records = Lists.newArrayList();
GenericRecord record =
GenericRecord.create(
TypeUtil.join(table.schema(), new Schema(MetadataColumns.ROW_POSITION)));
records.add(record.copy("id", 29, "data", "a", "_pos", 1L));
records.add(record.copy("id", 43, "data", "b", "_pos", 2L));
records.add(record.copy("id", 61, "data", "c", "_pos", 3L));
records.add(record.copy("id", 89, "data", "d", "_pos", 4L));

CheckingClosableIterable<Record> data = new CheckingClosableIterable<>(records);
CheckingClosableIterable<Long> deletePositions =
new CheckingClosableIterable<>(
deletes.stream().map(Pair::second).collect(Collectors.toList()));

CloseableIterable<Record> posDeletesIterable =
Deletes.streamingFilter(data, row -> row.get(2, Long.class), deletePositions);

// end iterator is always wrapped with FilterIterator
CloseableIterable<Record> eqDeletesIterable =
Deletes.filterDeleted(posDeletesIterable, i -> false, new DeleteCounter());
List<Record> result = Lists.newArrayList(eqDeletesIterable.iterator());

// as first two records deleted, expect only last two records
List<Record> expected = Lists.newArrayList();
expected.add(record.copy("id", 61, "data", "c", "_pos", 3L));
expected.add(record.copy("id", 89, "data", "d", "_pos", 4L));

Assert.assertEquals(result, expected);
Assert.assertTrue(data.isClosed());
Assert.assertTrue(deletePositions.isClosed());
}

@Test
public void testDeleteMarkerFileClosed() throws Exception {
// Add a data file
DataFile dataFile =
DataFiles.builder(SPEC)
.withPath("/path/to/data-a.parquet")
.withFileSizeInBytes(10)
.withPartitionPath("data_bucket=0")
.withRecordCount(1)
.build();
table.newFastAppend().appendFile(dataFile).commit();

// Add a delete file
List<Pair<CharSequence, Long>> deletes = Lists.newArrayList();
deletes.add(Pair.of(dataFile.path(), 1L));
deletes.add(Pair.of(dataFile.path(), 2L));

Pair<DeleteFile, CharSequenceSet> posDeletes =
FileHelpers.writeDeleteFile(
table, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes);
table
.newRowDelta()
.addDeletes(posDeletes.first())
.validateDataFilesExist(posDeletes.second())
.commit();

// mock records
List<Record> records = Lists.newArrayList();
GenericRecord template =
GenericRecord.create(
TypeUtil.join(
table.schema(),
new Schema(MetadataColumns.ROW_POSITION, MetadataColumns.IS_DELETED)));
records.add(record(template, 29, "a", 1, false));
records.add(record(template, 43, "b", 2, false));
records.add(record(template, 61, "c", 3, false));
records.add(record(template, 89, "d", 4, false));

CheckingClosableIterable<Record> data = new CheckingClosableIterable<>(records);
CheckingClosableIterable<Long> deletePositions =
new CheckingClosableIterable<>(
deletes.stream().map(Pair::second).collect(Collectors.toList()));

CloseableIterable<Record> resultIterable =
Deletes.streamingMarker(
data, row -> row.get(2, Long.class), deletePositions, row -> row.set(3, true));

// end iterator is always wrapped with FilterIterator
CloseableIterable<Record> eqDeletesIterable =
Deletes.filterDeleted(resultIterable, i -> false, new DeleteCounter());
List<Record> result = Lists.newArrayList(eqDeletesIterable.iterator());

// expect first two records deleted
List<Record> expected = Lists.newArrayList();
expected.add(record(template, 29, "a", 1, true));
expected.add(record(template, 43, "b", 2, true));
expected.add(record(template, 61, "c", 3, false));
expected.add(record(template, 89, "d", 4, false));

Assert.assertEquals(result, expected);
Assert.assertTrue(data.isClosed());
Assert.assertTrue(deletePositions.isClosed());
}

private GenericRecord record(
GenericRecord template, int id, String data, long pos, boolean isDeleted) {
GenericRecord copy = template.copy();
copy.set(0, id);
copy.set(1, data);
copy.set(2, pos);
copy.set(3, isDeleted);
return copy;
}

private static class CheckingClosableIterable<E> implements CloseableIterable<E> {
AtomicBoolean isClosed = new AtomicBoolean(false);
final Iterable<E> iterable;

CheckingClosableIterable(Iterable<E> iterable) {
this.iterable = iterable;
}

public boolean isClosed() {
return isClosed.get();
}

@Override
public void close() throws IOException {
isClosed.set(true);
}

@Override
public CloseableIterator<E> iterator() {
Iterator<E> it = iterable.iterator();
return new CloseableIterator<E>() {

@Override
public boolean hasNext() {
return it.hasNext();
}

@Override
public E next() {
return it.next();
}

@Override
public void close() throws IOException {
isClosed.set(true);
}
};
}
}
}