[
https://issues.apache.org/jira/browse/HADOOP-18546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17646455#comment-17646455
]
ASF GitHub Bot commented on HADOOP-18546:
-----------------------------------------
mehakmeet commented on code in PR #5205:
URL: https://github.com/apache/hadoop/pull/5205#discussion_r1046686949
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java:
##########
@@ -828,8 +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]")
+ .append(", ");
if (streamStatistics != null) {
- sb.append("AbfsInputStream@(").append(this.hashCode()).append("){");
sb.append(streamStatistics.toString());
sb.append("}");
Review Comment:
The closing bracket of the log should be outside the statistics if block
##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java:
##########
@@ -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
+ * 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.
Review Comment:
typo: "probes"
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java:
##########
@@ -1636,6 +1636,11 @@ public boolean hasPathCapability(final Path path, final
String capability)
new TracingContext(clientCorrelationId, fileSystemId,
FSOperationType.HAS_PATH_CAPABILITY, tracingHeaderFormat,
listener));
+
+ // probe for presence of HADOOP-18546 fix.
+ case "hadoop-18546":
Review Comment:
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?
> disable purging list of in progress reads in abfs stream closed
> ---------------------------------------------------------------
>
> Key: HADOOP-18546
> URL: https://issues.apache.org/jira/browse/HADOOP-18546
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/azure
> Affects Versions: 3.3.4
> Reporter: Steve Loughran
> Assignee: Pranav Saxena
> Priority: Major
> Labels: pull-request-available
> Fix For: 3.4.0, 3.3.5
>
>
> turn off the prune of in progress reads in
> ReadBufferManager::purgeBuffersForStream
> this will ensure active prefetches for a closed stream complete. they wiill
> then get to the completed list and hang around until evicted by timeout, but
> at least prefetching will be safe.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]