Skip to content

Commit

Permalink
Use try-with-resources in TestParallelIterable (#11810)
Browse files Browse the repository at this point in the history
  • Loading branch information
sopel39 authored Dec 18, 2024
1 parent b428fbc commit 204a49c
Showing 1 changed file with 153 additions and 139 deletions.
292 changes: 153 additions & 139 deletions core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java
Original file line number Diff line number Diff line change
Expand Up @@ -44,165 +44,179 @@ public class TestParallelIterable {
@Test
public void closeParallelIteratorWithoutCompleteIteration() {
ExecutorService executor = Executors.newFixedThreadPool(1);

Iterable<CloseableIterable<Integer>> transform =
Iterables.transform(
Lists.newArrayList(1, 2, 3, 4, 5),
item ->
new CloseableIterable<Integer>() {
@Override
public void close() {}

@Override
public CloseableIterator<Integer> iterator() {
return CloseableIterator.withClose(Collections.singletonList(item).iterator());
}
});

ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(transform, executor);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

assertThat(iterator.hasNext()).isTrue();
assertThat(iterator.next()).isNotNull();
Awaitility.await("Queue is populated")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> queueHasElements(iterator));
iterator.close();
Awaitility.await("Queue is cleared")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> assertThat(iterator.queueSize()).isEqualTo(0));
try {
Iterable<CloseableIterable<Integer>> transform =
Iterables.transform(
Lists.newArrayList(1, 2, 3, 4, 5),
item ->
new CloseableIterable<Integer>() {
@Override
public void close() {}

@Override
public CloseableIterator<Integer> iterator() {
return CloseableIterator.withClose(
Collections.singletonList(item).iterator());
}
});

ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(transform, executor);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

assertThat(iterator.hasNext()).isTrue();
assertThat(iterator.next()).isNotNull();
Awaitility.await("Queue is populated")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> queueHasElements(iterator));
iterator.close();
Awaitility.await("Queue is cleared")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> assertThat(iterator.queueSize()).isEqualTo(0));
} finally {
executor.shutdown();
}
}

@Test
public void closeMoreDataParallelIteratorWithoutCompleteIteration() {
ExecutorService executor = Executors.newFixedThreadPool(1);
Iterator<Integer> integerIterator =
new Iterator<Integer>() {
private int number = 1;

@Override
public boolean hasNext() {
if (number > 1000) {
return false;
try {
Iterator<Integer> integerIterator =
new Iterator<Integer>() {
private int number = 1;

@Override
public boolean hasNext() {
if (number > 1000) {
return false;
}

number++;
return true;
}

number++;
return true;
}

@Override
public Integer next() {
try {
// sleep to control number generate rate
Thread.sleep(10);
} catch (InterruptedException e) {
// Sleep interrupted, we ignore it!
@Override
public Integer next() {
try {
// sleep to control number generate rate
Thread.sleep(10);
} catch (InterruptedException e) {
// Sleep interrupted, we ignore it!
}
return number;
}
return number;
}
};
Iterable<CloseableIterable<Integer>> transform =
Iterables.transform(
Lists.newArrayList(1),
item ->
new CloseableIterable<Integer>() {
@Override
public void close() {}

@Override
public CloseableIterator<Integer> iterator() {
return CloseableIterator.withClose(integerIterator);
}
});

ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(transform, executor);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

assertThat(iterator.hasNext()).isTrue();
assertThat(iterator.next()).isNotNull();
Awaitility.await("Queue is populated")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> queueHasElements(iterator));
iterator.close();
Awaitility.await("Queue is cleared")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(
() ->
assertThat(iterator.queueSize())
.as("Queue is not empty after cleaning")
.isEqualTo(0));
};
Iterable<CloseableIterable<Integer>> transform =
Iterables.transform(
Lists.newArrayList(1),
item ->
new CloseableIterable<Integer>() {
@Override
public void close() {}

@Override
public CloseableIterator<Integer> iterator() {
return CloseableIterator.withClose(integerIterator);
}
});

ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(transform, executor);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

assertThat(iterator.hasNext()).isTrue();
assertThat(iterator.next()).isNotNull();
Awaitility.await("Queue is populated")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> queueHasElements(iterator));
iterator.close();
Awaitility.await("Queue is cleared")
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(
() ->
assertThat(iterator.queueSize())
.as("Queue is not empty after cleaning")
.isEqualTo(0));
} finally {
executor.shutdown();
}
}

