Skip to content

Commit 4d7360a

Browse files
steveloughranAnuj Modi
authored andcommitted
HADOOP-18546. Followup: ITestReadBufferManager fix (apache#5198)
This is a followup to the original HADOOP-18546 patch; cherry-picks of that should include this or follow up with it. Removes risk of race conditions in assertions of ITestReadBufferManager on the state of the in-progress and completed queues by removing assertions brittle to race conditions in scheduling/network IO * Waits for all the executor pool shutdown to complete before making any assertions * Assertions that there are no in progress reads MUST be cut as there may be some and they won't be cancelled. * Assertions that the completed list is without buffers of a closed stream are brittle because if there was an in progress stream which completed after stream.close() then it will end up in the list. Contributed by Steve Loughran
1 parent f39ec60 commit 4d7360a

File tree

1 file changed

+9
-15
lines changed

1 file changed

+9
-15
lines changed

hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java

Lines changed: 9 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import java.util.concurrent.Callable;
2626
import java.util.concurrent.ExecutorService;
2727
import java.util.concurrent.Executors;
28+
import java.util.concurrent.TimeUnit;
2829

2930
import org.apache.hadoop.conf.Configuration;
3031
import org.apache.hadoop.fs.FSDataInputStream;
@@ -74,17 +75,14 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception {
7475
}
7576
} finally {
7677
executorService.shutdown();
78+
// wait for all tasks to finish
79+
executorService.awaitTermination(1, TimeUnit.MINUTES);
7780
}
7881

7982
ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
80-
assertListEmpty("CompletedList", bufferManager.getCompletedReadListCopy());
83+
// verify there is no work in progress or the readahead queue.
8184
assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList());
8285
assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy());
83-
Assertions.assertThat(bufferManager.getFreeListCopy())
84-
.describedAs("After closing all streams free list contents should match with " + freeList)
85-
.hasSize(numBuffers)
86-
.containsExactlyInAnyOrderElementsOf(freeList);
87-
8886
}
8987

9088
private void assertListEmpty(String listName, List<ReadBuffer> list) {
@@ -116,22 +114,18 @@ public void testPurgeBufferManagerForSequentialStream() throws Exception {
116114
try {
117115
iStream2 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream();
118116
iStream2.read();
119-
// After closing stream1, none of the buffers associated with stream1 should be present.
120-
assertListDoesnotContainBuffersForIstream(bufferManager.getInProgressCopiedList(), iStream1);
121-
assertListDoesnotContainBuffersForIstream(bufferManager.getCompletedReadListCopy(), iStream1);
117+
// After closing stream1, no queued buffers of stream1 should be present
118+
// assertions can't be made about the state of the other lists as it is
119+
// too prone to race conditions.
122120
assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream1);
123121
} finally {
124122
// closing the stream later.
125123
IOUtils.closeStream(iStream2);
126124
}
127-
// After closing stream2, none of the buffers associated with stream2 should be present.
128-
assertListDoesnotContainBuffersForIstream(bufferManager.getInProgressCopiedList(), iStream2);
129-
assertListDoesnotContainBuffersForIstream(bufferManager.getCompletedReadListCopy(), iStream2);
125+
// After closing stream2, no queued buffers of stream2 should be present.
130126
assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream2);
131127

132-
// After closing both the streams, all lists should be empty.
133-
assertListEmpty("CompletedList", bufferManager.getCompletedReadListCopy());
134-
assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList());
128+
// After closing both the streams, read queue should be empty.
135129
assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy());
136130

137131
}

0 commit comments

Comments
 (0)