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 all 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
105 changes: 105 additions & 0 deletions core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java
Expand Up @@ -20,12 +20,16 @@

import static org.assertj.core.api.Assertions.assertThat;

import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Predicate;
import org.apache.avro.util.Utf8;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.TestHelpers.Row;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
Expand Down Expand Up @@ -321,4 +325,105 @@ public void testCombinedPositionSetRowFilter() {
.as("Filter should produce expected rows")
.containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L));
}

@Test
public void testClosePositionStreamRowDeleteMarker() {
List<Long> deletes = Lists.newArrayList(1L, 2L);

List<StructLike> records =
Lists.newArrayList(
Row.of(29, "a", 1L), Row.of(43, "b", 2L), Row.of(61, "c", 3L), Row.of(89, "d", 4L));

CheckingClosableIterable<StructLike> data = new CheckingClosableIterable<>(records);
CheckingClosableIterable<Long> deletePositions = new CheckingClosableIterable<>(deletes);

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

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

// as first two records deleted, expect only last two records
assertThat(Iterables.transform(result, row -> row.get(2, Long.class)))
.as("Filter should produce expected rows")
.containsExactlyElementsOf(Lists.newArrayList(3L, 4L));

assertThat(data.isClosed).isTrue();
assertThat(deletePositions.isClosed).isTrue();
}

@Test
public void testDeleteMarkerFileClosed() {

List<Long> deletes = Lists.newArrayList(1L, 2L);

List<StructLike> records =
Lists.newArrayList(
Row.of(29, "a", 1L, false),
Row.of(43, "b", 2L, false),
Row.of(61, "c", 3L, false),
Row.of(89, "d", 4L, false));

CheckingClosableIterable<StructLike> data = new CheckingClosableIterable<>(records);
CheckingClosableIterable<Long> deletePositions = new CheckingClosableIterable<>(deletes);

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

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

// as first two records deleted, expect only those two records marked
assertThat(Iterables.transform(result, row -> row.get(3, Boolean.class)))
.as("Filter should produce expected rows")
.containsExactlyElementsOf(Lists.newArrayList(true, true, false, false));

assertThat(data.isClosed).isTrue();
assertThat(deletePositions.isClosed).isTrue();
}

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() {
isClosed.set(true);
}
};
}
}
}