-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-18577. log/probes of HADOOP-18546 presence. #5205
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
Changes from 3 commits
b86c275
d019c4b
67267cb
06c0fed
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -828,9 +828,10 @@ public IOStatistics getIOStatistics() { | |
| @Override | ||
| public String toString() { | ||
| final StringBuilder sb = new StringBuilder(super.toString()); | ||
| sb.append("AbfsInputStream@(").append(this.hashCode()).append("){"); | ||
| sb.append("[HADOOP-18546]"); | ||
| if (streamStatistics != null) { | ||
| sb.append("AbfsInputStream@(").append(this.hashCode()).append("){"); | ||
| sb.append(streamStatistics.toString()); | ||
| sb.append(", ").append(streamStatistics); | ||
| sb.append("}"); | ||
|
||
| } | ||
| return sb.toString(); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -44,9 +44,23 @@ | |
| import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH; | ||
| import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; | ||
| import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; | ||
| import static org.apache.hadoop.test.LambdaTestUtils.eventually; | ||
|
|
||
| public class ITestReadBufferManager extends AbstractAbfsIntegrationTest { | ||
|
|
||
| /** | ||
| * Time before the JUnit test times out for eventually() clauses | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. typo: causes.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. no. eventually takes a lambda expression, which is what i was referring to |
||
| * to fail. This copes with slow network connections and debugging | ||
| * sessions, yet still allows for tests to fail with meaningful | ||
| * messages. | ||
| */ | ||
| public static final int TIMEOUT_OFFSET = 5 * 60_000; | ||
|
|
||
| /** | ||
| * Interval between eventually preobes. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. typo: "probes" |
||
| */ | ||
| public static final int PROBE_INTERVAL_MILLIS = 1_000; | ||
|
|
||
| public ITestReadBufferManager() throws Exception { | ||
| } | ||
|
|
||
|
|
@@ -61,6 +75,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { | |
| } | ||
| ExecutorService executorService = Executors.newFixedThreadPool(4); | ||
| AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); | ||
| // verify that the fs has the capability to validate the fix | ||
| Assertions.assertThat(fs.hasPathCapability(new Path("/"), "HADOOP-18546")) | ||
| .describedAs("path capability \"HADOOP-18546\" in %s", fs) | ||
| .isTrue(); | ||
|
|
||
| try { | ||
| for (int i = 0; i < 4; i++) { | ||
| final String fileName = methodName.getMethodName() + i; | ||
|
|
@@ -80,9 +99,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { | |
| } | ||
|
|
||
| ReadBufferManager bufferManager = ReadBufferManager.getBufferManager(); | ||
| // verify there is no work in progress or the readahead queue. | ||
| assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()); | ||
| // readahead queue is empty | ||
| assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); | ||
| // verify the in progress list eventually empties out. | ||
| eventually(getTestTimeoutMillis() - TIMEOUT_OFFSET, PROBE_INTERVAL_MILLIS, () -> | ||
| assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList())); | ||
| } | ||
|
|
||
| private void assertListEmpty(String listName, List<ReadBuffer> list) { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Naming the probe on a Hadoop Jira makes it a little difficult to understand it from the code directly. Should we have a general name for the probe related to the prefetch inconsistent reads and have the Hadoop jira mentioned in the comments only?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
i was trying to think of one but it is a "is this fix in". that way, if we need to add another we can ask for that as well
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
but having a probe for the fix "fs.azure.readahead.safe" would let us move to a different fix while retaining the same probe...