@Test
public void limitQueueSize() {
List<Iterable<Integer>> iterables =
ImmutableList.of(
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator());

Multiset<Integer> expectedValues =
IntStream.range(0, 100)
.boxed()
.flatMap(i -> Stream.of(i, i, i))
.collect(ImmutableMultiset.toImmutableMultiset());

int maxQueueSize = 20;
ExecutorService executor = Executors.newCachedThreadPool();
ParallelIterable<Integer> parallelIterable =
new ParallelIterable<>(iterables, executor, maxQueueSize);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

Multiset<Integer> actualValues = HashMultiset.create();

while (iterator.hasNext()) {
assertThat(iterator.queueSize())
.as("iterator internal queue size")
.isLessThanOrEqualTo(maxQueueSize + iterables.size());
actualValues.add(iterator.next());
try {
List<Iterable<Integer>> iterables =
ImmutableList.of(
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator());

Multiset<Integer> expectedValues =
IntStream.range(0, 100)
.boxed()
.flatMap(i -> Stream.of(i, i, i))
.collect(ImmutableMultiset.toImmutableMultiset());

int maxQueueSize = 20;
ParallelIterable<Integer> parallelIterable =
new ParallelIterable<>(iterables, executor, maxQueueSize);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

Multiset<Integer> actualValues = HashMultiset.create();

while (iterator.hasNext()) {
assertThat(iterator.queueSize())
.as("iterator internal queue size")
.isLessThanOrEqualTo(maxQueueSize + iterables.size());
actualValues.add(iterator.next());
}

assertThat(actualValues)
.as("multiset of values returned by the iterator")
.isEqualTo(expectedValues);

iterator.close();
} finally {
executor.shutdown();
}

assertThat(actualValues)
.as("multiset of values returned by the iterator")
.isEqualTo(expectedValues);

iterator.close();
executor.shutdownNow();
}

@Test
public void queueSizeOne() {
List<Iterable<Integer>> iterables =
ImmutableList.of(
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator());

Multiset<Integer> expectedValues =
IntStream.range(0, 100)
.boxed()
.flatMap(i -> Stream.of(i, i, i))
.collect(ImmutableMultiset.toImmutableMultiset());

ExecutorService executor = Executors.newCachedThreadPool();
ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(iterables, executor, 1);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

Multiset<Integer> actualValues = HashMultiset.create();

while (iterator.hasNext()) {
assertThat(iterator.queueSize())
.as("iterator internal queue size")
.isLessThanOrEqualTo(1 + iterables.size());
actualValues.add(iterator.next());
try {
List<Iterable<Integer>> iterables =
ImmutableList.of(
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator(),
() -> IntStream.range(0, 100).iterator());

Multiset<Integer> expectedValues =
IntStream.range(0, 100)
.boxed()
.flatMap(i -> Stream.of(i, i, i))
.collect(ImmutableMultiset.toImmutableMultiset());

ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(iterables, executor, 1);
ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator();

Multiset<Integer> actualValues = HashMultiset.create();

while (iterator.hasNext()) {
assertThat(iterator.queueSize())
.as("iterator internal queue size")
.isLessThanOrEqualTo(1 + iterables.size());
actualValues.add(iterator.next());
}

assertThat(actualValues)
.as("multiset of values returned by the iterator")
.isEqualTo(expectedValues);

iterator.close();
} finally {
executor.shutdown();
}

assertThat(actualValues)
.as("multiset of values returned by the iterator")
.isEqualTo(expectedValues);

iterator.close();
executor.shutdownNow();
}

private void queueHasElements(ParallelIterator<Integer> iterator) {
Expand Down

0 comments on commit 204a49c

Please sign in to comment.