Skip to content

Commit 0dac3d2

Browse files
authored
HADOOP-19531. [ABFS][FnsOverBlob] Streaming List Path Result Should Happen Inside Retry Loop (#7582)
Contributed by Anuj Modi Reviewed by Anmol Asrani, Manish Bhatt, Manika Joshi Signed off by: Anuj Modi<[email protected]>
1 parent 3d2f4d6 commit 0dac3d2

File tree

7 files changed

+129
-49
lines changed

7 files changed

+129
-49
lines changed

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

Lines changed: 21 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@
3838
import org.apache.http.Header;
3939
import org.apache.http.HttpEntity;
4040
import org.apache.http.HttpResponse;
41+
import org.apache.http.client.methods.CloseableHttpResponse;
4142
import org.apache.http.client.methods.HttpDelete;
4243
import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
4344
import org.apache.http.client.methods.HttpGet;
@@ -47,6 +48,7 @@
4748
import org.apache.http.client.methods.HttpPut;
4849
import org.apache.http.client.methods.HttpRequestBase;
4950
import org.apache.http.entity.ByteArrayEntity;
51+
import org.apache.http.util.EntityUtils;
5052

5153
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APACHE_IMPL;
5254
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
@@ -192,14 +194,26 @@ String getConnResponseMessage() throws IOException {
192194
public void processResponse(final byte[] buffer,
193195
final int offset,
194196
final int length) throws IOException {
195-
if (!isPayloadRequest) {
196-
prepareRequest();
197-
LOG.debug("Sending request: {}", httpRequestBase);
198-
httpResponse = executeRequest();
199-
LOG.debug("Request sent: {}; response {}", httpRequestBase,
200-
httpResponse);
197+
try {
198+
if (!isPayloadRequest) {
199+
prepareRequest();
200+
LOG.debug("Sending request: {}", httpRequestBase);
201+
httpResponse = executeRequest();
202+
LOG.debug("Request sent: {}; response {}", httpRequestBase,
203+
httpResponse);
204+
}
205+
parseResponseHeaderAndBody(buffer, offset, length);
206+
} finally {
207+
if (httpResponse != null) {
208+
try {
209+
EntityUtils.consume(httpResponse.getEntity());
210+
} finally {
211+
if (httpResponse instanceof CloseableHttpResponse) {
212+
((CloseableHttpResponse) httpResponse).close();
213+
}
214+
}
215+
}
201216
}
202-
parseResponseHeaderAndBody(buffer, offset, length);
203217
}
204218

205219
/**

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

Lines changed: 14 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -1605,12 +1605,9 @@ public Hashtable<String, String> getXMSProperties(AbfsHttpOperation result)
16051605
@Override
16061606
public ListResponseData parseListPathResults(AbfsHttpOperation result, URI uri)
16071607
throws AzureBlobFileSystemException {
1608-
BlobListResultSchema listResultSchema;
16091608
try (InputStream stream = result.getListResultStream()) {
1610-
if (stream == null) {
1611-
return null;
1612-
}
16131609
try {
1610+
BlobListResultSchema listResultSchema;
16141611
final SAXParser saxParser = saxParserThreadLocal.get();
16151612
saxParser.reset();
16161613
listResultSchema = new BlobListResultSchema();
@@ -1620,19 +1617,17 @@ public ListResponseData parseListPathResults(AbfsHttpOperation result, URI uri)
16201617
LOG.debug("ListBlobs listed {} blobs with {} as continuation token",
16211618
listResultSchema.paths().size(),
16221619
listResultSchema.getNextMarker());
1623-
} catch (SAXException | IOException e) {
1624-
throw new AbfsDriverException(e);
1620+
return filterDuplicateEntriesAndRenamePendingFiles(listResultSchema, uri);
1621+
} catch (SAXException | IOException ex) {
1622+
throw new AbfsDriverException(ERR_BLOB_LIST_PARSING, ex);
16251623
}
1626-
} catch (IOException e) {
1627-
LOG.error("Unable to deserialize list results for uri {}", uri.toString(), e);
1628-
throw new AbfsDriverException(ERR_BLOB_LIST_PARSING, e);
1629-
}
1630-
1631-
try {
1632-
return filterDuplicateEntriesAndRenamePendingFiles(listResultSchema, uri);
1633-
} catch (IOException e) {
1634-
LOG.error("Unable to filter list results for uri {}", uri.toString(), e);
1635-
throw new AbfsDriverException(ERR_BLOB_LIST_PARSING, e);
1624+
} catch (AbfsDriverException ex) {
1625+
// Throw as it is to avoid multiple wrapping.
1626+
LOG.error("Unable to deserialize list results for Uri {}", uri != null ? uri.toString(): "NULL", ex);
1627+
throw ex;
1628+
} catch (Exception ex) {
1629+
LOG.error("Unable to get stream for list results for uri {}", uri != null ? uri.toString(): "NULL", ex);
1630+
throw new AbfsDriverException(ERR_BLOB_LIST_PARSING, ex);
16361631
}
16371632
}
16381633

@@ -1929,8 +1924,10 @@ private List<AbfsHttpHeader> getMetadataHeadersList(final Hashtable<String, Stri
19291924
* @param listResultSchema List of entries returned by Blob Endpoint.
19301925
* @param uri URI to be used for path conversion.
19311926
* @return List of entries after removing duplicates.
1927+
* @throws IOException if path conversion fails.
19321928
*/
1933-
private ListResponseData filterDuplicateEntriesAndRenamePendingFiles(
1929+
@VisibleForTesting
1930+
public ListResponseData filterDuplicateEntriesAndRenamePendingFiles(
19341931
BlobListResultSchema listResultSchema, URI uri) throws IOException {
19351932
List<FileStatus> fileStatuses = new ArrayList<>();
19361933
Map<Path, Integer> renamePendingJsonPaths = new HashMap<>();

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

Lines changed: 25 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -38,8 +38,10 @@
3838
import java.util.UUID;
3939

4040
import com.fasterxml.jackson.core.JsonFactory;
41+
import com.fasterxml.jackson.core.JsonParseException;
4142
import com.fasterxml.jackson.core.JsonParser;
4243
import com.fasterxml.jackson.core.JsonToken;
44+
import com.fasterxml.jackson.databind.JsonMappingException;
4345
import com.fasterxml.jackson.databind.ObjectMapper;
4446

4547
import org.apache.hadoop.classification.VisibleForTesting;
@@ -1476,33 +1478,36 @@ public Hashtable<String, String> getXMSProperties(AbfsHttpOperation result)
14761478
* @throws AzureBlobFileSystemException if parsing fails.
14771479
*/
14781480
@Override
1479-
public ListResponseData parseListPathResults(AbfsHttpOperation result, URI uri) throws AzureBlobFileSystemException {
1480-
try (InputStream listResultInputStream = result.getListResultStream()) {
1481-
DfsListResultSchema listResultSchema;
1481+
public ListResponseData parseListPathResults(AbfsHttpOperation result, URI uri)
1482+
throws AzureBlobFileSystemException {
1483+
try (InputStream stream = result.getListResultStream()) {
14821484
try {
1485+
DfsListResultSchema listResultSchema;
14831486
final ObjectMapper objectMapper = new ObjectMapper();
1484-
listResultSchema = objectMapper.readValue(listResultInputStream,
1485-
DfsListResultSchema.class);
1487+
listResultSchema = objectMapper.readValue(stream, DfsListResultSchema.class);
14861488
result.setListResultSchema(listResultSchema);
14871489
LOG.debug("ListPath listed {} paths with {} as continuation token",
14881490
listResultSchema.paths().size(),
14891491
getContinuationFromResponse(result));
1490-
} catch (IOException ex) {
1491-
throw new AbfsDriverException(ex);
1492-
}
1493-
1494-
List<FileStatus> fileStatuses = new ArrayList<>();
1495-
for (DfsListResultEntrySchema entry : listResultSchema.paths()) {
1496-
fileStatuses.add(getVersionedFileStatusFromEntry(entry, uri));
1492+
List<FileStatus> fileStatuses = new ArrayList<>();
1493+
for (DfsListResultEntrySchema entry : listResultSchema.paths()) {
1494+
fileStatuses.add(getVersionedFileStatusFromEntry(entry, uri));
1495+
}
1496+
ListResponseData listResponseData = new ListResponseData();
1497+
listResponseData.setFileStatusList(fileStatuses);
1498+
listResponseData.setRenamePendingJsonPaths(null);
1499+
listResponseData.setContinuationToken(
1500+
getContinuationFromResponse(result));
1501+
return listResponseData;
1502+
} catch (JsonParseException | JsonMappingException ex) {
1503+
throw new AbfsDriverException(ERR_DFS_LIST_PARSING, ex);
14971504
}
1498-
ListResponseData listResponseData = new ListResponseData();
1499-
listResponseData.setFileStatusList(fileStatuses);
1500-
listResponseData.setRenamePendingJsonPaths(null);
1501-
listResponseData.setContinuationToken(
1502-
getContinuationFromResponse(result));
1503-
return listResponseData;
1504-
} catch (IOException ex) {
1505-
LOG.error("Unable to deserialize list results for Uri {}", uri.toString(), ex);
1505+
} catch (AbfsDriverException ex) {
1506+
// Throw as it is to avoid multiple wrapping.
1507+
LOG.error("Unable to deserialize list results for Uri {}", uri != null ? uri.toString(): "NULL", ex);
1508+
throw ex;
1509+
} catch (Exception ex) {
1510+
LOG.error("Unable to deserialize list results for Uri {}", uri != null ? uri.toString(): "NULL", ex);
15061511
throw new AbfsDriverException(ERR_DFS_LIST_PARSING, ex);
15071512
}
15081513
}

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

Lines changed: 29 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818

1919
package org.apache.hadoop.fs.azurebfs.services;
2020

21+
import java.io.ByteArrayInputStream;
22+
import java.io.ByteArrayOutputStream;
2123
import java.io.IOException;
2224
import java.io.InputStream;
2325
import java.net.HttpURLConnection;
@@ -221,7 +223,7 @@ public ListResultSchema getListResultSchema() {
221223
return listResultSchema;
222224
}
223225

224-
public final InputStream getListResultStream() {
226+
public InputStream getListResultStream() {
225227
return listResultStream;
226228
}
227229

@@ -396,8 +398,7 @@ final void parseResponse(final byte[] buffer,
396398
// consume the input stream to release resources
397399
int totalBytesRead = 0;
398400

399-
try {
400-
InputStream stream = getContentInputStream();
401+
try (InputStream stream = getContentInputStream()) {
401402
if (isNullInputStream(stream)) {
402403
return;
403404
}
@@ -409,7 +410,7 @@ final void parseResponse(final byte[] buffer,
409410
if (url.toString().contains(QUERY_PARAM_COMP + EQUAL + BLOCKLIST)) {
410411
parseBlockListResponse(stream);
411412
} else {
412-
listResultStream = stream;
413+
parseListPathResponse(stream);
413414
}
414415
} else {
415416
if (buffer != null) {
@@ -438,6 +439,11 @@ final void parseResponse(final byte[] buffer,
438439
method, getMaskedUrl(), ex.getMessage());
439440
log.debug("IO Error: ", ex);
440441
throw ex;
442+
} catch (Exception ex) {
443+
log.warn("Unexpected error: {} {}: {}",
444+
method, getMaskedUrl(), ex.getMessage());
445+
log.debug("Unexpected Error: ", ex);
446+
throw new IOException(ex);
441447
} finally {
442448
this.recvResponseTimeMs += elapsedTimeMs(startTime);
443449
this.bytesReceived = totalBytesRead;
@@ -500,6 +506,25 @@ private void parseBlockListResponse(final InputStream stream) throws IOException
500506
blockIdList = client.parseBlockListResponse(stream);
501507
}
502508

509+
/**
510+
* Parse the list path response from the network stream and save response into a buffer.
511+
* @param stream Network InputStream.
512+
* @throws IOException if an error occurs while reading the stream.
513+
*/
514+
private void parseListPathResponse(final InputStream stream) throws IOException {
515+
if (stream == null || listResultStream != null) {
516+
return;
517+
}
518+
try (ByteArrayOutputStream buffer = new ByteArrayOutputStream()) {
519+
byte[] tempBuffer = new byte[CLEAN_UP_BUFFER_SIZE];
520+
int bytesRead;
521+
while ((bytesRead = stream.read(tempBuffer, 0, CLEAN_UP_BUFFER_SIZE)) != -1) {
522+
buffer.write(tempBuffer, 0, bytesRead);
523+
}
524+
listResultStream = new ByteArrayInputStream(buffer.toByteArray());
525+
}
526+
}
527+
503528
public List<String> getBlockIdList() {
504529
return blockIdList;
505530
}

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

Lines changed: 37 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020

2121
import java.io.FileNotFoundException;
2222
import java.io.IOException;
23+
import java.net.SocketException;
2324
import java.net.SocketTimeoutException;
2425
import java.net.URL;
2526
import java.util.ArrayList;
@@ -42,6 +43,7 @@
4243
import org.apache.hadoop.fs.Path;
4344
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
4445
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
46+
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
4547
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
4648
import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient;
4749
import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler;
@@ -63,7 +65,10 @@
6365
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE;
6466
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS;
6567
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_METADATA_PREFIX;
68+
import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_BLOB_LIST_PARSING;
6669
import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX;
70+
import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE;
71+
import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION;
6772
import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE;
6873
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
6974
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
@@ -130,7 +135,9 @@ public Void call() throws Exception {
130135

131136
/**
132137
* Test to verify that each paginated call to ListBlobs uses a new tracing context.
133-
* @throws Exception
138+
* Test also verifies that the retry policy is called when a SocketTimeoutException
139+
* Test also verifies that empty list with valid continuation token is handled.
140+
* @throws Exception if there is an error or test assertions fails.
134141
*/
135142
@Test
136143
public void testListPathTracingContext() throws Exception {
@@ -160,6 +167,10 @@ public void testListPathTracingContext() throws Exception {
160167
List<FileStatus> fileStatuses = new ArrayList<>();
161168
spiedStore.listStatus(new Path("/"), "", fileStatuses, true, null, spiedTracingContext);
162169

170+
// Assert that there were retries due to SocketTimeoutException
171+
Mockito.verify(spiedClient, Mockito.times(1))
172+
.getRetryPolicy(CONNECTION_TIMEOUT_ABBREVIATION);
173+
163174
// Assert that there were 2 paginated ListPath calls were made 1 and 2.
164175
// 1. Without continuation token
165176
Mockito.verify(spiedClient, times(1)).listPath(
@@ -176,6 +187,31 @@ public void testListPathTracingContext() throws Exception {
176187
Mockito.verify(spiedTracingContext, times(0)).constructHeader(any(), any(), any());
177188
}
178189

190+
@Test
191+
public void testListPathParsingFailure() throws Exception {
192+
assumeBlobServiceType();
193+
AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem());
194+
AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore());
195+
AbfsBlobClient spiedClient = Mockito.spy(spiedStore.getClientHandler()
196+
.getBlobClient());
197+
Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore();
198+
Mockito.doReturn(spiedClient).when(spiedStore).getClient();
199+
200+
Mockito.doThrow(new SocketException(CONNECTION_RESET_MESSAGE)).when(spiedClient).filterDuplicateEntriesAndRenamePendingFiles(any(), any());
201+
List<FileStatus> fileStatuses = new ArrayList<>();
202+
AbfsDriverException ex = intercept(AbfsDriverException.class,
203+
() -> {
204+
spiedStore.listStatus(new Path("/"), "", fileStatuses,
205+
true, null, getTestTracingContext(spiedFs, true));
206+
});
207+
Assertions.assertThat(ex.getStatusCode())
208+
.describedAs("Expecting Network Error status code")
209+
.isEqualTo(-1);
210+
Assertions.assertThat(ex.getErrorMessage())
211+
.describedAs("Expecting COPY_ABORTED error code")
212+
.contains(ERR_BLOB_LIST_PARSING);
213+
}
214+
179215
/**
180216
* Creates a file, verifies that listStatus returns it,
181217
* even while the file is still open for writing.

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@
4545

4646
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
4747
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
48+
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT;
4849
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
4950
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
5051
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
@@ -67,6 +68,7 @@ public ITestApacheClientConnectionPool() throws Exception {
6768
public void testKacIsClosed() throws Throwable {
6869
Configuration configuration = new Configuration(getRawConfiguration());
6970
configuration.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.name());
71+
configuration.unset(FS_AZURE_METRIC_FORMAT);
7072
try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
7173
configuration)) {
7274
KeepAliveCache kac = fs.getAbfsStore().getClientHandler().getIngressClient()

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ public class TestAbfsClient {
5757
public void testTimerInitializationWithoutMetricCollection() throws Exception {
5858
final Configuration configuration = new Configuration();
5959
AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, ACCOUNT_NAME);
60+
abfsConfiguration.unset(FS_AZURE_METRIC_FORMAT);
6061

6162
AbfsCounters abfsCounters = Mockito.spy(new AbfsCountersImpl(new URI("abcd")));
6263
AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build();

0 commit comments

Comments
 (0)