Skip to content

Commit daa33aa

Browse files
committed
HADOOP-18577. ABFS: Add probes of readahead fix (#5205)
Followup patch to HADOOP-18456 as part of HADOOP-18521, ABFS ReadBufferManager buffer sharing across concurrent HTTP requests Add probes of readahead fix aid in checking safety of hadoop ABFS client across different releases. * ReadBufferManager constructor logs the fact it is safe at TRACE * AbfsInputStream declares it is fixed in toString() by including fs.azure.capability.readahead.safe" in the result. The ABFS FileSystem hasPathCapability("fs.azure.capability.readahead.safe") probe returns true to indicate the client's readahead manager has been fixed to be safe when prefetching. All Hadoop releases for which probe this returns false and for which the probe "fs.capability.etags.available" returns true at risk of returning invalid data when reading ADLS Gen2/Azure storage data. Contributed by Steve Loughran.
1 parent 65892a7 commit daa33aa

File tree

6 files changed

+115
-5
lines changed

6 files changed

+115
-5
lines changed

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -117,6 +117,7 @@
117117
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR;
118118
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT;
119119
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DATA_BLOCKS_BUFFER_DEFAULT;
120+
import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
120121
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
121122
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
122123
import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
@@ -235,6 +236,7 @@ public String toString() {
235236
sb.append("uri=").append(uri);
236237
sb.append(", user='").append(abfsStore.getUser()).append('\'');
237238
sb.append(", primaryUserGroup='").append(abfsStore.getPrimaryGroup()).append('\'');
239+
sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]");
238240
sb.append('}');
239241
return sb.toString();
240242
}
@@ -1636,6 +1638,11 @@ public boolean hasPathCapability(final Path path, final String capability)
16361638
new TracingContext(clientCorrelationId, fileSystemId,
16371639
FSOperationType.HAS_PATH_CAPABILITY, tracingHeaderFormat,
16381640
listener));
1641+
1642+
// probe for presence of the HADOOP-18546 readahead fix.
1643+
case CAPABILITY_SAFE_READAHEAD:
1644+
return true;
1645+
16391646
default:
16401647
return super.hasPathCapability(p, capability);
16411648
}
Lines changed: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,46 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.fs.azurebfs.constants;
20+
21+
import org.apache.hadoop.classification.InterfaceAudience;
22+
23+
/**
24+
* Constants which are used internally and which don't fit into the other
25+
* classes.
26+
* For use within the {@code hadoop-azure} module only.
27+
*/
28+
@InterfaceAudience.Private
29+
public final class InternalConstants {
30+
31+
private InternalConstants() {
32+
}
33+
34+
/**
35+
* Does this version of the store have safe readahead?
36+
* Possible combinations of this and the probe
37+
* {@code "fs.capability.etags.available"}.
38+
* <ol>
39+
* <li>{@value}: store is safe</li>
40+
* <li>!etags: store is safe</li>
41+
* <li>etags && !{@value}: store is <i>UNSAFE</i></li>
42+
* </ol>
43+
*/
44+
public static final String CAPABILITY_SAFE_READAHEAD =
45+
"fs.azure.capability.readahead.safe";
46+
}

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,7 @@
5050

5151
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB;
5252
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
53+
import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
5354
import static org.apache.hadoop.util.StringUtils.toLowerCase;
5455

5556
/**
@@ -828,11 +829,12 @@ public IOStatistics getIOStatistics() {
828829
@Override
829830
public String toString() {
830831
final StringBuilder sb = new StringBuilder(super.toString());
832+
sb.append("AbfsInputStream@(").append(this.hashCode()).append("){");
833+
sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]");
831834
if (streamStatistics != null) {
832-
sb.append("AbfsInputStream@(").append(this.hashCode()).append("){");
833-
sb.append(streamStatistics.toString());
834-
sb.append("}");
835+
sb.append(", ").append(streamStatistics);
835836
}
837+
sb.append("}");
836838
return sb.toString();
837839
}
838840

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,7 @@ private void init() {
101101

102102
// hide instance constructor
103103
private ReadBufferManager() {
104+
LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch");
104105
}
105106

106107

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

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,22 @@
2020

2121
import java.net.URI;
2222

23+
import org.assertj.core.api.Assertions;
2324
import org.junit.Test;
2425

2526
import org.apache.hadoop.conf.Configuration;
2627
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
2728
import org.apache.hadoop.fs.FileSystem;
29+
import org.apache.hadoop.fs.Path;
2830
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
2931
import org.apache.hadoop.fs.azurebfs.services.AuthType;
3032

33+
import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_AVAILABLE;
34+
import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME;
35+
import static org.apache.hadoop.fs.CommonPathCapabilities.FS_ACLS;
36+
import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
37+
import static org.junit.Assume.assumeTrue;
38+
3139
/**
3240
* Test AzureBlobFileSystem initialization.
3341
*/
@@ -74,4 +82,28 @@ public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
7482
assertNotNull("working directory", fs.getWorkingDirectory());
7583
}
7684
}
85+
86+
@Test
87+
public void testFileSystemCapabilities() throws Throwable {
88+
final AzureBlobFileSystem fs = getFileSystem();
89+
90+
final Path p = new Path("}");
91+
// etags always present
92+
Assertions.assertThat(fs.hasPathCapability(p, ETAGS_AVAILABLE))
93+
.describedAs("path capability %s in %s", ETAGS_AVAILABLE, fs)
94+
.isTrue();
95+
// readahead always correct
96+
Assertions.assertThat(fs.hasPathCapability(p, CAPABILITY_SAFE_READAHEAD))
97+
.describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs)
98+
.isTrue();
99+
100+
// etags-over-rename and ACLs are either both true or both false.
101+
final boolean etagsAcrossRename = fs.hasPathCapability(p, ETAGS_PRESERVED_IN_RENAME);
102+
final boolean acls = fs.hasPathCapability(p, FS_ACLS);
103+
Assertions.assertThat(etagsAcrossRename)
104+
.describedAs("capabilities %s=%s and %s=%s in %s",
105+
ETAGS_PRESERVED_IN_RENAME, etagsAcrossRename,
106+
FS_ACLS, acls, fs)
107+
.isEqualTo(acls);
108+
}
77109
}

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

Lines changed: 24 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -44,9 +44,24 @@
4444
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH;
4545
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
4646
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
47+
import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
48+
import static org.apache.hadoop.test.LambdaTestUtils.eventually;
4749

4850
public class ITestReadBufferManager extends AbstractAbfsIntegrationTest {
4951

52+
/**
53+
* Time before the JUnit test times out for eventually() clauses
54+
* to fail. This copes with slow network connections and debugging
55+
* sessions, yet still allows for tests to fail with meaningful
56+
* messages.
57+
*/
58+
public static final int TIMEOUT_OFFSET = 5 * 60_000;
59+
60+
/**
61+
* Interval between eventually preobes.
62+
*/
63+
public static final int PROBE_INTERVAL_MILLIS = 1_000;
64+
5065
public ITestReadBufferManager() throws Exception {
5166
}
5267

@@ -61,6 +76,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception {
6176
}
6277
ExecutorService executorService = Executors.newFixedThreadPool(4);
6378
AzureBlobFileSystem fs = getABFSWithReadAheadConfig();
79+
// verify that the fs has the capability to validate the fix
80+
Assertions.assertThat(fs.hasPathCapability(new Path("/"), CAPABILITY_SAFE_READAHEAD))
81+
.describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs)
82+
.isTrue();
83+
6484
try {
6585
for (int i = 0; i < 4; i++) {
6686
final String fileName = methodName.getMethodName() + i;
@@ -80,9 +100,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception {
80100
}
81101

82102
ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
83-
// verify there is no work in progress or the readahead queue.
84-
assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList());
103+
// readahead queue is empty
85104
assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy());
105+
// verify the in progress list eventually empties out.
106+
eventually(getTestTimeoutMillis() - TIMEOUT_OFFSET, PROBE_INTERVAL_MILLIS, () ->
107+
assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()));
86108
}
87109

88110
private void assertListEmpty(String listName, List<ReadBuffer> list) {

0 commit comments

Comments
 (0)