HADOOP-19120. ApacheHttpClient adaptation in ABFS. (#6633)
Apache httpclient 4.5.x is the new default implementation of http connections; this supports a large configurable pool of connections along with the ability to limit their lifespan. The networking library can be chosen using the configuration option fs.azure.networking.library The supported values are - APACHE_HTTP_CLIENT : Use Apache HttpClient [Default] - JDK_HTTP_URL_CONNECTION : Use JDK networking library Important: unless the networking library is switched back to the JDK, the apache httpcore and httpclient must be on the classpath Contributed by Pranav Saxena
This commit is contained in:
parent
e48cd0e987
commit
b60497ff41
@ -0,0 +1,39 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Exception to denote if the underlying stream, cache or other closable resource
|
||||
* is closed.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Unstable
|
||||
public class ClosedIOException extends PathIOException {
|
||||
|
||||
/**
|
||||
* Appends the custom error-message to the default error message.
|
||||
* @param path path that encountered the closed resource.
|
||||
* @param message custom error message.
|
||||
*/
|
||||
public ClosedIOException(String path, String message) {
|
||||
super(path, message);
|
||||
}
|
||||
}
|
@ -23,6 +23,7 @@
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.azurebfs.services.FixedSASTokenProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
@ -390,6 +391,20 @@ public class AbfsConfiguration{
|
||||
FS_AZURE_ENABLE_PAGINATED_DELETE, DefaultValue = DEFAULT_ENABLE_PAGINATED_DELETE)
|
||||
private boolean isPaginatedDeleteEnabled;
|
||||
|
||||
@IntegerConfigurationValidatorAnnotation(ConfigurationKey =
|
||||
FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES, DefaultValue = DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES)
|
||||
private int maxApacheHttpClientIoExceptionsRetries;
|
||||
|
||||
/**
|
||||
* Max idle TTL configuration for connection given in
|
||||
* {@value org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys#FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL}
|
||||
* with default of
|
||||
* {@value org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations#DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME}
|
||||
*/
|
||||
@LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL,
|
||||
DefaultValue = DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME)
|
||||
private long maxApacheHttpClientConnectionIdleTime;
|
||||
|
||||
private String clientProvidedEncryptionKey;
|
||||
private String clientProvidedEncryptionKeySHA;
|
||||
|
||||
@ -491,6 +506,17 @@ public long getLong(String key, long defaultValue) {
|
||||
return rawConfig.getLong(accountConf(key), rawConfig.getLong(key, defaultValue));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the account-specific value if it exists, then looks for an
|
||||
* account-agnostic value, and finally tries the default value.
|
||||
* @param key Account-agnostic configuration key
|
||||
* @param defaultValue Value returned if none is configured
|
||||
* @return value if one exists, else the default value
|
||||
*/
|
||||
public int getInt(String key, int defaultValue) {
|
||||
return rawConfig.getInt(accountConf(key), rawConfig.getInt(key, defaultValue));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the account-specific password in string form if it exists, then
|
||||
* looks for an account-agnostic value.
|
||||
@ -889,6 +915,24 @@ public DelegatingSSLSocketFactory.SSLChannelMode getPreferredSSLFactoryOption()
|
||||
return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Config to select netlib for server communication.
|
||||
*/
|
||||
public HttpOperationType getPreferredHttpOperationType() {
|
||||
return getEnum(FS_AZURE_NETWORKING_LIBRARY, DEFAULT_NETWORKING_LIBRARY);
|
||||
}
|
||||
|
||||
public int getMaxApacheHttpClientIoExceptionsRetries() {
|
||||
return maxApacheHttpClientIoExceptionsRetries;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return {@link #maxApacheHttpClientConnectionIdleTime}.
|
||||
*/
|
||||
public long getMaxApacheHttpClientConnectionIdleTime() {
|
||||
return maxApacheHttpClientConnectionIdleTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Enum config to allow user to pick format of x-ms-client-request-id header
|
||||
* @return tracingContextFormat config if valid, else default ALL_ID_FORMAT
|
||||
|
@ -750,7 +750,8 @@ public synchronized void close() throws IOException {
|
||||
IOSTATISTICS_LOGGING_LEVEL_DEFAULT);
|
||||
logIOStatisticsAtLevel(LOG, iostatisticsLoggingLevel, getIOStatistics());
|
||||
}
|
||||
IOUtils.cleanupWithLogger(LOG, abfsStore, delegationTokenManager);
|
||||
IOUtils.cleanupWithLogger(LOG, abfsStore, delegationTokenManager,
|
||||
getAbfsClient());
|
||||
this.isClosed = true;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Closing Abfs: {}", toString());
|
||||
|
@ -55,6 +55,7 @@
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
|
||||
import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
|
||||
@ -106,7 +107,6 @@
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClientContextBuilder;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClientRenameResult;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsCounters;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamContext;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamStatisticsImpl;
|
||||
@ -694,7 +694,7 @@ public OutputStream createFile(final Path path,
|
||||
populateAbfsOutputStreamContext(
|
||||
isAppendBlob,
|
||||
lease,
|
||||
client,
|
||||
getClient(),
|
||||
statistics,
|
||||
relativePath,
|
||||
0,
|
||||
@ -933,7 +933,7 @@ public AbfsInputStream openFileForRead(Path path,
|
||||
perfInfo.registerSuccess(true);
|
||||
|
||||
// Add statistics for InputStream
|
||||
return new AbfsInputStream(client, statistics, relativePath,
|
||||
return new AbfsInputStream(getClient(), statistics, relativePath,
|
||||
contentLength, populateAbfsInputStreamContext(
|
||||
parameters.map(OpenFileParameters::getOptions),
|
||||
contextEncryptionAdapter),
|
||||
|
@ -199,5 +199,16 @@ public static ApiVersion getCurrentVersion() {
|
||||
+ "non-hierarchical-namespace account:"
|
||||
+ CPK_CONFIG_LIST;
|
||||
|
||||
/**
|
||||
* System property that define maximum number of cached-connection per fileSystem for
|
||||
* ApacheHttpClient. JDK network library uses the same property to define maximum
|
||||
* number of cached-connections at JVM level.
|
||||
*/
|
||||
public static final String HTTP_MAX_CONN_SYS_PROP = "http.maxConnections";
|
||||
public static final String JDK_IMPL = "JDK";
|
||||
public static final String APACHE_IMPL = "Apache";
|
||||
public static final String JDK_FALLBACK = "JDK_fallback";
|
||||
public static final String KEEP_ALIVE_CACHE_CLOSED = "KeepAliveCache is closed";
|
||||
|
||||
private AbfsHttpConstants() {}
|
||||
}
|
||||
|
@ -321,5 +321,17 @@ public static String accountProperty(String property, String account) {
|
||||
* @see FileSystem#openFile(org.apache.hadoop.fs.Path)
|
||||
*/
|
||||
public static final String FS_AZURE_BUFFERED_PREAD_DISABLE = "fs.azure.buffered.pread.disable";
|
||||
/**Defines what network library to use for server IO calls: {@value}*/
|
||||
public static final String FS_AZURE_NETWORKING_LIBRARY = "fs.azure.networking.library";
|
||||
/**
|
||||
* Maximum number of IOExceptions retries for a single server call on ApacheHttpClient.
|
||||
* Breach of this count would turn off future uses of the ApacheHttpClient library
|
||||
* in the JVM lifecycle: {@value}
|
||||
*/
|
||||
public static final String FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES = "fs.azure.apache.http.client.max.io.exception.retries";
|
||||
/**Maximum ApacheHttpClient-connection cache size at filesystem level: {@value}*/
|
||||
public static final String FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE = "fs.azure.apache.http.client.max.cache.connection.size";
|
||||
/**Maximum idle time for a ApacheHttpClient-connection: {@value}*/
|
||||
public static final String FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL = "fs.azure.apache.http.client.idle.connection.ttl";
|
||||
private ConfigurationKeys() {}
|
||||
}
|
||||
|
@ -167,5 +167,14 @@ public final class FileSystemConfigurations {
|
||||
public static final int HUNDRED = 100;
|
||||
public static final long THOUSAND = 1000L;
|
||||
|
||||
public static final HttpOperationType DEFAULT_NETWORKING_LIBRARY
|
||||
= HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
|
||||
public static final int DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES = 3;
|
||||
|
||||
public static final long DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME = 5_000L;
|
||||
|
||||
public static final int DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS = 5;
|
||||
|
||||
private FileSystemConfigurations() {}
|
||||
}
|
||||
|
@ -0,0 +1,24 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.constants;
|
||||
|
||||
public enum HttpOperationType {
|
||||
JDK_HTTP_URL_CONNECTION,
|
||||
APACHE_HTTP_CLIENT;
|
||||
}
|
@ -0,0 +1,34 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
|
||||
|
||||
import org.apache.http.HttpResponse;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
|
||||
|
||||
/**
|
||||
* Exception that marks expect100 handshake error. This exception is thrown when
|
||||
* the expect100 handshake fails with ADLS server sending 4xx or 5xx status code.
|
||||
*/
|
||||
public class AbfsApacheHttpExpect100Exception extends HttpResponseException {
|
||||
|
||||
public AbfsApacheHttpExpect100Exception(final HttpResponse httpResponse) {
|
||||
super(EXPECT_100_JDK_ERROR, httpResponse);
|
||||
}
|
||||
}
|
@ -0,0 +1,40 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.http.HttpResponse;
|
||||
|
||||
/**
|
||||
* Encapsulates an exception thrown from ApacheHttpClient response parsing.
|
||||
*/
|
||||
public class HttpResponseException extends IOException {
|
||||
private final HttpResponse httpResponse;
|
||||
public HttpResponseException(final String s, final HttpResponse httpResponse) {
|
||||
super(s);
|
||||
Objects.requireNonNull(httpResponse, "httpResponse should be non-null");
|
||||
this.httpResponse = httpResponse;
|
||||
}
|
||||
|
||||
public HttpResponse getHttpResponse() {
|
||||
return httpResponse;
|
||||
}
|
||||
}
|
@ -0,0 +1,394 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.net.URL;
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
|
||||
import org.apache.http.Header;
|
||||
import org.apache.http.HttpEntity;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.client.methods.CloseableHttpResponse;
|
||||
import org.apache.http.client.methods.HttpDelete;
|
||||
import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
|
||||
import org.apache.http.client.methods.HttpGet;
|
||||
import org.apache.http.client.methods.HttpHead;
|
||||
import org.apache.http.client.methods.HttpPatch;
|
||||
import org.apache.http.client.methods.HttpPost;
|
||||
import org.apache.http.client.methods.HttpPut;
|
||||
import org.apache.http.client.methods.HttpRequestBase;
|
||||
import org.apache.http.entity.ByteArrayEntity;
|
||||
import org.apache.http.util.EntityUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APACHE_IMPL;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_POST;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
|
||||
import static org.apache.http.entity.ContentType.TEXT_PLAIN;
|
||||
|
||||
/**
|
||||
* Implementation of {@link AbfsHttpOperation} for orchestrating server calls using
|
||||
* Apache Http Client.
|
||||
*/
|
||||
public class AbfsAHCHttpOperation extends AbfsHttpOperation {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
AbfsAHCHttpOperation.class);
|
||||
|
||||
/**
|
||||
* Request object for network call over ApacheHttpClient.
|
||||
*/
|
||||
private final HttpRequestBase httpRequestBase;
|
||||
|
||||
/**
|
||||
* Response object received from a server call over ApacheHttpClient.
|
||||
*/
|
||||
private HttpResponse httpResponse;
|
||||
|
||||
/**
|
||||
* Flag to indicate if the request is a payload request. HTTP methods PUT, POST,
|
||||
* PATCH qualify for payload requests.
|
||||
*/
|
||||
private final boolean isPayloadRequest;
|
||||
|
||||
/**
|
||||
* ApacheHttpClient to make network calls.
|
||||
*/
|
||||
private final AbfsApacheHttpClient abfsApacheHttpClient;
|
||||
|
||||
public AbfsAHCHttpOperation(final URL url,
|
||||
final String method,
|
||||
final List<AbfsHttpHeader> requestHeaders,
|
||||
final Duration connectionTimeout,
|
||||
final Duration readTimeout,
|
||||
final AbfsApacheHttpClient abfsApacheHttpClient) throws IOException {
|
||||
super(LOG, url, method, requestHeaders, connectionTimeout, readTimeout);
|
||||
this.isPayloadRequest = HTTP_METHOD_PUT.equals(method)
|
||||
|| HTTP_METHOD_PATCH.equals(method)
|
||||
|| HTTP_METHOD_POST.equals(method);
|
||||
this.abfsApacheHttpClient = abfsApacheHttpClient;
|
||||
LOG.debug("Creating AbfsAHCHttpOperation for URL: {}, method: {}",
|
||||
url, method);
|
||||
|
||||
final URI requestUri;
|
||||
try {
|
||||
requestUri = url.toURI();
|
||||
} catch (URISyntaxException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
switch (getMethod()) {
|
||||
case HTTP_METHOD_PUT:
|
||||
httpRequestBase = new HttpPut(requestUri);
|
||||
break;
|
||||
case HTTP_METHOD_PATCH:
|
||||
httpRequestBase = new HttpPatch(requestUri);
|
||||
break;
|
||||
case HTTP_METHOD_POST:
|
||||
httpRequestBase = new HttpPost(requestUri);
|
||||
break;
|
||||
case HTTP_METHOD_GET:
|
||||
httpRequestBase = new HttpGet(requestUri);
|
||||
break;
|
||||
case HTTP_METHOD_DELETE:
|
||||
httpRequestBase = new HttpDelete(requestUri);
|
||||
break;
|
||||
case HTTP_METHOD_HEAD:
|
||||
httpRequestBase = new HttpHead(requestUri);
|
||||
break;
|
||||
default:
|
||||
/*
|
||||
* This would not happen as the AbfsClient would always be sending valid
|
||||
* method.
|
||||
*/
|
||||
throw new PathIOException(getUrl().toString(),
|
||||
"Unsupported HTTP method: " + getMethod());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return AbfsManagedHttpClientContext instance that captures latencies at
|
||||
* different phases of network call.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
AbfsManagedHttpClientContext getHttpClientContext() {
|
||||
return new AbfsManagedHttpClientContext();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
protected InputStream getErrorStream() throws IOException {
|
||||
HttpEntity entity = httpResponse.getEntity();
|
||||
if (entity == null) {
|
||||
return null;
|
||||
}
|
||||
return entity.getContent();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
String getConnProperty(final String key) {
|
||||
for (AbfsHttpHeader header : getRequestHeaders()) {
|
||||
if (header.getName().equals(key)) {
|
||||
return header.getValue();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
URL getConnUrl() {
|
||||
return getUrl();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
Integer getConnResponseCode() throws IOException {
|
||||
return getStatusCode();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
String getConnResponseMessage() throws IOException {
|
||||
return getStatusDescription();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void processResponse(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length) throws IOException {
|
||||
try {
|
||||
if (!isPayloadRequest) {
|
||||
prepareRequest();
|
||||
LOG.debug("Sending request: {}", httpRequestBase);
|
||||
httpResponse = executeRequest();
|
||||
LOG.debug("Request sent: {}; response {}", httpRequestBase,
|
||||
httpResponse);
|
||||
}
|
||||
parseResponseHeaderAndBody(buffer, offset, length);
|
||||
} finally {
|
||||
if (httpResponse != null) {
|
||||
try {
|
||||
EntityUtils.consume(httpResponse.getEntity());
|
||||
} finally {
|
||||
if (httpResponse instanceof CloseableHttpResponse) {
|
||||
((CloseableHttpResponse) httpResponse).close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse response stream for headers and body.
|
||||
*
|
||||
* @param buffer byte array to store response body.
|
||||
* @param offset offset in the buffer to start storing the response body.
|
||||
* @param length length of the response body.
|
||||
*
|
||||
* @throws IOException network error while read response stream
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void parseResponseHeaderAndBody(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length) throws IOException {
|
||||
setStatusCode(parseStatusCode(httpResponse));
|
||||
|
||||
setStatusDescription(httpResponse.getStatusLine().getReasonPhrase());
|
||||
setRequestId();
|
||||
|
||||
// dump the headers
|
||||
if (LOG.isDebugEnabled()) {
|
||||
AbfsIoUtils.dumpHeadersToDebugLog("Request Headers",
|
||||
getRequestProperties());
|
||||
}
|
||||
parseResponse(buffer, offset, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse status code from response
|
||||
*
|
||||
* @param httpResponse response object
|
||||
* @return status code
|
||||
*/
|
||||
@VisibleForTesting
|
||||
int parseStatusCode(HttpResponse httpResponse) {
|
||||
return httpResponse.getStatusLine().getStatusCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute network call for the request
|
||||
*
|
||||
* @return response object
|
||||
* @throws IOException network error while executing the request
|
||||
*/
|
||||
@VisibleForTesting
|
||||
HttpResponse executeRequest() throws IOException {
|
||||
AbfsManagedHttpClientContext abfsHttpClientContext
|
||||
= getHttpClientContext();
|
||||
try {
|
||||
LOG.debug("Executing request: {}", httpRequestBase);
|
||||
HttpResponse response = abfsApacheHttpClient.execute(httpRequestBase,
|
||||
abfsHttpClientContext, getConnectionTimeout(), getReadTimeout());
|
||||
setConnectionTimeMs(abfsHttpClientContext.getConnectTime());
|
||||
setSendRequestTimeMs(abfsHttpClientContext.getSendTime());
|
||||
setRecvResponseTimeMs(abfsHttpClientContext.getReadTime());
|
||||
return response;
|
||||
} catch (IOException e) {
|
||||
LOG.debug("Failed to execute request: {}", httpRequestBase, e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void setRequestProperty(final String key, final String value) {
|
||||
List<AbfsHttpHeader> headers = getRequestHeaders();
|
||||
if (headers != null) {
|
||||
headers.add(new AbfsHttpHeader(key, value));
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
Map<String, List<String>> getRequestProperties() {
|
||||
Map<String, List<String>> map = new HashMap<>();
|
||||
for (AbfsHttpHeader header : getRequestHeaders()) {
|
||||
map.put(header.getName(),
|
||||
new ArrayList<String>() {{
|
||||
add(header.getValue());
|
||||
}});
|
||||
}
|
||||
return map;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public String getResponseHeader(final String headerName) {
|
||||
if (httpResponse == null) {
|
||||
return null;
|
||||
}
|
||||
Header header = httpResponse.getFirstHeader(headerName);
|
||||
if (header != null) {
|
||||
return header.getValue();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
protected InputStream getContentInputStream()
|
||||
throws IOException {
|
||||
if (httpResponse == null || httpResponse.getEntity() == null) {
|
||||
return null;
|
||||
}
|
||||
return httpResponse.getEntity().getContent();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void sendPayload(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length)
|
||||
throws IOException {
|
||||
if (!isPayloadRequest) {
|
||||
return;
|
||||
}
|
||||
|
||||
setExpectedBytesToBeSent(length);
|
||||
if (buffer != null) {
|
||||
HttpEntity httpEntity = new ByteArrayEntity(buffer, offset, length,
|
||||
TEXT_PLAIN);
|
||||
((HttpEntityEnclosingRequestBase) httpRequestBase).setEntity(
|
||||
httpEntity);
|
||||
}
|
||||
|
||||
prepareRequest();
|
||||
try {
|
||||
LOG.debug("Sending request: {}", httpRequestBase);
|
||||
httpResponse = executeRequest();
|
||||
} catch (AbfsApacheHttpExpect100Exception ex) {
|
||||
LOG.debug(
|
||||
"Getting output stream failed with expect header enabled, returning back."
|
||||
+ "Expect 100 assertion failed for uri {} with status code: {}",
|
||||
getMaskedUrl(), parseStatusCode(ex.getHttpResponse()),
|
||||
ex);
|
||||
setConnectionDisconnectedOnError();
|
||||
httpResponse = ex.getHttpResponse();
|
||||
} catch (IOException ex) {
|
||||
LOG.debug("Getting output stream failed for uri {}, exception: {}",
|
||||
getMaskedUrl(), ex);
|
||||
throw ex;
|
||||
} finally {
|
||||
if (httpResponse != null) {
|
||||
LOG.debug("Request sent: {}; response {}", httpRequestBase,
|
||||
httpResponse);
|
||||
}
|
||||
if (!isConnectionDisconnectedOnError()
|
||||
&& httpRequestBase instanceof HttpEntityEnclosingRequestBase) {
|
||||
setBytesSent(length);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the header on the request.
|
||||
*/
|
||||
private void prepareRequest() {
|
||||
for (AbfsHttpHeader header : getRequestHeaders()) {
|
||||
httpRequestBase.setHeader(header.getName(), header.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public String getRequestProperty(String name) {
|
||||
for (AbfsHttpHeader header : getRequestHeaders()) {
|
||||
if (header.getName().equals(name)) {
|
||||
return header.getValue();
|
||||
}
|
||||
}
|
||||
return EMPTY_STRING;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public String getTracingContextSuffix() {
|
||||
return APACHE_IMPL;
|
||||
}
|
||||
}
|
@ -0,0 +1,144 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.client.config.RequestConfig;
|
||||
import org.apache.http.client.methods.HttpRequestBase;
|
||||
import org.apache.http.config.Registry;
|
||||
import org.apache.http.config.RegistryBuilder;
|
||||
import org.apache.http.conn.socket.ConnectionSocketFactory;
|
||||
import org.apache.http.conn.socket.PlainConnectionSocketFactory;
|
||||
import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.impl.client.HttpClientBuilder;
|
||||
import org.apache.http.impl.client.HttpClients;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTP_SCHEME;
|
||||
import static org.apache.http.conn.ssl.SSLConnectionSocketFactory.getDefaultHostnameVerifier;
|
||||
|
||||
/**
|
||||
* Client for AzureBlobFileSystem to execute HTTP requests over ApacheHttpClient.
|
||||
*/
|
||||
final class AbfsApacheHttpClient implements Closeable {
|
||||
|
||||
/**
|
||||
* ApacheHttpClient instance that executes HTTP request.
|
||||
*/
|
||||
private final CloseableHttpClient httpClient;
|
||||
|
||||
/**
|
||||
* Flag to indicate if the client is usable. This is a JVM level flag, state of
|
||||
* this flag is shared across all instances of fileSystems. Once switched off,
|
||||
* the ApacheHttpClient would not be used for whole JVM lifecycle.
|
||||
*/
|
||||
private static boolean usable = true;
|
||||
|
||||
/**
|
||||
* Registers the switch off of ApacheHttpClient for all future use in the JVM.
|
||||
*/
|
||||
static void registerFallback() {
|
||||
usable = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return if ApacheHttpClient is usable.
|
||||
*/
|
||||
static boolean usable() {
|
||||
return usable;
|
||||
}
|
||||
|
||||
AbfsApacheHttpClient(DelegatingSSLSocketFactory delegatingSSLSocketFactory,
|
||||
final int readTimeout, final KeepAliveCache keepAliveCache) {
|
||||
final AbfsConnectionManager connMgr = new AbfsConnectionManager(
|
||||
createSocketFactoryRegistry(
|
||||
new SSLConnectionSocketFactory(delegatingSSLSocketFactory,
|
||||
getDefaultHostnameVerifier())),
|
||||
new AbfsHttpClientConnectionFactory(), keepAliveCache);
|
||||
final HttpClientBuilder builder = HttpClients.custom();
|
||||
builder.setConnectionManager(connMgr)
|
||||
.setRequestExecutor(new AbfsManagedHttpRequestExecutor(readTimeout))
|
||||
.disableContentCompression()
|
||||
.disableRedirectHandling()
|
||||
.disableAutomaticRetries()
|
||||
/*
|
||||
* To prevent the read of system property http.agent. The agent is set
|
||||
* in request headers by AbfsClient. System property read is an
|
||||
* overhead.
|
||||
*/
|
||||
.setUserAgent(EMPTY_STRING);
|
||||
httpClient = builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (httpClient != null) {
|
||||
httpClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes the HTTP request.
|
||||
*
|
||||
* @param httpRequest HTTP request to execute.
|
||||
* @param abfsHttpClientContext HttpClient context.
|
||||
* @param connectTimeout Connection timeout.
|
||||
* @param readTimeout Read timeout.
|
||||
*
|
||||
* @return HTTP response.
|
||||
* @throws IOException network error.
|
||||
*/
|
||||
public HttpResponse execute(HttpRequestBase httpRequest,
|
||||
final AbfsManagedHttpClientContext abfsHttpClientContext,
|
||||
final int connectTimeout,
|
||||
final int readTimeout) throws IOException {
|
||||
RequestConfig.Builder requestConfigBuilder = RequestConfig
|
||||
.custom()
|
||||
.setConnectTimeout(connectTimeout)
|
||||
.setSocketTimeout(readTimeout);
|
||||
httpRequest.setConfig(requestConfigBuilder.build());
|
||||
return httpClient.execute(httpRequest, abfsHttpClientContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the socket factory registry for HTTP and HTTPS.
|
||||
*
|
||||
* @param sslSocketFactory SSL socket factory.
|
||||
* @return Socket factory registry.
|
||||
*/
|
||||
private Registry<ConnectionSocketFactory> createSocketFactoryRegistry(
|
||||
ConnectionSocketFactory sslSocketFactory) {
|
||||
if (sslSocketFactory == null) {
|
||||
return RegistryBuilder.<ConnectionSocketFactory>create()
|
||||
.register(HTTP_SCHEME,
|
||||
PlainConnectionSocketFactory.getSocketFactory())
|
||||
.build();
|
||||
}
|
||||
return RegistryBuilder.<ConnectionSocketFactory>create()
|
||||
.register(HTTP_SCHEME, PlainConnectionSocketFactory.getSocketFactory())
|
||||
.register(HTTPS_SCHEME, sslSocketFactory)
|
||||
.build();
|
||||
}
|
||||
}
|
@ -42,6 +42,7 @@
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
|
||||
@ -139,6 +140,10 @@ public class AbfsClient implements Closeable {
|
||||
private boolean isSendMetricCall;
|
||||
private SharedKeyCredentials metricSharedkeyCredentials = null;
|
||||
|
||||
private KeepAliveCache keepAliveCache;
|
||||
|
||||
private AbfsApacheHttpClient abfsApacheHttpClient;
|
||||
|
||||
/**
|
||||
* logging the rename failure if metadata is in an incomplete state.
|
||||
*/
|
||||
@ -187,6 +192,15 @@ private AbfsClient(final URL baseUrl,
|
||||
+ "{}", e.getMessage());
|
||||
}
|
||||
}
|
||||
if (abfsConfiguration.getPreferredHttpOperationType()
|
||||
== HttpOperationType.APACHE_HTTP_CLIENT) {
|
||||
keepAliveCache = new KeepAliveCache(abfsConfiguration);
|
||||
|
||||
abfsApacheHttpClient = new AbfsApacheHttpClient(
|
||||
DelegatingSSLSocketFactory.getDefaultFactory(),
|
||||
abfsConfiguration.getHttpReadTimeout(),
|
||||
keepAliveCache);
|
||||
}
|
||||
|
||||
this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName);
|
||||
this.abfsPerfTracker = abfsClientContext.getAbfsPerfTracker();
|
||||
@ -255,6 +269,12 @@ public void close() throws IOException {
|
||||
runningTimerTask.cancel();
|
||||
timer.purge();
|
||||
}
|
||||
if (keepAliveCache != null) {
|
||||
keepAliveCache.close();
|
||||
}
|
||||
if (abfsApacheHttpClient != null) {
|
||||
abfsApacheHttpClient.close();
|
||||
}
|
||||
if (tokenProvider instanceof Closeable) {
|
||||
IOUtils.cleanupWithLogger(LOG,
|
||||
(Closeable) tokenProvider);
|
||||
@ -1214,7 +1234,8 @@ public AbfsRestOperation deletePath(final String path, final boolean recursive,
|
||||
this,
|
||||
HTTP_METHOD_DELETE,
|
||||
url,
|
||||
requestHeaders);
|
||||
requestHeaders,
|
||||
abfsConfiguration);
|
||||
try {
|
||||
op.execute(tracingContext);
|
||||
} catch (AzureBlobFileSystemException e) {
|
||||
@ -1600,6 +1621,9 @@ String initializeUserAgent(final AbfsConfiguration abfsConfiguration,
|
||||
sb.append(HUNDRED_CONTINUE);
|
||||
sb.append(SEMICOLON);
|
||||
}
|
||||
sb.append(SINGLE_WHITE_SPACE)
|
||||
.append(abfsConfiguration.getPreferredHttpOperationType())
|
||||
.append(SEMICOLON);
|
||||
|
||||
sb.append(SINGLE_WHITE_SPACE);
|
||||
sb.append(abfsConfiguration.getClusterName());
|
||||
@ -1928,7 +1952,8 @@ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType
|
||||
buffer,
|
||||
bufferOffset,
|
||||
bufferLength,
|
||||
sasTokenForReuse);
|
||||
sasTokenForReuse,
|
||||
abfsConfiguration);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -1949,7 +1974,8 @@ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType
|
||||
this,
|
||||
httpMethod,
|
||||
url,
|
||||
requestHeaders
|
||||
requestHeaders,
|
||||
abfsConfiguration
|
||||
);
|
||||
}
|
||||
|
||||
@ -1973,6 +1999,16 @@ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType
|
||||
this,
|
||||
httpMethod,
|
||||
url,
|
||||
requestHeaders, sasTokenForReuse);
|
||||
requestHeaders, sasTokenForReuse, abfsConfiguration);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
AbfsApacheHttpClient getAbfsApacheHttpClient() {
|
||||
return abfsApacheHttpClient;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
KeepAliveCache getKeepAliveCache() {
|
||||
return keepAliveCache;
|
||||
}
|
||||
}
|
||||
|
@ -130,6 +130,7 @@ private boolean updateBytesTransferred(boolean isThrottledOperation,
|
||||
|
||||
/**
|
||||
* Updates the metrics for successful and failed read and write operations.
|
||||
*
|
||||
* @param operationType Only applicable for read and write operations.
|
||||
* @param abfsHttpOperation Used for status code and data transferred.
|
||||
*/
|
||||
@ -170,7 +171,7 @@ public void updateMetrics(AbfsRestOperationType operationType,
|
||||
}
|
||||
break;
|
||||
case ReadFile:
|
||||
String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE);
|
||||
String range = abfsHttpOperation.getRequestProperty(HttpHeaderConfigurations.RANGE);
|
||||
contentLength = getContentLengthIfKnown(range);
|
||||
if (contentLength > 0) {
|
||||
readThrottler.addBytesTransferred(contentLength,
|
||||
|
@ -0,0 +1,207 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.http.HttpClientConnection;
|
||||
import org.apache.http.config.Registry;
|
||||
import org.apache.http.config.SocketConfig;
|
||||
import org.apache.http.conn.ConnectionPoolTimeoutException;
|
||||
import org.apache.http.conn.ConnectionRequest;
|
||||
import org.apache.http.conn.HttpClientConnectionManager;
|
||||
import org.apache.http.conn.HttpClientConnectionOperator;
|
||||
import org.apache.http.conn.ManagedHttpClientConnection;
|
||||
import org.apache.http.conn.routing.HttpRoute;
|
||||
import org.apache.http.conn.socket.ConnectionSocketFactory;
|
||||
import org.apache.http.impl.conn.DefaultHttpClientConnectionOperator;
|
||||
import org.apache.http.impl.conn.ManagedHttpClientConnectionFactory;
|
||||
import org.apache.http.protocol.HttpContext;
|
||||
|
||||
/**
|
||||
* AbfsConnectionManager is a custom implementation of {@code HttpClientConnectionManager}.
|
||||
* This implementation manages connection-pooling heuristics and custom implementation
|
||||
* of {@link ManagedHttpClientConnectionFactory}.
|
||||
*/
|
||||
class AbfsConnectionManager implements HttpClientConnectionManager {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
AbfsConnectionManager.class);
|
||||
|
||||
/**
|
||||
* Connection pool for the ABFS managed connections.
|
||||
*/
|
||||
private final KeepAliveCache kac;
|
||||
|
||||
/**
|
||||
* Factory to create new connections.
|
||||
*/
|
||||
private final AbfsHttpClientConnectionFactory httpConnectionFactory;
|
||||
|
||||
/**
|
||||
* Operator to manage the network connection state of ABFS managed connections.
|
||||
*/
|
||||
private final HttpClientConnectionOperator connectionOperator;
|
||||
|
||||
AbfsConnectionManager(Registry<ConnectionSocketFactory> socketFactoryRegistry,
|
||||
AbfsHttpClientConnectionFactory connectionFactory, KeepAliveCache kac) {
|
||||
this.httpConnectionFactory = connectionFactory;
|
||||
this.kac = kac;
|
||||
this.connectionOperator = new DefaultHttpClientConnectionOperator(
|
||||
socketFactoryRegistry, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a custom implementation of connection request for the given route.
|
||||
* The implementation would return a connection from the {@link KeepAliveCache} if available,
|
||||
* else it would create a new non-connected {@link AbfsManagedApacheHttpConnection}.
|
||||
*/
|
||||
@Override
|
||||
public ConnectionRequest requestConnection(final HttpRoute route,
|
||||
final Object state) {
|
||||
return new ConnectionRequest() {
|
||||
|
||||
/**
|
||||
* Synchronously gets a connection from the {@link KeepAliveCache} or
|
||||
* creates a new un-connected instance of {@link AbfsManagedApacheHttpConnection}.
|
||||
*/
|
||||
@Override
|
||||
public HttpClientConnection get(final long timeout,
|
||||
final TimeUnit timeUnit)
|
||||
throws InterruptedException, ExecutionException,
|
||||
ConnectionPoolTimeoutException {
|
||||
String requestId = UUID.randomUUID().toString();
|
||||
logDebug("Connection requested for request {}", requestId);
|
||||
try {
|
||||
HttpClientConnection clientConn = kac.get();
|
||||
if (clientConn != null) {
|
||||
logDebug("Connection retrieved from KAC: {} for requestId: {}",
|
||||
clientConn, requestId);
|
||||
return clientConn;
|
||||
}
|
||||
logDebug("Creating new connection for requestId: {}", requestId);
|
||||
ManagedHttpClientConnection conn = httpConnectionFactory.create(route,
|
||||
null);
|
||||
logDebug("Connection created: {} for requestId: {}", conn, requestId);
|
||||
return conn;
|
||||
} catch (IOException ex) {
|
||||
throw new ExecutionException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean cancel() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Releases a connection for reuse. It can be reused only if validDuration is greater than 0.
|
||||
* This method is called by {@link org.apache.http.impl.execchain} internal class `ConnectionHolder`.
|
||||
* If it wants to reuse the connection, it will send a non-zero validDuration, else it will send 0.
|
||||
* @param conn the connection to release
|
||||
* @param newState the new state of the connection
|
||||
* @param validDuration the duration for which the connection is valid
|
||||
* @param timeUnit the time unit for the validDuration
|
||||
*/
|
||||
@Override
|
||||
public void releaseConnection(final HttpClientConnection conn,
|
||||
final Object newState,
|
||||
final long validDuration,
|
||||
final TimeUnit timeUnit) {
|
||||
if (validDuration == 0) {
|
||||
return;
|
||||
}
|
||||
if (conn.isOpen() && conn instanceof AbfsManagedApacheHttpConnection) {
|
||||
boolean connAddedInKac = kac.put(conn);
|
||||
if (connAddedInKac) {
|
||||
logDebug("Connection cached: {}", conn);
|
||||
} else {
|
||||
logDebug("Connection not cached, and is released: {}", conn);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void connect(final HttpClientConnection conn,
|
||||
final HttpRoute route,
|
||||
final int connectTimeout,
|
||||
final HttpContext context) throws IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
logDebug("Connecting {} to {}", conn, route.getTargetHost());
|
||||
connectionOperator.connect((AbfsManagedApacheHttpConnection) conn,
|
||||
route.getTargetHost(), route.getLocalSocketAddress(),
|
||||
connectTimeout, SocketConfig.DEFAULT, context);
|
||||
logDebug("Connection established: {}", conn);
|
||||
if (context instanceof AbfsManagedHttpClientContext) {
|
||||
((AbfsManagedHttpClientContext) context).setConnectTime(
|
||||
System.currentTimeMillis() - start);
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void upgrade(final HttpClientConnection conn,
|
||||
final HttpRoute route,
|
||||
final HttpContext context) throws IOException {
|
||||
connectionOperator.upgrade((AbfsManagedApacheHttpConnection) conn,
|
||||
route.getTargetHost(), context);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void routeComplete(final HttpClientConnection conn,
|
||||
final HttpRoute route,
|
||||
final HttpContext context) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void closeIdleConnections(final long idletime,
|
||||
final TimeUnit timeUnit) {
|
||||
kac.evictIdleConnection();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void closeExpiredConnections() {
|
||||
kac.evictIdleConnection();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void shutdown() {
|
||||
kac.close();
|
||||
}
|
||||
|
||||
private void logDebug(String message, Object... args) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(message, args);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,45 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import org.apache.http.config.ConnectionConfig;
|
||||
import org.apache.http.conn.ManagedHttpClientConnection;
|
||||
import org.apache.http.conn.routing.HttpRoute;
|
||||
import org.apache.http.impl.conn.ManagedHttpClientConnectionFactory;
|
||||
|
||||
/**
|
||||
* Custom implementation of {@link ManagedHttpClientConnectionFactory} and overrides
|
||||
* {@link ManagedHttpClientConnectionFactory#create(HttpRoute, ConnectionConfig)} to return
|
||||
* {@link AbfsManagedApacheHttpConnection}.
|
||||
*/
|
||||
public class AbfsHttpClientConnectionFactory extends ManagedHttpClientConnectionFactory {
|
||||
|
||||
/**
|
||||
* Creates a new {@link AbfsManagedApacheHttpConnection} instance which has to
|
||||
* be connected.
|
||||
* @param route route for which connection is required.
|
||||
* @param config connection configuration.
|
||||
* @return new {@link AbfsManagedApacheHttpConnection} instance.
|
||||
*/
|
||||
@Override
|
||||
public ManagedHttpClientConnection create(final HttpRoute route,
|
||||
final ConnectionConfig config) {
|
||||
return new AbfsManagedApacheHttpConnection(super.create(route, config), route);
|
||||
}
|
||||
}
|
@ -20,18 +20,12 @@
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.ProtocolException;
|
||||
import java.net.URL;
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
|
||||
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
||||
import java.util.Map;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
@ -40,37 +34,39 @@
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
|
||||
|
||||
/**
|
||||
* Represents an HTTP operation.
|
||||
* Base Http operation class for orchestrating server IO calls. Child classes would
|
||||
* define the certain orchestration implementation on the basis of network library used.
|
||||
* <p>
|
||||
* For JDK netlib usage, the child class would be {@link AbfsJdkHttpOperation}. <br>
|
||||
* For ApacheHttpClient netlib usage, the child class would be {@link AbfsAHCHttpOperation}.
|
||||
*/
|
||||
public class AbfsHttpOperation implements AbfsPerfLoggable {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class);
|
||||
public abstract class AbfsHttpOperation implements AbfsPerfLoggable {
|
||||
|
||||
private final Logger log;
|
||||
|
||||
private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024;
|
||||
|
||||
private static final int ONE_THOUSAND = 1000;
|
||||
|
||||
private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND;
|
||||
|
||||
private final String method;
|
||||
private final URL url;
|
||||
private String maskedUrl;
|
||||
private String maskedEncodedUrl;
|
||||
|
||||
private HttpURLConnection connection;
|
||||
private int statusCode;
|
||||
private String statusDescription;
|
||||
private String storageErrorCode = "";
|
||||
private String storageErrorMessage = "";
|
||||
private String requestId = "";
|
||||
private String storageErrorMessage = "";
|
||||
private String requestId = "";
|
||||
private String expectedAppendPos = "";
|
||||
private ListResultSchema listResultSchema = null;
|
||||
|
||||
@ -85,6 +81,23 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
||||
private boolean shouldMask = false;
|
||||
private boolean connectionDisconnectedOnError = false;
|
||||
|
||||
/**Request headers to be sent in the request.*/
|
||||
private final List<AbfsHttpHeader> requestHeaders;
|
||||
|
||||
/**
|
||||
* Timeout that defines maximum allowed connection establishment time for a request.
|
||||
* Timeout is in milliseconds. Not all requests need to establish a new connection,
|
||||
* it depends on the connection pooling-heuristic of the networking library.
|
||||
*/
|
||||
private final int connectionTimeout;
|
||||
|
||||
/**
|
||||
* Timeout in milliseconds that defines maximum allowed time to read the response.
|
||||
* This timeout starts once request is sent. It includes server reponse time,
|
||||
* network latency, and time to read the response.
|
||||
*/
|
||||
private final int readTimeout;
|
||||
|
||||
public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(
|
||||
final URL url,
|
||||
final String method,
|
||||
@ -94,6 +107,21 @@ public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(
|
||||
return httpOp;
|
||||
}
|
||||
|
||||
public AbfsHttpOperation(
|
||||
final Logger log,
|
||||
final URL url,
|
||||
final String method,
|
||||
final List<AbfsHttpHeader> requestHeaders,
|
||||
final Duration connectionTimeout,
|
||||
final Duration readTimeout) {
|
||||
this.log = log;
|
||||
this.url = url;
|
||||
this.method = method;
|
||||
this.requestHeaders = requestHeaders;
|
||||
this.connectionTimeout = (int) connectionTimeout.toMillis();
|
||||
this.readTimeout = (int) readTimeout.toMillis();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor for FixedResult instance, avoiding connection init.
|
||||
* @param url request url
|
||||
@ -103,13 +131,25 @@ public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(
|
||||
protected AbfsHttpOperation(final URL url,
|
||||
final String method,
|
||||
final int httpStatus) {
|
||||
this.log = LoggerFactory.getLogger(AbfsHttpOperation.class);
|
||||
this.url = url;
|
||||
this.method = method;
|
||||
this.statusCode = httpStatus;
|
||||
this.requestHeaders = new ArrayList<>();
|
||||
this.connectionTimeout = 0;
|
||||
this.readTimeout = 0;
|
||||
}
|
||||
|
||||
protected HttpURLConnection getConnection() {
|
||||
return connection;
|
||||
int getConnectionTimeout() {
|
||||
return connectionTimeout;
|
||||
}
|
||||
|
||||
int getReadTimeout() {
|
||||
return readTimeout;
|
||||
}
|
||||
|
||||
List<AbfsHttpHeader> getRequestHeaders() {
|
||||
return requestHeaders;
|
||||
}
|
||||
|
||||
public String getMethod() {
|
||||
@ -137,8 +177,7 @@ public String getStorageErrorMessage() {
|
||||
}
|
||||
|
||||
public String getClientRequestId() {
|
||||
return this.connection
|
||||
.getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID);
|
||||
return getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID);
|
||||
}
|
||||
|
||||
public String getExpectedAppendPos() {
|
||||
@ -165,13 +204,21 @@ public long getBytesReceived() {
|
||||
return bytesReceived;
|
||||
}
|
||||
|
||||
public URL getUrl() {
|
||||
return url;
|
||||
}
|
||||
|
||||
public ListResultSchema getListResultSchema() {
|
||||
return listResultSchema;
|
||||
}
|
||||
|
||||
public String getResponseHeader(String httpHeader) {
|
||||
return connection.getHeaderField(httpHeader);
|
||||
}
|
||||
/**
|
||||
* Get response header value for the given headerKey.
|
||||
*
|
||||
* @param httpHeader header key.
|
||||
* @return header value.
|
||||
*/
|
||||
public abstract String getResponseHeader(String httpHeader);
|
||||
|
||||
// Returns a trace message for the request
|
||||
@Override
|
||||
@ -235,6 +282,7 @@ public String getLogString() {
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String getMaskedUrl() {
|
||||
if (!shouldMask) {
|
||||
return url.toString();
|
||||
@ -246,7 +294,7 @@ public String getMaskedUrl() {
|
||||
return maskedUrl;
|
||||
}
|
||||
|
||||
public String getMaskedEncodedUrl() {
|
||||
public final String getMaskedEncodedUrl() {
|
||||
if (maskedEncodedUrl != null) {
|
||||
return maskedEncodedUrl;
|
||||
}
|
||||
@ -255,40 +303,6 @@ public String getMaskedEncodedUrl() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes a new HTTP request and opens the connection.
|
||||
*
|
||||
* @param url The full URL including query string parameters.
|
||||
* @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
|
||||
* @param requestHeaders The HTTP request headers.READ_TIMEOUT
|
||||
* @param connectionTimeout The Connection Timeout value to be used while establishing http connection
|
||||
* @param readTimeout The Read Timeout value to be used with http connection while making a request
|
||||
* @throws IOException if an error occurs.
|
||||
*/
|
||||
public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttpHeader> requestHeaders,
|
||||
final int connectionTimeout, final int readTimeout)
|
||||
throws IOException {
|
||||
this.url = url;
|
||||
this.method = method;
|
||||
|
||||
this.connection = openConnection();
|
||||
if (this.connection instanceof HttpsURLConnection) {
|
||||
HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
|
||||
SSLSocketFactory sslSocketFactory = DelegatingSSLSocketFactory.getDefaultFactory();
|
||||
if (sslSocketFactory != null) {
|
||||
secureConn.setSSLSocketFactory(sslSocketFactory);
|
||||
}
|
||||
}
|
||||
|
||||
this.connection.setConnectTimeout(connectionTimeout);
|
||||
this.connection.setReadTimeout(readTimeout);
|
||||
this.connection.setRequestMethod(method);
|
||||
|
||||
for (AbfsHttpHeader header : requestHeaders) {
|
||||
setRequestProperty(header.getName(), header.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the HTTP request. Note that HttpUrlConnection requires that an
|
||||
* empty buffer be sent in order to set the "Content-Length: 0" header, which
|
||||
* is required by our endpoint.
|
||||
@ -299,74 +313,9 @@ public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttp
|
||||
*
|
||||
* @throws IOException if an error occurs.
|
||||
*/
|
||||
public void sendRequest(byte[] buffer, int offset, int length) throws IOException {
|
||||
this.connection.setDoOutput(true);
|
||||
this.connection.setFixedLengthStreamingMode(length);
|
||||
if (buffer == null) {
|
||||
// An empty buffer is sent to set the "Content-Length: 0" header, which
|
||||
// is required by our endpoint.
|
||||
buffer = new byte[]{};
|
||||
offset = 0;
|
||||
length = 0;
|
||||
}
|
||||
|
||||
// send the request body
|
||||
|
||||
long startTime = 0;
|
||||
startTime = System.nanoTime();
|
||||
OutputStream outputStream = null;
|
||||
// Updates the expected bytes to be sent based on length.
|
||||
this.expectedBytesToBeSent = length;
|
||||
try {
|
||||
try {
|
||||
/* Without expect header enabled, if getOutputStream() throws
|
||||
an exception, it gets caught by the restOperation. But with
|
||||
expect header enabled we return back without throwing an exception
|
||||
for the correct response code processing.
|
||||
*/
|
||||
outputStream = getConnOutputStream();
|
||||
} catch (IOException e) {
|
||||
connectionDisconnectedOnError = true;
|
||||
/* If getOutputStream fails with an expect-100 exception , we return back
|
||||
without throwing an exception to the caller. Else, we throw back the exception.
|
||||
*/
|
||||
String expectHeader = getConnProperty(EXPECT);
|
||||
if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)
|
||||
&& e instanceof ProtocolException
|
||||
&& EXPECT_100_JDK_ERROR.equals(e.getMessage())) {
|
||||
LOG.debug("Getting output stream failed with expect header enabled, returning back ", e);
|
||||
/*
|
||||
* In case expect-100 assertion has failed, headers and inputStream should not
|
||||
* be parsed. Reason being, conn.getHeaderField(), conn.getHeaderFields(),
|
||||
* conn.getInputStream() will lead to repeated server call.
|
||||
* ref: https://bugs.openjdk.org/browse/JDK-8314978.
|
||||
* Reading conn.responseCode() and conn.getResponseMessage() is safe in
|
||||
* case of Expect-100 error. Reason being, in JDK, it stores the responseCode
|
||||
* in the HttpUrlConnection object before throwing exception to the caller.
|
||||
*/
|
||||
this.statusCode = getConnResponseCode();
|
||||
this.statusDescription = getConnResponseMessage();
|
||||
return;
|
||||
} else {
|
||||
LOG.debug("Getting output stream failed without expect header enabled, throwing exception ", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
// update bytes sent for successful as well as failed attempts via the
|
||||
// accompanying statusCode.
|
||||
this.bytesSent = length;
|
||||
|
||||
// If this fails with or without expect header enabled,
|
||||
// it throws an IOException.
|
||||
outputStream.write(buffer, offset, length);
|
||||
} finally {
|
||||
// Closing the opened output stream
|
||||
if (outputStream != null) {
|
||||
outputStream.close();
|
||||
}
|
||||
this.sendRequestTimeMs = elapsedTimeMs(startTime);
|
||||
}
|
||||
}
|
||||
public abstract void sendPayload(byte[] buffer, int offset, int length) throws
|
||||
IOException;
|
||||
|
||||
/**
|
||||
* Gets and processes the HTTP response.
|
||||
@ -377,35 +326,31 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio
|
||||
*
|
||||
* @throws IOException if an error occurs.
|
||||
*/
|
||||
public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException {
|
||||
if (connectionDisconnectedOnError) {
|
||||
LOG.debug("This connection was not successful or has been disconnected, "
|
||||
+ "hence not parsing headers and inputStream");
|
||||
return;
|
||||
}
|
||||
processConnHeadersAndInputStreams(buffer, offset, length);
|
||||
}
|
||||
public abstract void processResponse(byte[] buffer,
|
||||
int offset,
|
||||
int length) throws IOException;
|
||||
|
||||
void processConnHeadersAndInputStreams(final byte[] buffer,
|
||||
/**
|
||||
* Set request header.
|
||||
*
|
||||
* @param key header key.
|
||||
* @param value header value.
|
||||
*/
|
||||
public abstract void setRequestProperty(String key, String value);
|
||||
|
||||
/**
|
||||
* Parse response body from the connection.
|
||||
*
|
||||
* @param buffer byte array to store the response body.
|
||||
* @param offset offset in the buffer.
|
||||
* @param length length of the response body.
|
||||
*
|
||||
* @throws IOException if network error occurs while reading the response.
|
||||
*/
|
||||
final void parseResponse(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length) throws IOException {
|
||||
// get the response
|
||||
long startTime = 0;
|
||||
startTime = System.nanoTime();
|
||||
|
||||
this.statusCode = getConnResponseCode();
|
||||
this.recvResponseTimeMs = elapsedTimeMs(startTime);
|
||||
|
||||
this.statusDescription = getConnResponseMessage();
|
||||
|
||||
this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID);
|
||||
if (this.requestId == null) {
|
||||
this.requestId = AbfsHttpConstants.EMPTY_STRING;
|
||||
}
|
||||
// dump the headers
|
||||
AbfsIoUtils.dumpHeadersToDebugLog("Response Headers",
|
||||
connection.getHeaderFields());
|
||||
|
||||
long startTime;
|
||||
if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(this.method)) {
|
||||
// If it is HEAD, and it is ERROR
|
||||
return;
|
||||
@ -416,12 +361,19 @@ void processConnHeadersAndInputStreams(final byte[] buffer,
|
||||
if (statusCode >= HttpURLConnection.HTTP_BAD_REQUEST) {
|
||||
processStorageErrorResponse();
|
||||
this.recvResponseTimeMs += elapsedTimeMs(startTime);
|
||||
this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0);
|
||||
String contentLength = getResponseHeader(
|
||||
HttpHeaderConfigurations.CONTENT_LENGTH);
|
||||
if (contentLength != null) {
|
||||
this.bytesReceived = Long.parseLong(contentLength);
|
||||
} else {
|
||||
this.bytesReceived = 0L;
|
||||
}
|
||||
|
||||
} else {
|
||||
// consume the input stream to release resources
|
||||
int totalBytesRead = 0;
|
||||
|
||||
try (InputStream stream = this.connection.getInputStream()) {
|
||||
try (InputStream stream = getContentInputStream()) {
|
||||
if (isNullInputStream(stream)) {
|
||||
return;
|
||||
}
|
||||
@ -429,12 +381,15 @@ void processConnHeadersAndInputStreams(final byte[] buffer,
|
||||
|
||||
// this is a list operation and need to retrieve the data
|
||||
// need a better solution
|
||||
if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) {
|
||||
if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method)
|
||||
&& buffer == null) {
|
||||
parseListFilesResponse(stream);
|
||||
} else {
|
||||
if (buffer != null) {
|
||||
while (totalBytesRead < length) {
|
||||
int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead);
|
||||
int bytesRead = stream.read(buffer, offset + totalBytesRead,
|
||||
length
|
||||
- totalBytesRead);
|
||||
if (bytesRead == -1) {
|
||||
endOfStream = true;
|
||||
break;
|
||||
@ -452,9 +407,9 @@ void processConnHeadersAndInputStreams(final byte[] buffer,
|
||||
}
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
LOG.warn("IO/Network error: {} {}: {}",
|
||||
log.warn("IO/Network error: {} {}: {}",
|
||||
method, getMaskedUrl(), ex.getMessage());
|
||||
LOG.debug("IO Error: ", ex);
|
||||
log.debug("IO Error: ", ex);
|
||||
throw ex;
|
||||
} finally {
|
||||
this.recvResponseTimeMs += elapsedTimeMs(startTime);
|
||||
@ -463,23 +418,12 @@ void processConnHeadersAndInputStreams(final byte[] buffer,
|
||||
}
|
||||
}
|
||||
|
||||
public void setRequestProperty(String key, String value) {
|
||||
this.connection.setRequestProperty(key, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Open the HTTP connection.
|
||||
*
|
||||
* @throws IOException if an error occurs.
|
||||
* Get the response stream from the connection.
|
||||
* @return InputStream: response stream from the connection after network call.
|
||||
* @throws IOException if the response stream could not be created from the connection.
|
||||
*/
|
||||
private HttpURLConnection openConnection() throws IOException {
|
||||
long start = System.nanoTime();
|
||||
try {
|
||||
return (HttpURLConnection) url.openConnection();
|
||||
} finally {
|
||||
connectionTimeMs = elapsedTimeMs(start);
|
||||
}
|
||||
}
|
||||
protected abstract InputStream getContentInputStream() throws IOException;
|
||||
|
||||
/**
|
||||
* When the request fails, this function is used to parse the responseAbfsHttpClient.LOG.debug("ExpectedError: ", ex);
|
||||
@ -499,7 +443,7 @@ private HttpURLConnection openConnection() throws IOException {
|
||||
*
|
||||
*/
|
||||
private void processStorageErrorResponse() {
|
||||
try (InputStream stream = connection.getErrorStream()) {
|
||||
try (InputStream stream = getErrorStream()) {
|
||||
if (stream == null) {
|
||||
return;
|
||||
}
|
||||
@ -536,24 +480,25 @@ private void processStorageErrorResponse() {
|
||||
// Ignore errors that occur while attempting to parse the storage
|
||||
// error, since the response may have been handled by the HTTP driver
|
||||
// or for other reasons have an unexpected
|
||||
LOG.debug("ExpectedError: ", ex);
|
||||
log.debug("ExpectedError: ", ex);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the elapsed time in milliseconds.
|
||||
* Get the error stream from the connection.
|
||||
* @return InputStream
|
||||
* @throws IOException if the error stream could not be created from the response stream.
|
||||
*/
|
||||
private long elapsedTimeMs(final long startTime) {
|
||||
return (System.nanoTime() - startTime) / ONE_MILLION;
|
||||
}
|
||||
protected abstract InputStream getErrorStream() throws IOException;
|
||||
|
||||
/**
|
||||
* Parse the list file response
|
||||
*
|
||||
* @param stream InputStream contains the list results.
|
||||
* @throws IOException
|
||||
* @throws IOException if the response cannot be deserialized.
|
||||
*/
|
||||
private void parseListFilesResponse(final InputStream stream) throws IOException {
|
||||
private void parseListFilesResponse(final InputStream stream)
|
||||
throws IOException {
|
||||
if (stream == null) {
|
||||
return;
|
||||
}
|
||||
@ -565,13 +510,21 @@ private void parseListFilesResponse(final InputStream stream) throws IOException
|
||||
|
||||
try {
|
||||
final ObjectMapper objectMapper = new ObjectMapper();
|
||||
this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class);
|
||||
this.listResultSchema = objectMapper.readValue(stream,
|
||||
ListResultSchema.class);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Unable to deserialize list results", ex);
|
||||
log.error("Unable to deserialize list results", ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the elapsed time in milliseconds.
|
||||
*/
|
||||
final long elapsedTimeMs(final long startTime) {
|
||||
return (System.nanoTime() - startTime) / ONE_MILLION;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check null stream, this is to pass findbugs's redundant check for NULL
|
||||
* @param stream InputStream
|
||||
@ -585,55 +538,148 @@ private boolean isNullInputStream(InputStream stream) {
|
||||
* @param key The request property key.
|
||||
* @return request peoperty value.
|
||||
*/
|
||||
String getConnProperty(String key) {
|
||||
return connection.getRequestProperty(key);
|
||||
}
|
||||
abstract String getConnProperty(String key);
|
||||
|
||||
/**
|
||||
* Gets the connection url.
|
||||
* @return url.
|
||||
*/
|
||||
URL getConnUrl() {
|
||||
return connection.getURL();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the connection request method.
|
||||
* @return request method.
|
||||
*/
|
||||
String getConnRequestMethod() {
|
||||
return connection.getRequestMethod();
|
||||
}
|
||||
abstract URL getConnUrl();
|
||||
|
||||
/**
|
||||
* Gets the connection response code.
|
||||
* @return response code.
|
||||
* @throws IOException
|
||||
*/
|
||||
Integer getConnResponseCode() throws IOException {
|
||||
return connection.getResponseCode();
|
||||
}
|
||||
abstract Integer getConnResponseCode() throws IOException;
|
||||
|
||||
/**
|
||||
* Gets the connection output stream.
|
||||
* @return output stream.
|
||||
* @throws IOException
|
||||
*/
|
||||
OutputStream getConnOutputStream() throws IOException {
|
||||
return connection.getOutputStream();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the connection response message.
|
||||
* @return response message.
|
||||
* @throws IOException
|
||||
*/
|
||||
String getConnResponseMessage() throws IOException {
|
||||
return connection.getResponseMessage();
|
||||
abstract String getConnResponseMessage() throws IOException;
|
||||
|
||||
/**
|
||||
* Get request headers.
|
||||
*
|
||||
* @return request headers.
|
||||
*/
|
||||
abstract Map<String, List<String>> getRequestProperties();
|
||||
|
||||
/**
|
||||
* Get request header value for a header name.
|
||||
*
|
||||
* @param headerName header name.
|
||||
* @return header value.
|
||||
*/
|
||||
abstract String getRequestProperty(String headerName);
|
||||
|
||||
boolean getConnectionDisconnectedOnError() {
|
||||
return connectionDisconnectedOnError;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Boolean getConnectionDisconnectedOnError() {
|
||||
/**
|
||||
* Get the suffix to add to the tracing context that defines what http-client is
|
||||
* used to make the network call
|
||||
* @return the suffix to distinguish http client
|
||||
*/
|
||||
public abstract String getTracingContextSuffix();
|
||||
|
||||
public final long getSendLatency() {
|
||||
return sendRequestTimeMs;
|
||||
}
|
||||
|
||||
public final long getRecvLatency() {
|
||||
return recvResponseTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set response status code for the server call.
|
||||
*
|
||||
* @param statusCode status code.
|
||||
*/
|
||||
protected void setStatusCode(final int statusCode) {
|
||||
this.statusCode = statusCode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets response status description for the server call.
|
||||
*
|
||||
* @param statusDescription status description.
|
||||
*/
|
||||
protected void setStatusDescription(final String statusDescription) {
|
||||
this.statusDescription = statusDescription;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set x-ms-request-id value from the server call response header.
|
||||
*/
|
||||
protected void setRequestId() {
|
||||
requestId = getResponseHeader(
|
||||
HttpHeaderConfigurations.X_MS_REQUEST_ID);
|
||||
if (requestId == null) {
|
||||
requestId = AbfsHttpConstants.EMPTY_STRING;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets byteSent metric.
|
||||
*
|
||||
* @param bytesSent bytes sent.
|
||||
*/
|
||||
protected void setBytesSent(final int bytesSent) {
|
||||
this.bytesSent = bytesSent;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets expected bytes to be sent.
|
||||
*
|
||||
* @param expectedBytesToBeSent expected bytes to be sent.
|
||||
*/
|
||||
protected void setExpectedBytesToBeSent(final int expectedBytesToBeSent) {
|
||||
this.expectedBytesToBeSent = expectedBytesToBeSent;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets connection time in milliseconds taken to establish the connection.
|
||||
*
|
||||
* @param connectionTimeMs connection time in milliseconds.
|
||||
*/
|
||||
protected void setConnectionTimeMs(final long connectionTimeMs) {
|
||||
this.connectionTimeMs = connectionTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets send request time in milliseconds.
|
||||
*
|
||||
* @param sendRequestTimeMs send request time in milliseconds.
|
||||
*/
|
||||
protected void setSendRequestTimeMs(final long sendRequestTimeMs) {
|
||||
this.sendRequestTimeMs = sendRequestTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets receive response time in milliseconds.
|
||||
*
|
||||
* @param recvResponseTimeMs receive response time in milliseconds.
|
||||
*/
|
||||
protected void setRecvResponseTimeMs(final long recvResponseTimeMs) {
|
||||
this.recvResponseTimeMs = recvResponseTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks network error and expect100 failures for send-payload phase.
|
||||
*/
|
||||
protected void setConnectionDisconnectedOnError() {
|
||||
this.connectionDisconnectedOnError = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return value of {@link #connectionDisconnectedOnError}
|
||||
*/
|
||||
protected boolean isConnectionDisconnectedOnError() {
|
||||
return connectionDisconnectedOnError;
|
||||
}
|
||||
|
||||
@ -652,9 +698,75 @@ public AbfsHttpOperationWithFixedResult(final URL url,
|
||||
super(url, method, httpStatus);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processResponse(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length)
|
||||
throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setRequestProperty(final String key, final String value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputStream getContentInputStream() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputStream getErrorStream() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
String getConnProperty(final String key) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
URL getConnUrl() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
Integer getConnResponseCode() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
String getConnResponseMessage() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
Map<String, List<String>> getRequestProperties() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
String getRequestProperty(final String headerName) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTracingContextSuffix() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getResponseHeader(final String httpHeader) {
|
||||
return "";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendPayload(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length)
|
||||
throws IOException {
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,300 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.ProtocolException;
|
||||
import java.net.URL;
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
|
||||
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_FALLBACK;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_IMPL;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
|
||||
|
||||
/**
|
||||
* Implementation of {@link AbfsHttpOperation} for orchestrating calls using JDK's HttpURLConnection.
|
||||
*/
|
||||
public class AbfsJdkHttpOperation extends AbfsHttpOperation {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
AbfsJdkHttpOperation.class);
|
||||
|
||||
private final HttpURLConnection connection;
|
||||
|
||||
/**
|
||||
* Initializes a new HTTP request and opens the connection.
|
||||
*
|
||||
* @param url The full URL including query string parameters.
|
||||
* @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
|
||||
* @param requestHeaders The HTTP request headers.READ_TIMEOUT
|
||||
* @param connectionTimeout The Connection Timeout value to be used while establishing http connection
|
||||
* @param readTimeout The Read Timeout value to be used with http connection while making a request
|
||||
* @throws IOException if an error occurs.
|
||||
*/
|
||||
public AbfsJdkHttpOperation(final URL url,
|
||||
final String method,
|
||||
final List<AbfsHttpHeader> requestHeaders,
|
||||
final Duration connectionTimeout,
|
||||
final Duration readTimeout)
|
||||
throws IOException {
|
||||
super(LOG, url, method, requestHeaders, connectionTimeout, readTimeout);
|
||||
|
||||
this.connection = openConnection();
|
||||
if (this.connection instanceof HttpsURLConnection) {
|
||||
HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
|
||||
SSLSocketFactory sslSocketFactory
|
||||
= DelegatingSSLSocketFactory.getDefaultFactory();
|
||||
if (sslSocketFactory != null) {
|
||||
secureConn.setSSLSocketFactory(sslSocketFactory);
|
||||
}
|
||||
}
|
||||
|
||||
this.connection.setConnectTimeout(getConnectionTimeout());
|
||||
this.connection.setReadTimeout(getReadTimeout());
|
||||
this.connection.setRequestMethod(method);
|
||||
|
||||
for (AbfsHttpHeader header : requestHeaders) {
|
||||
setRequestProperty(header.getName(), header.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
public String getResponseHeader(String httpHeader) {
|
||||
return connection.getHeaderField(httpHeader);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
public void sendPayload(byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
this.connection.setDoOutput(true);
|
||||
this.connection.setFixedLengthStreamingMode(length);
|
||||
if (buffer == null) {
|
||||
// An empty buffer is sent to set the "Content-Length: 0" header, which
|
||||
// is required by our endpoint.
|
||||
buffer = new byte[]{};
|
||||
offset = 0;
|
||||
length = 0;
|
||||
}
|
||||
|
||||
// send the request body
|
||||
|
||||
long startTime = 0;
|
||||
startTime = System.nanoTime();
|
||||
OutputStream outputStream = null;
|
||||
// Updates the expected bytes to be sent based on length.
|
||||
setExpectedBytesToBeSent(length);
|
||||
try {
|
||||
try {
|
||||
/* Without expect header enabled, if getOutputStream() throws
|
||||
an exception, it gets caught by the restOperation. But with
|
||||
expect header enabled we return back without throwing an exception
|
||||
for the correct response code processing.
|
||||
*/
|
||||
outputStream = getConnOutputStream();
|
||||
} catch (IOException e) {
|
||||
setConnectionDisconnectedOnError();
|
||||
/* If getOutputStream fails with an expect-100 exception , we return back
|
||||
without throwing an exception to the caller. Else, we throw back the exception.
|
||||
*/
|
||||
String expectHeader = getConnProperty(EXPECT);
|
||||
if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)
|
||||
&& e instanceof ProtocolException
|
||||
&& EXPECT_100_JDK_ERROR.equals(e.getMessage())) {
|
||||
LOG.debug(
|
||||
"Getting output stream failed with expect header enabled, returning back ",
|
||||
e);
|
||||
/*
|
||||
* In case expect-100 assertion has failed, headers and inputStream should not
|
||||
* be parsed. Reason being, conn.getHeaderField(), conn.getHeaderFields(),
|
||||
* conn.getInputStream() will lead to repeated server call.
|
||||
* ref: https://bugs.openjdk.org/browse/JDK-8314978.
|
||||
* Reading conn.responseCode() and conn.getResponseMessage() is safe in
|
||||
* case of Expect-100 error. Reason being, in JDK, it stores the responseCode
|
||||
* in the HttpUrlConnection object before throwing exception to the caller.
|
||||
*/
|
||||
setStatusCode(getConnResponseCode());
|
||||
setStatusDescription(getConnResponseMessage());
|
||||
return;
|
||||
} else {
|
||||
LOG.debug(
|
||||
"Getting output stream failed without expect header enabled, throwing exception ",
|
||||
e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
// update bytes sent for successful as well as failed attempts via the
|
||||
// accompanying statusCode.
|
||||
setBytesSent(length);
|
||||
|
||||
// If this fails with or without expect header enabled,
|
||||
// it throws an IOException.
|
||||
outputStream.write(buffer, offset, length);
|
||||
} finally {
|
||||
// Closing the opened output stream
|
||||
if (outputStream != null) {
|
||||
outputStream.close();
|
||||
}
|
||||
setSendRequestTimeMs(elapsedTimeMs(startTime));
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
String getRequestProperty(final String headerName) {
|
||||
return connection.getRequestProperty(headerName);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
Map<String, List<String>> getRequestProperties() {
|
||||
return connection.getRequestProperties();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
protected InputStream getContentInputStream() throws IOException {
|
||||
return connection.getInputStream();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
public void processResponse(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length) throws IOException {
|
||||
if (isConnectionDisconnectedOnError()) {
|
||||
LOG.debug("This connection was not successful or has been disconnected, "
|
||||
+ "hence not parsing headers and inputStream");
|
||||
return;
|
||||
}
|
||||
processConnHeadersAndInputStreams(buffer, offset, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses headers and body of the response. Execute server call if {@link #sendPayload(byte[], int, int)}
|
||||
* is not called.
|
||||
*
|
||||
* @param buffer buffer to store the response body.
|
||||
* @param offset offset in the buffer.
|
||||
* @param length length of the response body.
|
||||
*
|
||||
* @throws IOException network error or parsing error.
|
||||
*/
|
||||
void processConnHeadersAndInputStreams(final byte[] buffer,
|
||||
final int offset,
|
||||
final int length) throws IOException {
|
||||
// get the response
|
||||
long startTime = 0;
|
||||
startTime = System.nanoTime();
|
||||
|
||||
setStatusCode(getConnResponseCode());
|
||||
setRecvResponseTimeMs(elapsedTimeMs(startTime));
|
||||
|
||||
setStatusDescription(getConnResponseMessage());
|
||||
setRequestId();
|
||||
|
||||
// dump the headers
|
||||
AbfsIoUtils.dumpHeadersToDebugLog("Response Headers",
|
||||
connection.getHeaderFields());
|
||||
|
||||
if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(getMethod())) {
|
||||
// If it is HEAD, and it is ERROR
|
||||
return;
|
||||
}
|
||||
|
||||
parseResponse(buffer, offset, length);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
public void setRequestProperty(String key, String value) {
|
||||
this.connection.setRequestProperty(key, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link HttpURLConnection} instance. This instance is not connected.
|
||||
* Any API call on the instance would make it reuse an existing connection or
|
||||
* establish a new connection.
|
||||
*
|
||||
* @throws IOException if an error occurs.
|
||||
*/
|
||||
private HttpURLConnection openConnection() throws IOException {
|
||||
long start = System.nanoTime();
|
||||
try {
|
||||
return (HttpURLConnection) getUrl().openConnection();
|
||||
} finally {
|
||||
setConnectionTimeMs(elapsedTimeMs(start));
|
||||
}
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
protected InputStream getErrorStream() {
|
||||
return connection.getErrorStream();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
String getConnProperty(String key) {
|
||||
return connection.getRequestProperty(key);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
URL getConnUrl() {
|
||||
return connection.getURL();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
Integer getConnResponseCode() throws IOException {
|
||||
return connection.getResponseCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the connection output stream.
|
||||
* @return output stream.
|
||||
* @throws IOException if creating outputStream on connection failed
|
||||
*/
|
||||
OutputStream getConnOutputStream() throws IOException {
|
||||
return connection.getOutputStream();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
String getConnResponseMessage() throws IOException {
|
||||
return connection.getResponseMessage();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public String getTracingContextSuffix() {
|
||||
return AbfsApacheHttpClient.usable() ? JDK_IMPL : JDK_FALLBACK;
|
||||
}
|
||||
}
|
@ -0,0 +1,240 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import javax.net.ssl.SSLSession;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.Socket;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.http.HttpConnectionMetrics;
|
||||
import org.apache.http.HttpEntityEnclosingRequest;
|
||||
import org.apache.http.HttpException;
|
||||
import org.apache.http.HttpRequest;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.conn.ManagedHttpClientConnection;
|
||||
import org.apache.http.conn.routing.HttpRoute;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
|
||||
|
||||
/**
|
||||
* This class wraps the {@link ManagedHttpClientConnection} and provides
|
||||
* insights onto the connection level activity.
|
||||
*/
|
||||
class AbfsManagedApacheHttpConnection
|
||||
implements ManagedHttpClientConnection {
|
||||
|
||||
/**
|
||||
* Underlying ApacheHttpClient connection that actually does the work over network.
|
||||
*/
|
||||
private final ManagedHttpClientConnection httpClientConnection;
|
||||
|
||||
/**
|
||||
* Managed HTTP context to track the connection level activity.
|
||||
*/
|
||||
private AbfsManagedHttpClientContext managedHttpContext;
|
||||
|
||||
private final int hashCode;
|
||||
|
||||
AbfsManagedApacheHttpConnection(ManagedHttpClientConnection conn,
|
||||
final HttpRoute route) {
|
||||
this.httpClientConnection = conn;
|
||||
this.hashCode = (UUID.randomUUID().toString()
|
||||
+ httpClientConnection.getId()).hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the managed HTTP context to track the connection level activity.
|
||||
*/
|
||||
void setManagedHttpContext(AbfsManagedHttpClientContext managedHttpContext) {
|
||||
this.managedHttpContext = managedHttpContext;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
httpClientConnection.close();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
|
||||
@Override
|
||||
public boolean isOpen() {
|
||||
return httpClientConnection.isOpen();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public boolean isStale() {
|
||||
return httpClientConnection.isStale();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void setSocketTimeout(final int timeout) {
|
||||
httpClientConnection.setSocketTimeout(timeout);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public int getSocketTimeout() {
|
||||
return httpClientConnection.getSocketTimeout();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void shutdown() throws IOException {
|
||||
httpClientConnection.shutdown();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public HttpConnectionMetrics getMetrics() {
|
||||
return httpClientConnection.getMetrics();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public boolean isResponseAvailable(final int timeout) throws IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
boolean val = httpClientConnection.isResponseAvailable(timeout);
|
||||
managedHttpContext.addReadTime(System.currentTimeMillis() - start);
|
||||
return val;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void sendRequestHeader(final HttpRequest request)
|
||||
throws HttpException, IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
httpClientConnection.sendRequestHeader(request);
|
||||
managedHttpContext.addSendTime(System.currentTimeMillis() - start);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void sendRequestEntity(final HttpEntityEnclosingRequest request)
|
||||
throws HttpException, IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
httpClientConnection.sendRequestEntity(request);
|
||||
managedHttpContext.addSendTime(System.currentTimeMillis() - start);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public HttpResponse receiveResponseHeader()
|
||||
throws HttpException, IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
HttpResponse response = httpClientConnection.receiveResponseHeader();
|
||||
managedHttpContext.addReadTime(System.currentTimeMillis() - start);
|
||||
return response;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void receiveResponseEntity(final HttpResponse response)
|
||||
throws HttpException, IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
httpClientConnection.receiveResponseEntity(response);
|
||||
managedHttpContext.addReadTime(System.currentTimeMillis() - start);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
httpClientConnection.flush();
|
||||
managedHttpContext.addSendTime(System.currentTimeMillis() - start);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public String getId() {
|
||||
return httpClientConnection.getId();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void bind(final Socket socket) throws IOException {
|
||||
httpClientConnection.bind(socket);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public Socket getSocket() {
|
||||
return httpClientConnection.getSocket();
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public SSLSession getSSLSession() {
|
||||
return httpClientConnection.getSSLSession();
|
||||
}
|
||||
|
||||
/**Gets the local address to which the socket is bound.*/
|
||||
@Override
|
||||
public InetAddress getLocalAddress() {
|
||||
return httpClientConnection.getLocalAddress();
|
||||
}
|
||||
|
||||
/**Gets the local port to which the socket is bound.*/
|
||||
@Override
|
||||
public int getLocalPort() {
|
||||
return httpClientConnection.getLocalPort();
|
||||
}
|
||||
|
||||
/**Returns the address to which the socket is connected.*/
|
||||
@Override
|
||||
public InetAddress getRemoteAddress() {
|
||||
return httpClientConnection.getRemoteAddress();
|
||||
}
|
||||
|
||||
/**Returns the remote port number to which this socket is connected.*/
|
||||
@Override
|
||||
public int getRemotePort() {
|
||||
return httpClientConnection.getRemotePort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o) {
|
||||
if (o instanceof AbfsManagedApacheHttpConnection) {
|
||||
return httpClientConnection.getId().equals(
|
||||
((AbfsManagedApacheHttpConnection) o).httpClientConnection.getId());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return hashCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder stringBuilder = new StringBuilder();
|
||||
stringBuilder.append(
|
||||
httpClientConnection.getRemoteAddress().getHostName())
|
||||
.append(COLON)
|
||||
.append(httpClientConnection.getRemotePort())
|
||||
.append(COLON)
|
||||
.append(hashCode());
|
||||
return stringBuilder.toString();
|
||||
}
|
||||
}
|
@ -0,0 +1,76 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import org.apache.http.HttpClientConnection;
|
||||
import org.apache.http.client.protocol.HttpClientContext;
|
||||
|
||||
/**
|
||||
* Registers the latency of different phases of a network call.
|
||||
*/
|
||||
public class AbfsManagedHttpClientContext extends HttpClientContext {
|
||||
|
||||
/**Connection establishment time*/
|
||||
private long connectTime = 0L;
|
||||
|
||||
/**Time taken to receive and read response*/
|
||||
private long readTime = 0L;
|
||||
|
||||
/***Time taken to send request*/
|
||||
private long sendTime = 0L;
|
||||
|
||||
public AbfsManagedHttpClientContext() {
|
||||
}
|
||||
|
||||
/**
|
||||
* This to be used only in tests to get connection level activity.
|
||||
* @param httpClientConnection HttpClientConnection which has to be intercepted
|
||||
* by test-implementation.
|
||||
* @return HttpClientConnection which is intercepted by test-implementation. For production
|
||||
* implementation, it will return the same httpClientConnection.
|
||||
*/
|
||||
protected HttpClientConnection interceptConnectionActivity(
|
||||
HttpClientConnection httpClientConnection) {
|
||||
return httpClientConnection;
|
||||
}
|
||||
|
||||
public long getConnectTime() {
|
||||
return connectTime;
|
||||
}
|
||||
|
||||
public void setConnectTime(long connectTime) {
|
||||
this.connectTime = connectTime;
|
||||
}
|
||||
|
||||
public long getReadTime() {
|
||||
return readTime;
|
||||
}
|
||||
|
||||
public long getSendTime() {
|
||||
return sendTime;
|
||||
}
|
||||
|
||||
public void addSendTime(long sendTime) {
|
||||
this.sendTime += sendTime;
|
||||
}
|
||||
|
||||
public void addReadTime(long readTime) {
|
||||
this.readTime += readTime;
|
||||
}
|
||||
}
|
@ -0,0 +1,109 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
|
||||
import org.apache.http.HttpClientConnection;
|
||||
import org.apache.http.HttpException;
|
||||
import org.apache.http.HttpRequest;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.protocol.HttpContext;
|
||||
import org.apache.http.protocol.HttpRequestExecutor;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
|
||||
|
||||
/**
|
||||
* This class extends {@link HttpRequestExecutor} to intercept the connection
|
||||
* activity and register the latency of different phases of a network call. It
|
||||
* also overrides the HttpRequestExecutor's expect100 failure handling as the ADLS
|
||||
* can send any failure statusCode in expect100 hand-shake failure and non
|
||||
* necessarily 1XX code.
|
||||
*/
|
||||
public class AbfsManagedHttpRequestExecutor extends HttpRequestExecutor {
|
||||
|
||||
public AbfsManagedHttpRequestExecutor(final int expect100WaitTimeout) {
|
||||
super(expect100WaitTimeout);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public HttpResponse execute(final HttpRequest request,
|
||||
final HttpClientConnection conn,
|
||||
final HttpContext context) throws IOException, HttpException {
|
||||
if (context instanceof AbfsManagedHttpClientContext
|
||||
&& conn instanceof AbfsManagedApacheHttpConnection) {
|
||||
((AbfsManagedApacheHttpConnection) conn).setManagedHttpContext(
|
||||
(AbfsManagedHttpClientContext) context);
|
||||
}
|
||||
return super.execute(request, conn, context);
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
protected HttpResponse doSendRequest(final HttpRequest request,
|
||||
final HttpClientConnection conn,
|
||||
final HttpContext context) throws IOException, HttpException {
|
||||
final HttpClientConnection inteceptedConnection;
|
||||
if (context instanceof AbfsManagedHttpClientContext) {
|
||||
inteceptedConnection
|
||||
= ((AbfsManagedHttpClientContext) context).interceptConnectionActivity(
|
||||
conn);
|
||||
} else {
|
||||
inteceptedConnection = conn;
|
||||
}
|
||||
final HttpResponse res = super.doSendRequest(request, inteceptedConnection,
|
||||
context);
|
||||
|
||||
/*
|
||||
* ApacheHttpClient implementation does not raise an exception if the status
|
||||
* of expect100 hand-shake is not less than 200. Although it sends payload only
|
||||
* if the statusCode of the expect100 hand-shake is 100.
|
||||
*
|
||||
* ADLS can send any failure statusCode in exect100 handshake. So, an exception
|
||||
* needs to be explicitly raised if expect100 assertion is failure but the
|
||||
* ApacheHttpClient has not raised an exception.
|
||||
*
|
||||
* Response is only returned by this method if there is no expect100 request header
|
||||
* or the expect100 assertion is failed.
|
||||
*/
|
||||
if (request != null && request.containsHeader(EXPECT) && res != null) {
|
||||
throw new AbfsApacheHttpExpect100Exception(res);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
protected HttpResponse doReceiveResponse(final HttpRequest request,
|
||||
final HttpClientConnection conn,
|
||||
final HttpContext context) throws HttpException, IOException {
|
||||
final HttpClientConnection interceptedConnection;
|
||||
if (context instanceof AbfsManagedHttpClientContext) {
|
||||
interceptedConnection
|
||||
= ((AbfsManagedHttpClientContext) context).interceptConnectionActivity(
|
||||
conn);
|
||||
} else {
|
||||
interceptedConnection = conn;
|
||||
}
|
||||
return super.doReceiveResponse(request,
|
||||
interceptedConnection, context);
|
||||
}
|
||||
}
|
@ -18,6 +18,10 @@
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
/**
|
||||
* Implementation of {@link AbfsThrottlingIntercept} that does not throttle
|
||||
* the ABFS process.
|
||||
*/
|
||||
final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept {
|
||||
|
||||
public static final AbfsNoOpThrottlingIntercept INSTANCE = new AbfsNoOpThrottlingIntercept();
|
||||
@ -25,11 +29,13 @@ final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept {
|
||||
private AbfsNoOpThrottlingIntercept() {
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void updateMetrics(final AbfsRestOperationType operationType,
|
||||
final AbfsHttpOperation abfsHttpOperation) {
|
||||
final AbfsHttpOperation httpOperation) {
|
||||
}
|
||||
|
||||
/**{@inheritDoc}*/
|
||||
@Override
|
||||
public void sendingRequest(final AbfsRestOperationType operationType,
|
||||
final AbfsCounters abfsCounters) {
|
||||
|
@ -23,14 +23,19 @@
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
import java.net.UnknownHostException;
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.ClosedIOException;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsStatistic;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
|
||||
@ -40,6 +45,7 @@
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsBackoffMetrics;
|
||||
import org.apache.http.impl.execchain.RequestAbortedException;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ZERO;
|
||||
import static org.apache.hadoop.util.Time.now;
|
||||
@ -95,11 +101,18 @@ public class AbfsRestOperation {
|
||||
private String failureReason;
|
||||
private AbfsRetryPolicy retryPolicy;
|
||||
|
||||
private final AbfsConfiguration abfsConfiguration;
|
||||
|
||||
/**
|
||||
* This variable stores the tracing context used for last Rest Operation.
|
||||
*/
|
||||
private TracingContext lastUsedTracingContext;
|
||||
|
||||
/**
|
||||
* Number of retries due to IOException.
|
||||
*/
|
||||
private int apacheHttpClientIoExceptions = 0;
|
||||
|
||||
/**
|
||||
* Checks if there is non-null HTTP response.
|
||||
* @return true if there is a non-null HTTP response from the ABFS call.
|
||||
@ -150,8 +163,10 @@ String getSasToken() {
|
||||
final AbfsClient client,
|
||||
final String method,
|
||||
final URL url,
|
||||
final List<AbfsHttpHeader> requestHeaders) {
|
||||
this(operationType, client, method, url, requestHeaders, null);
|
||||
final List<AbfsHttpHeader> requestHeaders,
|
||||
final AbfsConfiguration abfsConfiguration) {
|
||||
this(operationType, client, method, url, requestHeaders, null, abfsConfiguration
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -168,7 +183,8 @@ String getSasToken() {
|
||||
final String method,
|
||||
final URL url,
|
||||
final List<AbfsHttpHeader> requestHeaders,
|
||||
final String sasToken) {
|
||||
final String sasToken,
|
||||
final AbfsConfiguration abfsConfiguration) {
|
||||
this.operationType = operationType;
|
||||
this.client = client;
|
||||
this.method = method;
|
||||
@ -185,8 +201,9 @@ String getSasToken() {
|
||||
if (abfsBackoffMetrics != null) {
|
||||
this.metricsMap = abfsBackoffMetrics.getMetricsMap();
|
||||
}
|
||||
this.maxIoRetries = client.getAbfsConfiguration().getMaxIoRetries();
|
||||
this.maxIoRetries = abfsConfiguration.getMaxIoRetries();
|
||||
this.intercept = client.getIntercept();
|
||||
this.abfsConfiguration = abfsConfiguration;
|
||||
this.retryPolicy = client.getExponentialRetryPolicy();
|
||||
}
|
||||
|
||||
@ -199,7 +216,7 @@ String getSasToken() {
|
||||
* @param url The full URL including query string parameters.
|
||||
* @param requestHeaders The HTTP request headers.
|
||||
* @param buffer For uploads, this is the request entity body. For downloads,
|
||||
* this will hold the response entity body.
|
||||
* this will hold the response entity body.
|
||||
* @param bufferOffset An offset into the buffer where the data beings.
|
||||
* @param bufferLength The length of the data in the buffer.
|
||||
* @param sasToken A sasToken for optional re-use by AbfsInputStream/AbfsOutputStream.
|
||||
@ -212,8 +229,10 @@ String getSasToken() {
|
||||
byte[] buffer,
|
||||
int bufferOffset,
|
||||
int bufferLength,
|
||||
String sasToken) {
|
||||
this(operationType, client, method, url, requestHeaders, sasToken);
|
||||
String sasToken,
|
||||
final AbfsConfiguration abfsConfiguration) {
|
||||
this(operationType, client, method, url, requestHeaders, sasToken, abfsConfiguration
|
||||
);
|
||||
this.buffer = buffer;
|
||||
this.bufferOffset = bufferOffset;
|
||||
this.bufferLength = bufferLength;
|
||||
@ -230,7 +249,9 @@ public void execute(TracingContext tracingContext)
|
||||
// triggered by a single file system call, using a new tracing context.
|
||||
lastUsedTracingContext = createNewTracingContext(tracingContext);
|
||||
try {
|
||||
abfsCounters.getLastExecutionTime().set(now());
|
||||
if (abfsCounters != null) {
|
||||
abfsCounters.getLastExecutionTime().set(now());
|
||||
}
|
||||
client.timerOrchestrator(TimerFunctionality.RESUME, null);
|
||||
IOStatisticsBinding.trackDurationOfInvocation(abfsCounters,
|
||||
AbfsStatistic.getStatNameFromHttpCall(method),
|
||||
@ -340,7 +361,7 @@ String getClientLatency() {
|
||||
*/
|
||||
private boolean executeHttpOperation(final int retryCount,
|
||||
TracingContext tracingContext) throws AzureBlobFileSystemException {
|
||||
AbfsHttpOperation httpOperation;
|
||||
final AbfsHttpOperation httpOperation;
|
||||
// Used to avoid CST Metric Update in Case of UnknownHost/IO Exception.
|
||||
boolean wasKnownExceptionThrown = false;
|
||||
|
||||
@ -361,15 +382,13 @@ private boolean executeHttpOperation(final int retryCount,
|
||||
try {
|
||||
// dump the headers
|
||||
AbfsIoUtils.dumpHeadersToDebugLog("Request Headers",
|
||||
httpOperation.getConnection().getRequestProperties());
|
||||
httpOperation.getRequestProperties());
|
||||
intercept.sendingRequest(operationType, abfsCounters);
|
||||
if (hasRequestBody) {
|
||||
// HttpUrlConnection requires
|
||||
httpOperation.sendRequest(buffer, bufferOffset, bufferLength);
|
||||
httpOperation.sendPayload(buffer, bufferOffset, bufferLength);
|
||||
incrementCounter(AbfsStatistic.SEND_REQUESTS, 1);
|
||||
incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength);
|
||||
}
|
||||
|
||||
httpOperation.processResponse(buffer, bufferOffset, bufferLength);
|
||||
if (!isThrottledRequest && httpOperation.getStatusCode()
|
||||
>= HttpURLConnection.HTTP_INTERNAL_ERROR) {
|
||||
@ -435,6 +454,9 @@ private boolean executeHttpOperation(final int retryCount,
|
||||
retryPolicy = client.getRetryPolicy(failureReason);
|
||||
LOG.warn("Unknown host name: {}. Retrying to resolve the host name...",
|
||||
hostname);
|
||||
if (httpOperation instanceof AbfsAHCHttpOperation) {
|
||||
registerApacheHttpClientIoException();
|
||||
}
|
||||
if (abfsBackoffMetrics != null) {
|
||||
synchronized (this) {
|
||||
abfsBackoffMetrics.incrementNumberOfNetworkFailedRequests();
|
||||
@ -457,6 +479,13 @@ private boolean executeHttpOperation(final int retryCount,
|
||||
}
|
||||
failureReason = RetryReason.getAbbreviation(ex, -1, "");
|
||||
retryPolicy = client.getRetryPolicy(failureReason);
|
||||
if (httpOperation instanceof AbfsAHCHttpOperation) {
|
||||
registerApacheHttpClientIoException();
|
||||
if (ex instanceof RequestAbortedException
|
||||
&& ex.getCause() instanceof ClosedIOException) {
|
||||
throw new AbfsDriverException((IOException) ex.getCause());
|
||||
}
|
||||
}
|
||||
if (!retryPolicy.shouldRetry(retryCount, -1)) {
|
||||
updateBackoffMetrics(retryCount, httpOperation.getStatusCode());
|
||||
throw new InvalidAbfsRestOperationException(ex, retryCount);
|
||||
@ -474,6 +503,18 @@ private boolean executeHttpOperation(final int retryCount,
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Registers switch off of ApacheHttpClient in case of IOException retries increases
|
||||
* more than the threshold.
|
||||
*/
|
||||
private void registerApacheHttpClientIoException() {
|
||||
apacheHttpClientIoExceptions++;
|
||||
if (apacheHttpClientIoExceptions
|
||||
>= abfsConfiguration.getMaxApacheHttpClientIoExceptionsRetries()) {
|
||||
AbfsApacheHttpClient.registerFallback();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sign an operation.
|
||||
* @param httpOperation operation to sign
|
||||
@ -483,43 +524,64 @@ private boolean executeHttpOperation(final int retryCount,
|
||||
@VisibleForTesting
|
||||
public void signRequest(final AbfsHttpOperation httpOperation, int bytesToSign) throws IOException {
|
||||
if (client.isSendMetricCall()) {
|
||||
client.getMetricSharedkeyCredentials().signRequest(httpOperation.getConnection(), bytesToSign);
|
||||
client.getMetricSharedkeyCredentials().signRequest(httpOperation, bytesToSign);
|
||||
} else {
|
||||
switch (client.getAuthType()) {
|
||||
case Custom:
|
||||
case OAuth:
|
||||
LOG.debug("Authenticating request with OAuth2 access token");
|
||||
httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
|
||||
client.getAccessToken());
|
||||
break;
|
||||
case SAS:
|
||||
// do nothing; the SAS token should already be appended to the query string
|
||||
httpOperation.setMaskForSAS(); //mask sig/oid from url for logs
|
||||
break;
|
||||
case SharedKey:
|
||||
default:
|
||||
// sign the HTTP request
|
||||
LOG.debug("Signing request with shared key");
|
||||
// sign the HTTP request
|
||||
client.getSharedKeyCredentials().signRequest(
|
||||
httpOperation.getConnection(),
|
||||
bytesToSign);
|
||||
break;
|
||||
case Custom:
|
||||
case OAuth:
|
||||
LOG.debug("Authenticating request with OAuth2 access token");
|
||||
httpOperation.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
|
||||
client.getAccessToken());
|
||||
break;
|
||||
case SAS:
|
||||
// do nothing; the SAS token should already be appended to the query string
|
||||
httpOperation.setMaskForSAS(); //mask sig/oid from url for logs
|
||||
break;
|
||||
case SharedKey:
|
||||
default:
|
||||
// sign the HTTP request
|
||||
LOG.debug("Signing request with shared key");
|
||||
// sign the HTTP request
|
||||
client.getSharedKeyCredentials().signRequest(
|
||||
httpOperation,
|
||||
bytesToSign);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates new object of {@link AbfsHttpOperation} with the url, method, requestHeader fields and
|
||||
* timeout values as set in configuration of the AbfsRestOperation object.
|
||||
*
|
||||
* @return {@link AbfsHttpOperation} to be used for sending requests
|
||||
* Creates new object of {@link AbfsHttpOperation} with the url, method, and
|
||||
* requestHeaders fields of the AbfsRestOperation object.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
AbfsHttpOperation createHttpOperation() throws IOException {
|
||||
return new AbfsHttpOperation(url, method, requestHeaders,
|
||||
client.getAbfsConfiguration().getHttpConnectionTimeout(),
|
||||
client.getAbfsConfiguration().getHttpReadTimeout());
|
||||
HttpOperationType httpOperationType
|
||||
= abfsConfiguration.getPreferredHttpOperationType();
|
||||
if (httpOperationType == HttpOperationType.APACHE_HTTP_CLIENT
|
||||
&& isApacheClientUsable()) {
|
||||
return createAbfsAHCHttpOperation();
|
||||
}
|
||||
return createAbfsHttpOperation();
|
||||
}
|
||||
|
||||
private boolean isApacheClientUsable() {
|
||||
return AbfsApacheHttpClient.usable();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
AbfsJdkHttpOperation createAbfsHttpOperation() throws IOException {
|
||||
return new AbfsJdkHttpOperation(url, method, requestHeaders,
|
||||
Duration.ofMillis(client.getAbfsConfiguration().getHttpConnectionTimeout()),
|
||||
Duration.ofMillis(client.getAbfsConfiguration().getHttpReadTimeout()));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
AbfsAHCHttpOperation createAbfsAHCHttpOperation() throws IOException {
|
||||
return new AbfsAHCHttpOperation(url, method, requestHeaders,
|
||||
Duration.ofMillis(client.getAbfsConfiguration().getHttpConnectionTimeout()),
|
||||
Duration.ofMillis(client.getAbfsConfiguration().getHttpReadTimeout()),
|
||||
client.getAbfsApacheHttpClient());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -30,11 +30,12 @@ public interface AbfsThrottlingIntercept {
|
||||
|
||||
/**
|
||||
* Updates the metrics for successful and failed read and write operations.
|
||||
*
|
||||
* @param operationType Only applicable for read and write operations.
|
||||
* @param abfsHttpOperation Used for status code and data transferred.
|
||||
* @param httpOperation Used for status code and data transferred.
|
||||
*/
|
||||
void updateMetrics(AbfsRestOperationType operationType,
|
||||
AbfsHttpOperation abfsHttpOperation);
|
||||
AbfsHttpOperation httpOperation);
|
||||
|
||||
/**
|
||||
* Called before the request is sent. Client-side throttling
|
||||
|
@ -0,0 +1,306 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Stack;
|
||||
import java.util.Timer;
|
||||
import java.util.TimerTask;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.ClosedIOException;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.http.HttpClientConnection;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_MAX_CONN_SYS_PROP;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
|
||||
|
||||
/**
|
||||
* Connection-pooling heuristics used by {@link AbfsConnectionManager}. Each
|
||||
* instance of FileSystem has its own KeepAliveCache.
|
||||
* <p>
|
||||
* Why this implementation is required in comparison to {@link org.apache.http.impl.conn.PoolingHttpClientConnectionManager}
|
||||
* connection-pooling:
|
||||
* <ol>
|
||||
* <li>PoolingHttpClientConnectionManager heuristic caches all the reusable connections it has created.
|
||||
* JDK's implementation only caches a limited number of connections. The limit is given by JVM system
|
||||
* property "http.maxConnections". If there is no system-property, it defaults to 5.</li>
|
||||
* <li>In PoolingHttpClientConnectionManager, it expects the application to provide `setMaxPerRoute` and `setMaxTotal`,
|
||||
* which the implementation uses as the total number of connections it can create. For application using ABFS, it is not
|
||||
* feasible to provide a value in the initialisation of the connectionManager. JDK's implementation has no cap on the
|
||||
* number of connections it can create.</li>
|
||||
* </ol>
|
||||
*/
|
||||
class KeepAliveCache extends Stack<KeepAliveCache.KeepAliveEntry>
|
||||
implements
|
||||
Closeable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(KeepAliveCache.class);
|
||||
|
||||
/**
|
||||
* Scheduled timer that evicts idle connections.
|
||||
*/
|
||||
private final transient Timer timer;
|
||||
|
||||
/**
|
||||
* Task provided to the timer that owns eviction logic.
|
||||
*/
|
||||
private final transient TimerTask timerTask;
|
||||
|
||||
/**
|
||||
* Flag to indicate if the cache is closed.
|
||||
*/
|
||||
private final AtomicBoolean isClosed = new AtomicBoolean(false);
|
||||
|
||||
/**
|
||||
* Counter to keep track of the number of KeepAliveCache instances created.
|
||||
*/
|
||||
private static final AtomicInteger KAC_COUNTER = new AtomicInteger(0);
|
||||
|
||||
/**
|
||||
* Maximum number of connections that can be cached.
|
||||
*/
|
||||
private final int maxConn;
|
||||
|
||||
/**
|
||||
* Time-to-live for an idle connection.
|
||||
*/
|
||||
private final long connectionIdleTTL;
|
||||
|
||||
/**
|
||||
* Flag to indicate if the eviction thread is paused.
|
||||
*/
|
||||
private final AtomicBoolean isPaused = new AtomicBoolean(false);
|
||||
|
||||
/**
|
||||
* Account name for which the cache is created. To be used only in exception
|
||||
* messages.
|
||||
*/
|
||||
private final String accountNamePath;
|
||||
|
||||
@VisibleForTesting
|
||||
synchronized void pauseThread() {
|
||||
isPaused.set(true);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
synchronized void resumeThread() {
|
||||
isPaused.set(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return connectionIdleTTL.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public long getConnectionIdleTTL() {
|
||||
return connectionIdleTTL;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an {@link KeepAliveCache} instance using filesystem's configuration.
|
||||
* <p>
|
||||
* The size of the cache is determined by the configuration
|
||||
* {@value org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys#FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE}.
|
||||
* If the configuration is not set, the system-property {@value org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants#HTTP_MAX_CONN_SYS_PROP}.
|
||||
* If the system-property is not set or set to 0, the default value
|
||||
* {@value org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations#DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS} is used.
|
||||
* <p>
|
||||
* This schedules an eviction thread to run every connectionIdleTTL milliseconds
|
||||
* given by the configuration {@link AbfsConfiguration#getMaxApacheHttpClientConnectionIdleTime()}.
|
||||
* @param abfsConfiguration Configuration of the filesystem.
|
||||
*/
|
||||
KeepAliveCache(AbfsConfiguration abfsConfiguration) {
|
||||
accountNamePath = abfsConfiguration.getAccountName();
|
||||
this.timer = new Timer("abfs-kac-" + KAC_COUNTER.getAndIncrement(), true);
|
||||
|
||||
int sysPropMaxConn = Integer.parseInt(System.getProperty(HTTP_MAX_CONN_SYS_PROP, "0"));
|
||||
final int defaultMaxConn;
|
||||
if (sysPropMaxConn > 0) {
|
||||
defaultMaxConn = sysPropMaxConn;
|
||||
} else {
|
||||
defaultMaxConn = DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
|
||||
}
|
||||
this.maxConn = abfsConfiguration.getInt(
|
||||
FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE,
|
||||
defaultMaxConn);
|
||||
|
||||
this.connectionIdleTTL
|
||||
= abfsConfiguration.getMaxApacheHttpClientConnectionIdleTime();
|
||||
this.timerTask = new TimerTask() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (isPaused.get() || isClosed.get()) {
|
||||
return;
|
||||
}
|
||||
evictIdleConnection();
|
||||
}
|
||||
};
|
||||
timer.schedule(timerTask, 0, connectionIdleTTL);
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate over the cache and evict the idle connections. An idle connection is
|
||||
* one that has been in the cache for more than connectionIdleTTL milliseconds.
|
||||
*/
|
||||
synchronized void evictIdleConnection() {
|
||||
long currentTime = System.currentTimeMillis();
|
||||
int i;
|
||||
for (i = 0; i < size(); i++) {
|
||||
KeepAliveEntry e = elementAt(i);
|
||||
if ((currentTime - e.idleStartTime) > connectionIdleTTL
|
||||
|| e.httpClientConnection.isStale()) {
|
||||
HttpClientConnection hc = e.httpClientConnection;
|
||||
closeHttpClientConnection(hc);
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
subList(0, i).clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Safe close of the HttpClientConnection.
|
||||
*
|
||||
* @param hc HttpClientConnection to be closed
|
||||
*/
|
||||
private void closeHttpClientConnection(final HttpClientConnection hc) {
|
||||
try {
|
||||
hc.close();
|
||||
} catch (IOException ex) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Close failed for connection: {}", hc, ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close all connections in cache and cancel the eviction timer.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void close() {
|
||||
boolean closed = isClosed.getAndSet(true);
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
closeInternal();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void closeInternal() {
|
||||
timerTask.cancel();
|
||||
timer.purge();
|
||||
while (!empty()) {
|
||||
KeepAliveEntry e = pop();
|
||||
closeHttpClientConnection(e.httpClientConnection);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Gets the latest added HttpClientConnection from the cache. The returned connection
|
||||
* is non-stale and has been in the cache for less than connectionIdleTTL milliseconds.
|
||||
* <p>
|
||||
* The cache is checked from the top of the stack. If the connection is stale or has been
|
||||
* in the cache for more than connectionIdleTTL milliseconds, it is closed and the next
|
||||
* connection is checked. Once a valid connection is found, it is returned.
|
||||
* @return HttpClientConnection: if a valid connection is found, else null.
|
||||
* @throws IOException if the cache is closed.
|
||||
*/
|
||||
public synchronized HttpClientConnection get()
|
||||
throws IOException {
|
||||
if (isClosed.get()) {
|
||||
throw new ClosedIOException(accountNamePath, KEEP_ALIVE_CACHE_CLOSED);
|
||||
}
|
||||
if (empty()) {
|
||||
return null;
|
||||
}
|
||||
HttpClientConnection hc = null;
|
||||
long currentTime = System.currentTimeMillis();
|
||||
do {
|
||||
KeepAliveEntry e = pop();
|
||||
if ((currentTime - e.idleStartTime) > connectionIdleTTL
|
||||
|| e.httpClientConnection.isStale()) {
|
||||
closeHttpClientConnection(e.httpClientConnection);
|
||||
} else {
|
||||
hc = e.httpClientConnection;
|
||||
}
|
||||
} while ((hc == null) && (!empty()));
|
||||
return hc;
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts the HttpClientConnection in the cache. If the size of cache is equal to
|
||||
* maxConn, the oldest connection is closed and removed from the cache, which
|
||||
* will make space for the new connection. If the cache is closed or of zero size,
|
||||
* the connection is closed and not added to the cache.
|
||||
*
|
||||
* @param httpClientConnection HttpClientConnection to be cached
|
||||
* @return true if the HttpClientConnection is added in active cache, false otherwise.
|
||||
*/
|
||||
public synchronized boolean put(HttpClientConnection httpClientConnection) {
|
||||
if (isClosed.get() || maxConn == 0) {
|
||||
closeHttpClientConnection(httpClientConnection);
|
||||
return false;
|
||||
}
|
||||
if (size() == maxConn) {
|
||||
closeHttpClientConnection(get(0).httpClientConnection);
|
||||
subList(0, 1).clear();
|
||||
}
|
||||
KeepAliveEntry entry = new KeepAliveEntry(httpClientConnection,
|
||||
System.currentTimeMillis());
|
||||
push(entry);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean equals(final Object o) {
|
||||
return super.equals(o);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int hashCode() {
|
||||
return super.hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Entry data-structure in the cache.
|
||||
*/
|
||||
static class KeepAliveEntry {
|
||||
|
||||
/**HttpClientConnection in the cache entry.*/
|
||||
private final HttpClientConnection httpClientConnection;
|
||||
|
||||
/**Time at which the HttpClientConnection was added to the cache.*/
|
||||
private final long idleStartTime;
|
||||
|
||||
KeepAliveEntry(HttpClientConnection hc, long idleStartTime) {
|
||||
this.httpClientConnection = hc;
|
||||
this.idleStartTime = idleStartTime;
|
||||
}
|
||||
}
|
||||
}
|
@ -21,7 +21,6 @@
|
||||
import javax.crypto.Mac;
|
||||
import javax.crypto.spec.SecretKeySpec;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
import java.net.URLDecoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
@ -81,7 +80,7 @@ public SharedKeyCredentials(final String accountName,
|
||||
initializeMac();
|
||||
}
|
||||
|
||||
public void signRequest(HttpURLConnection connection, final long contentLength) throws UnsupportedEncodingException {
|
||||
public void signRequest(AbfsHttpOperation connection, final long contentLength) throws UnsupportedEncodingException {
|
||||
|
||||
String gmtTime = getGMTTime();
|
||||
connection.setRequestProperty(HttpHeaderConfigurations.X_MS_DATE, gmtTime);
|
||||
@ -113,7 +112,7 @@ private String computeHmac256(final String stringToSign) {
|
||||
* @param conn the HttpURLConnection for the operation
|
||||
* @param canonicalizedString the canonicalized string to add the canonicalized headerst to.
|
||||
*/
|
||||
private static void addCanonicalizedHeaders(final HttpURLConnection conn, final StringBuilder canonicalizedString) {
|
||||
private static void addCanonicalizedHeaders(final AbfsHttpOperation conn, final StringBuilder canonicalizedString) {
|
||||
// Look for header names that start with
|
||||
// HeaderNames.PrefixForStorageHeader
|
||||
// Then sort them in case-insensitive manner.
|
||||
@ -201,13 +200,13 @@ private static void appendCanonicalizedElement(final StringBuilder builder, fina
|
||||
*/
|
||||
private static String canonicalizeHttpRequest(final URL address,
|
||||
final String accountName, final String method, final String contentType,
|
||||
final long contentLength, final String date, final HttpURLConnection conn)
|
||||
final long contentLength, final String date, final AbfsHttpOperation conn)
|
||||
throws UnsupportedEncodingException {
|
||||
|
||||
// The first element should be the Method of the request.
|
||||
// I.e. GET, POST, PUT, or HEAD.
|
||||
final StringBuilder canonicalizedString = new StringBuilder(EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH);
|
||||
canonicalizedString.append(conn.getRequestMethod());
|
||||
canonicalizedString.append(conn.getMethod());
|
||||
|
||||
// The next elements are
|
||||
// If any element is missing it may be empty.
|
||||
@ -446,7 +445,7 @@ private static String trimStart(final String value) {
|
||||
return value.substring(spaceDex);
|
||||
}
|
||||
|
||||
private static String getHeaderValue(final HttpURLConnection conn, final String headerName, final String defaultValue) {
|
||||
private static String getHeaderValue(final AbfsHttpOperation conn, final String headerName, final String defaultValue) {
|
||||
final String headerValue = conn.getRequestProperty(headerName);
|
||||
return headerValue == null ? defaultValue : headerValue;
|
||||
}
|
||||
@ -461,7 +460,7 @@ private static String getHeaderValue(final HttpURLConnection conn, final String
|
||||
* -1 if unknown
|
||||
* @return a canonicalized string.
|
||||
*/
|
||||
private String canonicalize(final HttpURLConnection conn,
|
||||
private String canonicalize(final AbfsHttpOperation conn,
|
||||
final String accountName,
|
||||
final Long contentLength) throws UnsupportedEncodingException {
|
||||
|
||||
@ -472,8 +471,8 @@ private String canonicalize(final HttpURLConnection conn,
|
||||
|
||||
String contentType = getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_TYPE, "");
|
||||
|
||||
return canonicalizeHttpRequest(conn.getURL(), accountName,
|
||||
conn.getRequestMethod(), contentType, contentLength, null, conn);
|
||||
return canonicalizeHttpRequest(conn.getConnUrl(), accountName,
|
||||
conn.getMethod(), contentType, contentLength, null, conn);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -192,6 +192,7 @@ public void constructHeader(AbfsHttpOperation httpOperation, String previousFail
|
||||
+ getPrimaryRequestIdForHeader(retryCount > 0) + ":" + streamID
|
||||
+ ":" + opType + ":" + retryCount;
|
||||
header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation);
|
||||
header += (":" + httpOperation.getTracingContextSuffix());
|
||||
metricHeader += !(metricResults.trim().isEmpty()) ? metricResults : "";
|
||||
break;
|
||||
case TWO_ID_FORMAT:
|
||||
|
@ -866,6 +866,45 @@ and all associated tests to see how to make use of these extension points.
|
||||
|
||||
_Warning_ These extension points are unstable.
|
||||
|
||||
### <a href="networking"></a>Networking Layer:
|
||||
|
||||
ABFS Driver can use the following networking libraries:
|
||||
- ApacheHttpClient:
|
||||
- <a href = "https://hc.apache.org/httpcomponents-client-4.5.x/index.html">Library Documentation</a>.
|
||||
- Default networking library.
|
||||
- JDK networking library:
|
||||
- <a href="https://docs.oracle.com/javase/8/docs/api/java/net/HttpURLConnection.html">Library documentation</a>.
|
||||
|
||||
The networking library can be configured using the configuration `fs.azure.networking.library`
|
||||
while initializing the filesystem.
|
||||
Following are the supported values:
|
||||
- `APACHE_HTTP_CLIENT` : Use Apache HttpClient [Default]
|
||||
- `JDK_HTTP_URL_CONNECTION` : Use JDK networking library
|
||||
|
||||
#### <a href="ahc_networking_conf"></a>ApacheHttpClient networking layer configuration Options:
|
||||
|
||||
Following are the configuration options for ApacheHttpClient networking layer that
|
||||
can be provided at the initialization of the filesystem:
|
||||
1. `fs.azure.apache.http.client.idle.connection.ttl`:
|
||||
1. Maximum idle time in milliseconds for a connection to be kept alive in the connection pool.
|
||||
If the connection is not reused within the time limit, the connection shall be closed.
|
||||
2. Default value: 5000 milliseconds.
|
||||
2. `fs.azure.apache.http.client.max.cache.connection.size`:
|
||||
1. Maximum number of connections that can be cached in the connection pool for
|
||||
a filesystem instance. Total number of concurrent connections has no limit.
|
||||
2. Default value: 5.
|
||||
3. `fs.azure.apache.http.client.max.io.exception.retries`:
|
||||
1. Maximum number of times the client will retry on IOExceptions for a single request
|
||||
with ApacheHttpClient networking-layer. Breach of this limit would turn off
|
||||
the future uses of the ApacheHttpClient library in the current JVM instance.
|
||||
2. Default value: 3.
|
||||
|
||||
#### <a href="ahc_classpath"></a> ApacheHttpClient classpath requirements:
|
||||
|
||||
ApacheHttpClient is a `compile` maven dependency in hadoop-azure and would be
|
||||
included in the hadoop-azure jar. For using hadoop-azure with ApacheHttpClient no
|
||||
additional information is required in the classpath.
|
||||
|
||||
## <a href="options"></a> Other configuration options
|
||||
|
||||
Consult the javadocs for `org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys`,
|
||||
|
@ -33,6 +33,7 @@
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
||||
import org.apache.hadoop.fs.azurebfs.security.EncodingHelper;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.assertj.core.api.Assumptions;
|
||||
@ -51,7 +52,6 @@
|
||||
import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.EncryptionType;
|
||||
import org.apache.hadoop.fs.impl.OpenFileParameters;
|
||||
|
@ -31,6 +31,7 @@
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||
|
||||
@ -55,22 +56,71 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
|
||||
* For test performance, a full x*y test matrix is not used.
|
||||
* @return the test parameters
|
||||
*/
|
||||
@Parameterized.Parameters(name = "Size={0}-readahead={1}")
|
||||
@Parameterized.Parameters(name = "Size={0}-readahead={1}-Client={2}")
|
||||
public static Iterable<Object[]> sizes() {
|
||||
return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE, true},
|
||||
{DEFAULT_READ_BUFFER_SIZE, false},
|
||||
{DEFAULT_READ_BUFFER_SIZE, true},
|
||||
{APPENDBLOB_MAX_WRITE_BUFFER_SIZE, false},
|
||||
{MAX_BUFFER_SIZE, true}});
|
||||
return Arrays.asList(new Object[][]{
|
||||
{
|
||||
MIN_BUFFER_SIZE,
|
||||
true,
|
||||
HttpOperationType.JDK_HTTP_URL_CONNECTION
|
||||
},
|
||||
{
|
||||
MIN_BUFFER_SIZE,
|
||||
true,
|
||||
HttpOperationType.APACHE_HTTP_CLIENT
|
||||
},
|
||||
{
|
||||
DEFAULT_READ_BUFFER_SIZE,
|
||||
false,
|
||||
HttpOperationType.JDK_HTTP_URL_CONNECTION
|
||||
},
|
||||
{
|
||||
DEFAULT_READ_BUFFER_SIZE,
|
||||
false,
|
||||
HttpOperationType.APACHE_HTTP_CLIENT
|
||||
},
|
||||
{
|
||||
DEFAULT_READ_BUFFER_SIZE,
|
||||
true,
|
||||
HttpOperationType.JDK_HTTP_URL_CONNECTION
|
||||
},
|
||||
{
|
||||
DEFAULT_READ_BUFFER_SIZE,
|
||||
true,
|
||||
HttpOperationType.APACHE_HTTP_CLIENT
|
||||
},
|
||||
{
|
||||
APPENDBLOB_MAX_WRITE_BUFFER_SIZE,
|
||||
false,
|
||||
HttpOperationType.JDK_HTTP_URL_CONNECTION
|
||||
},
|
||||
{
|
||||
APPENDBLOB_MAX_WRITE_BUFFER_SIZE,
|
||||
false,
|
||||
HttpOperationType.APACHE_HTTP_CLIENT
|
||||
},
|
||||
{
|
||||
MAX_BUFFER_SIZE,
|
||||
true,
|
||||
HttpOperationType.JDK_HTTP_URL_CONNECTION
|
||||
},
|
||||
{
|
||||
MAX_BUFFER_SIZE,
|
||||
true,
|
||||
HttpOperationType.APACHE_HTTP_CLIENT
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private final int size;
|
||||
private final boolean readaheadEnabled;
|
||||
private final HttpOperationType httpOperationType;
|
||||
|
||||
public ITestAbfsReadWriteAndSeek(final int size,
|
||||
final boolean readaheadEnabled) throws Exception {
|
||||
final boolean readaheadEnabled, final HttpOperationType httpOperationType) throws Exception {
|
||||
this.size = size;
|
||||
this.readaheadEnabled = readaheadEnabled;
|
||||
this.httpOperationType = httpOperationType;
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -35,6 +35,7 @@
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
@ -44,7 +45,6 @@
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
@ -56,6 +56,7 @@
|
||||
import static java.net.HttpURLConnection.HTTP_OK;
|
||||
import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.nullable;
|
||||
@ -238,7 +239,9 @@ public void testFilterFSWriteAfterClose() throws Throwable {
|
||||
intercept(FileNotFoundException.class,
|
||||
() -> {
|
||||
try (FilterOutputStream fos = new FilterOutputStream(out)) {
|
||||
fos.write('a');
|
||||
byte[] bytes = new byte[8*ONE_MB];
|
||||
fos.write(bytes);
|
||||
fos.write(bytes);
|
||||
fos.flush();
|
||||
out.hsync();
|
||||
fs.delete(testPath, false);
|
||||
|
@ -40,9 +40,9 @@
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
|
||||
import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AuthType;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.fs.permission.AclEntryScope;
|
||||
import org.apache.hadoop.fs.permission.AclStatus;
|
||||
|
@ -37,8 +37,8 @@
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TestMockHelpers;
|
||||
@ -264,7 +264,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception {
|
||||
AbfsRestOperation idempotencyRetOp = Mockito.spy(ITestAbfsClient.getRestOp(
|
||||
DeletePath, mockClient, HTTP_METHOD_DELETE,
|
||||
ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"),
|
||||
ITestAbfsClient.getTestRequestHeaders(mockClient)));
|
||||
ITestAbfsClient.getTestRequestHeaders(mockClient), getConfiguration()));
|
||||
idempotencyRetOp.hardSetResult(HTTP_OK);
|
||||
|
||||
doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any());
|
||||
|
@ -35,6 +35,7 @@
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_HTTP_CONNECTION_TIMEOUT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_HTTP_READ_TIMEOUT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES;
|
||||
@ -257,22 +258,24 @@ public void testHttpReadTimeout() throws Exception {
|
||||
}
|
||||
|
||||
public void testHttpTimeouts(int connectionTimeoutMs, int readTimeoutMs)
|
||||
throws Exception {
|
||||
throws Exception {
|
||||
Configuration conf = this.getRawConfiguration();
|
||||
// set to small values that will cause timeouts
|
||||
conf.setInt(AZURE_HTTP_CONNECTION_TIMEOUT, connectionTimeoutMs);
|
||||
conf.setInt(AZURE_HTTP_READ_TIMEOUT, readTimeoutMs);
|
||||
conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
|
||||
false);
|
||||
// Reduce retry count to reduce test run time
|
||||
conf.setInt(AZURE_MAX_IO_RETRIES, 1);
|
||||
final AzureBlobFileSystem fs = getFileSystem(conf);
|
||||
Assertions.assertThat(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getHttpConnectionTimeout())
|
||||
.describedAs("HTTP connection time should be picked from config")
|
||||
.isEqualTo(connectionTimeoutMs);
|
||||
fs.getAbfsStore().getAbfsConfiguration().getHttpConnectionTimeout())
|
||||
.describedAs("HTTP connection time should be picked from config")
|
||||
.isEqualTo(connectionTimeoutMs);
|
||||
Assertions.assertThat(
|
||||
fs.getAbfsStore().getAbfsConfiguration().getHttpReadTimeout())
|
||||
.describedAs("HTTP Read time should be picked from config")
|
||||
.isEqualTo(readTimeoutMs);
|
||||
fs.getAbfsStore().getAbfsConfiguration().getHttpReadTimeout())
|
||||
.describedAs("HTTP Read time should be picked from config")
|
||||
.isEqualTo(readTimeoutMs);
|
||||
Path testPath = path(methodName.getMethodName());
|
||||
ContractTestUtils.createFile(fs, testPath, false, new byte[0]);
|
||||
}
|
||||
|
@ -18,6 +18,7 @@
|
||||
package org.apache.hadoop.fs.azurebfs;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
|
||||
import org.junit.Assert;
|
||||
@ -28,6 +29,8 @@
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
|
||||
@ -302,11 +305,29 @@ public void testFileSystemClose() throws Exception {
|
||||
fs.close();
|
||||
Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed());
|
||||
|
||||
LambdaTestUtils.intercept(RejectedExecutionException.class, () -> {
|
||||
Callable<String> exceptionRaisingCallable = () -> {
|
||||
try (FSDataOutputStream out2 = fs.append(testFilePath)) {
|
||||
}
|
||||
return "Expected exception on new append after closed FS";
|
||||
});
|
||||
};
|
||||
/*
|
||||
* For ApacheHttpClient, the failure would happen when trying to get a connection
|
||||
* from KeepAliveCache, which is not possible after the FS is closed, as that
|
||||
* also closes the cache.
|
||||
*
|
||||
* For JDK_Client, the failure happens when trying to submit a task to the
|
||||
* executor service, which is not possible after the FS is closed, as that
|
||||
* also shuts down the executor service.
|
||||
*/
|
||||
|
||||
if (getConfiguration().getPreferredHttpOperationType()
|
||||
== HttpOperationType.APACHE_HTTP_CLIENT) {
|
||||
LambdaTestUtils.intercept(AbfsDriverException.class,
|
||||
exceptionRaisingCallable);
|
||||
} else {
|
||||
LambdaTestUtils.intercept(RejectedExecutionException.class,
|
||||
exceptionRaisingCallable);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = TEST_EXECUTION_TIMEOUT)
|
||||
|
@ -38,9 +38,9 @@
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.enums.Trilean;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AuthType;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
|
||||
@ -51,6 +51,8 @@
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
|
||||
import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.EXPONENTIAL_RETRY_POLICY_ABBREVIATION;
|
||||
import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.STATIC_RETRY_POLICY_ABBREVIATION;
|
||||
import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION;
|
||||
@ -135,10 +137,16 @@ public void runCorrelationTestForAllMethods() throws Exception {
|
||||
|
||||
testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus
|
||||
ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath"));
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true), //open,
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true, JDK_HTTP_URL_CONNECTION), //open,
|
||||
// read, write
|
||||
ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID"));
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false), //read (bypassreadahead)
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true, APACHE_HTTP_CLIENT), //open,
|
||||
// read, write
|
||||
ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID"));
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, JDK_HTTP_URL_CONNECTION), //read (bypassreadahead)
|
||||
ITestAbfsReadWriteAndSeek.class
|
||||
.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek"));
|
||||
testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, APACHE_HTTP_CLIENT), //read (bypassreadahead)
|
||||
ITestAbfsReadWriteAndSeek.class
|
||||
.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek"));
|
||||
testClasses.put(new ITestAzureBlobFileSystemAppend(), //append
|
||||
|
@ -22,6 +22,7 @@
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
@ -33,7 +34,7 @@
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
|
||||
import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.Base64;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.SASGenerator;
|
||||
@ -106,11 +107,11 @@ private byte[] getUserDelegationKey(String accountName, String appID, String app
|
||||
requestBody.append(ske);
|
||||
requestBody.append("</Expiry></KeyInfo>");
|
||||
|
||||
AbfsHttpOperation op = new AbfsHttpOperation(url, method, requestHeaders,
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, method, requestHeaders,
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
byte[] requestBuffer = requestBody.toString().getBytes(StandardCharsets.UTF_8.toString());
|
||||
op.sendRequest(requestBuffer, 0, requestBuffer.length);
|
||||
op.sendPayload(requestBuffer, 0, requestBuffer.length);
|
||||
|
||||
byte[] responseBuffer = new byte[4 * 1024];
|
||||
op.processResponse(responseBuffer, 0, responseBuffer.length);
|
||||
|
@ -63,18 +63,19 @@ private AbfsClientTestUtil() {
|
||||
|
||||
public static void setMockAbfsRestOperationForListPathOperation(
|
||||
final AbfsClient spiedClient,
|
||||
FunctionRaisingIOE<AbfsHttpOperation, AbfsHttpOperation> functionRaisingIOE)
|
||||
FunctionRaisingIOE<AbfsJdkHttpOperation, AbfsJdkHttpOperation> functionRaisingIOE)
|
||||
throws Exception {
|
||||
ExponentialRetryPolicy exponentialRetryPolicy = Mockito.mock(ExponentialRetryPolicy.class);
|
||||
StaticRetryPolicy staticRetryPolicy = Mockito.mock(StaticRetryPolicy.class);
|
||||
AbfsThrottlingIntercept intercept = Mockito.mock(AbfsThrottlingIntercept.class);
|
||||
AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
|
||||
AbfsJdkHttpOperation httpOperation = Mockito.mock(AbfsJdkHttpOperation.class);
|
||||
AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation(
|
||||
AbfsRestOperationType.ListPaths,
|
||||
spiedClient,
|
||||
HTTP_METHOD_GET,
|
||||
null,
|
||||
new ArrayList<>()
|
||||
new ArrayList<>(),
|
||||
spiedClient.getAbfsConfiguration()
|
||||
));
|
||||
|
||||
Mockito.doReturn(abfsRestOperation).when(spiedClient).getAbfsRestOperation(
|
||||
@ -99,7 +100,6 @@ public static void addGeneralMockBehaviourToRestOpAndHttpOp(final AbfsRestOperat
|
||||
HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class);
|
||||
Mockito.doNothing().when(httpURLConnection)
|
||||
.setRequestProperty(nullable(String.class), nullable(String.class));
|
||||
Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection();
|
||||
Mockito.doReturn("").when(abfsRestOperation).getClientLatency();
|
||||
Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation();
|
||||
}
|
||||
|
@ -20,26 +20,32 @@
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.ProtocolException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.net.URL;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsCountersImpl;
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
||||
import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
|
||||
@ -48,6 +54,7 @@
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
||||
import org.apache.http.HttpResponse;
|
||||
|
||||
import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
|
||||
@ -61,6 +68,8 @@
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
@ -87,6 +96,7 @@
|
||||
* Test useragent of abfs client.
|
||||
*
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
|
||||
|
||||
private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net";
|
||||
@ -100,6 +110,17 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
|
||||
|
||||
private final Pattern userAgentStringPattern;
|
||||
|
||||
@Parameterized.Parameter
|
||||
public HttpOperationType httpOperationType;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Iterable<Object[]> params() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{HttpOperationType.JDK_HTTP_URL_CONNECTION},
|
||||
{APACHE_HTTP_CLIENT}
|
||||
});
|
||||
}
|
||||
|
||||
public ITestAbfsClient() throws Exception {
|
||||
StringBuilder regEx = new StringBuilder();
|
||||
regEx.append("^");
|
||||
@ -151,6 +172,7 @@ private String getUserAgentString(AbfsConfiguration config,
|
||||
|
||||
@Test
|
||||
public void verifyBasicInfo() throws Exception {
|
||||
Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
|
||||
final Configuration configuration = new Configuration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
|
||||
@ -179,7 +201,8 @@ private void verifyBasicInfo(String userAgentStr) {
|
||||
|
||||
@Test
|
||||
public void verifyUserAgentPrefix()
|
||||
throws IOException, IllegalAccessException, URISyntaxException {
|
||||
throws IOException, IllegalAccessException, URISyntaxException {
|
||||
Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
|
||||
final Configuration configuration = new Configuration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, FS_AZURE_USER_AGENT_PREFIX);
|
||||
@ -214,6 +237,7 @@ public void verifyUserAgentPrefix()
|
||||
@Test
|
||||
public void verifyUserAgentExpectHeader()
|
||||
throws IOException, IllegalAccessException, URISyntaxException {
|
||||
Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
|
||||
final Configuration configuration = new Configuration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, FS_AZURE_USER_AGENT_PREFIX);
|
||||
@ -240,6 +264,7 @@ public void verifyUserAgentExpectHeader()
|
||||
|
||||
@Test
|
||||
public void verifyUserAgentWithoutSSLProvider() throws Exception {
|
||||
Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
|
||||
final Configuration configuration = new Configuration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY,
|
||||
@ -263,6 +288,7 @@ public void verifyUserAgentWithoutSSLProvider() throws Exception {
|
||||
|
||||
@Test
|
||||
public void verifyUserAgentClusterName() throws Exception {
|
||||
Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
|
||||
final String clusterName = "testClusterName";
|
||||
final Configuration configuration = new Configuration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
@ -291,6 +317,7 @@ public void verifyUserAgentClusterName() throws Exception {
|
||||
|
||||
@Test
|
||||
public void verifyUserAgentClusterType() throws Exception {
|
||||
Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
|
||||
final String clusterType = "testClusterType";
|
||||
final Configuration configuration = new Configuration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
@ -390,6 +417,7 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance,
|
||||
abfsConfig.getAccountName().substring(0,
|
||||
abfsConfig.getAccountName().indexOf(DOT)), abfsConfig));
|
||||
when(client.getAbfsCounters()).thenReturn(abfsCounters);
|
||||
Mockito.doReturn(baseAbfsClientInstance.getAbfsApacheHttpClient()).when(client).getAbfsApacheHttpClient();
|
||||
|
||||
// override baseurl
|
||||
client = ITestAbfsClient.setAbfsClientField(client, "abfsConfiguration",
|
||||
@ -475,13 +503,14 @@ public static AbfsRestOperation getRestOp(AbfsRestOperationType type,
|
||||
AbfsClient client,
|
||||
String method,
|
||||
URL url,
|
||||
List<AbfsHttpHeader> requestHeaders) {
|
||||
List<AbfsHttpHeader> requestHeaders, AbfsConfiguration abfsConfiguration) {
|
||||
return new AbfsRestOperation(
|
||||
type,
|
||||
client,
|
||||
method,
|
||||
url,
|
||||
requestHeaders);
|
||||
requestHeaders,
|
||||
abfsConfiguration);
|
||||
}
|
||||
|
||||
public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) {
|
||||
@ -499,6 +528,14 @@ private byte[] getRandomBytesArray(int length) {
|
||||
return b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AzureBlobFileSystem getFileSystem(final Configuration configuration)
|
||||
throws Exception {
|
||||
Configuration conf = new Configuration(configuration);
|
||||
conf.set(ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY, httpOperationType.toString());
|
||||
return (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to verify that client retries append request without
|
||||
* expect header enabled if append with expect header enabled fails
|
||||
@ -508,9 +545,10 @@ private byte[] getRandomBytesArray(int length) {
|
||||
@Test
|
||||
public void testExpectHundredContinue() throws Exception {
|
||||
// Get the filesystem.
|
||||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
final AzureBlobFileSystem fs = getFileSystem(getRawConfiguration());
|
||||
|
||||
final Configuration configuration = new Configuration();
|
||||
final Configuration configuration = fs.getAbfsStore().getAbfsConfiguration()
|
||||
.getRawConfiguration();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
AbfsClient abfsClient = fs.getAbfsStore().getClient();
|
||||
|
||||
@ -570,44 +608,49 @@ public void testExpectHundredContinue() throws Exception {
|
||||
url,
|
||||
requestHeaders, buffer,
|
||||
appendRequestParameters.getoffset(),
|
||||
appendRequestParameters.getLength(), null));
|
||||
appendRequestParameters.getLength(), null, abfsConfig));
|
||||
|
||||
AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url,
|
||||
HTTP_METHOD_PUT, requestHeaders, DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT));
|
||||
Mockito.doAnswer(answer -> {
|
||||
AbfsHttpOperation httpOperation = Mockito.spy((AbfsHttpOperation) answer.callRealMethod());
|
||||
// Sets the expect request property if expect header is enabled.
|
||||
if (appendRequestParameters.isExpectHeaderEnabled()) {
|
||||
Mockito.doReturn(HUNDRED_CONTINUE).when(httpOperation)
|
||||
.getConnProperty(EXPECT);
|
||||
}
|
||||
Mockito.doNothing().when(httpOperation).setRequestProperty(Mockito
|
||||
.any(), Mockito.any());
|
||||
Mockito.doReturn(url).when(httpOperation).getConnUrl();
|
||||
|
||||
// Sets the expect request property if expect header is enabled.
|
||||
if (appendRequestParameters.isExpectHeaderEnabled()) {
|
||||
Mockito.doReturn(HUNDRED_CONTINUE).when(abfsHttpOperation)
|
||||
.getConnProperty(EXPECT);
|
||||
}
|
||||
// Give user error code 404 when processResponse is called.
|
||||
Mockito.doReturn(HTTP_METHOD_PUT).when(httpOperation).getMethod();
|
||||
Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getStatusCode();
|
||||
Mockito.doReturn("Resource Not Found")
|
||||
.when(httpOperation)
|
||||
.getConnResponseMessage();
|
||||
|
||||
HttpURLConnection urlConnection = mock(HttpURLConnection.class);
|
||||
Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito
|
||||
.any(), Mockito.any());
|
||||
Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod();
|
||||
Mockito.doReturn(url).when(urlConnection).getURL();
|
||||
Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection();
|
||||
if (httpOperation instanceof AbfsJdkHttpOperation) {
|
||||
// Make the getOutputStream throw IOException to see it returns from the sendRequest correctly.
|
||||
Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
|
||||
.when((AbfsJdkHttpOperation) httpOperation)
|
||||
.getConnOutputStream();
|
||||
}
|
||||
|
||||
Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito
|
||||
.any(), Mockito.any());
|
||||
Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl();
|
||||
|
||||
// Give user error code 404 when processResponse is called.
|
||||
Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod();
|
||||
Mockito.doReturn(HTTP_NOT_FOUND).when(abfsHttpOperation).getConnResponseCode();
|
||||
Mockito.doReturn("Resource Not Found")
|
||||
.when(abfsHttpOperation)
|
||||
.getConnResponseMessage();
|
||||
|
||||
// Make the getOutputStream throw IOException to see it returns from the sendRequest correctly.
|
||||
Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
|
||||
.when(abfsHttpOperation)
|
||||
.getConnOutputStream();
|
||||
|
||||
// Sets the httpOperation for the rest operation.
|
||||
Mockito.doReturn(abfsHttpOperation)
|
||||
.when(op)
|
||||
.createHttpOperation();
|
||||
if (httpOperation instanceof AbfsAHCHttpOperation) {
|
||||
Mockito.doNothing()
|
||||
.when((AbfsAHCHttpOperation) httpOperation)
|
||||
.parseResponseHeaderAndBody(Mockito.any(byte[].class),
|
||||
Mockito.anyInt(), Mockito.anyInt());
|
||||
Mockito.doReturn(HTTP_NOT_FOUND)
|
||||
.when((AbfsAHCHttpOperation) httpOperation)
|
||||
.parseStatusCode(Mockito.nullable(
|
||||
HttpResponse.class));
|
||||
Mockito.doThrow(
|
||||
new AbfsApacheHttpExpect100Exception(Mockito.mock(HttpResponse.class)))
|
||||
.when((AbfsAHCHttpOperation) httpOperation)
|
||||
.executeRequest();
|
||||
}
|
||||
return httpOperation;
|
||||
}).when(op).createHttpOperation();
|
||||
|
||||
// Mock the restOperation for the client.
|
||||
Mockito.doReturn(op)
|
||||
|
@ -0,0 +1,406 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.URL;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.http.HttpClientConnection;
|
||||
import org.apache.http.HttpEntityEnclosingRequest;
|
||||
import org.apache.http.HttpException;
|
||||
import org.apache.http.HttpRequest;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.client.protocol.HttpClientContext;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
|
||||
public class ITestAbfsHttpClientRequestExecutor extends
|
||||
AbstractAbfsIntegrationTest {
|
||||
|
||||
public ITestAbfsHttpClientRequestExecutor() throws Exception {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify the correctness of expect 100 continue handling by ApacheHttpClient
|
||||
* with AbfsManagedHttpRequestExecutor.
|
||||
*/
|
||||
@Test
|
||||
public void testExpect100ContinueHandling() throws Exception {
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
Path path = new Path("/testExpect100ContinueHandling");
|
||||
|
||||
Configuration conf = new Configuration(fs.getConf());
|
||||
conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
|
||||
AzureBlobFileSystem fs2 = Mockito.spy(
|
||||
(AzureBlobFileSystem) FileSystem.newInstance(conf));
|
||||
|
||||
AzureBlobFileSystemStore store = Mockito.spy(fs2.getAbfsStore());
|
||||
Mockito.doReturn(store).when(fs2).getAbfsStore();
|
||||
|
||||
AbfsClient client = Mockito.spy(store.getClient());
|
||||
Mockito.doReturn(client).when(store).getClient();
|
||||
|
||||
final int[] invocation = {0};
|
||||
Mockito.doAnswer(answer -> {
|
||||
AbfsRestOperation op = Mockito.spy(
|
||||
(AbfsRestOperation) answer.callRealMethod());
|
||||
final ConnectionInfo connectionInfo = new ConnectionInfo();
|
||||
|
||||
/*
|
||||
* Assert that correct actions are taking place over the connection to handle
|
||||
* expect100 assertions, failure and success.
|
||||
*
|
||||
* The test would make two calls to the server. The first two calls would
|
||||
* be because of attempt to write in a non-existing file. The first call would have
|
||||
* expect100 header, and the server would respond with 404. The second call would
|
||||
* be a retry from AbfsOutputStream, and would not have expect100 header.
|
||||
*
|
||||
* The third call would be because of attempt to write in an existing file. The call
|
||||
* would have expect100 assertion pass and would send the data.
|
||||
*
|
||||
* Following is the expectation from the first attempt:
|
||||
* 1. sendHeaders should be called once. This is for expect100 assertion invocation.
|
||||
* 2. receiveResponse should be called once. This is to receive expect100 assertion.
|
||||
* 2. sendBody should not be called.
|
||||
*
|
||||
* Following is the expectation from the second attempt:
|
||||
* 1. sendHeaders should be called once. This is not for expect100 assertion invocation.
|
||||
* 2. sendBody should be called once. It will not have any expect100 assertion.
|
||||
* Once headers are sent, body is sent.
|
||||
* 3. receiveResponse should be called once. This is to receive the response from the server.
|
||||
*
|
||||
* Following is the expectation from the third attempt:
|
||||
* 1. sendHeaders should be called once. This is for expect100 assertion invocation.
|
||||
* 2. receiveResponse should be called. This is to receive the response from the server for expect100 assertion.
|
||||
* 3. sendBody called as expect100 assertion is pass.
|
||||
* 4. receiveResponse should be called. This is to receive the response from the server.
|
||||
*/
|
||||
mockHttpOperationBehavior(connectionInfo, op);
|
||||
Mockito.doAnswer(executeAnswer -> {
|
||||
invocation[0]++;
|
||||
final Throwable throwable;
|
||||
if (invocation[0] == 3) {
|
||||
executeAnswer.callRealMethod();
|
||||
throwable = null;
|
||||
} else {
|
||||
throwable = intercept(IOException.class, () -> {
|
||||
try {
|
||||
executeAnswer.callRealMethod();
|
||||
} catch (IOException ex) {
|
||||
//This exception is expected to be thrown by the op.execute() method.
|
||||
throw ex;
|
||||
} catch (Throwable interceptedAssertedThrowable) {
|
||||
//Any other throwable thrown by Mockito's callRealMethod would be
|
||||
//considered as an assertion error.
|
||||
}
|
||||
});
|
||||
}
|
||||
/*
|
||||
* The first call would be with expect headers, and expect 100 continue assertion has to happen which would fail.
|
||||
* For expect100 assertion to happen, header IO happens before body IO. If assertion fails, no body IO happens.
|
||||
* The second call would not be using expect headers.
|
||||
*
|
||||
* The third call would be with expect headers, and expect 100 continue assertion has to happen which would pass.
|
||||
*/
|
||||
if (invocation[0] == 1) {
|
||||
Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getSendBodyInvocation())
|
||||
.isEqualTo(0);
|
||||
Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(
|
||||
connectionInfo.getReceiveResponseBodyInvocation())
|
||||
.isEqualTo(1);
|
||||
}
|
||||
if (invocation[0] == 2) {
|
||||
Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getSendBodyInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(
|
||||
connectionInfo.getReceiveResponseBodyInvocation())
|
||||
.isEqualTo(1);
|
||||
}
|
||||
if (invocation[0] == 3) {
|
||||
Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getSendBodyInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
|
||||
.isEqualTo(2);
|
||||
Assertions.assertThat(
|
||||
connectionInfo.getReceiveResponseBodyInvocation())
|
||||
.isEqualTo(1);
|
||||
}
|
||||
Assertions.assertThat(invocation[0]).isLessThanOrEqualTo(3);
|
||||
if (throwable != null) {
|
||||
throw throwable;
|
||||
}
|
||||
return null;
|
||||
}).when(op).execute(Mockito.any(TracingContext.class));
|
||||
return op;
|
||||
}).when(client).getAbfsRestOperation(
|
||||
Mockito.any(AbfsRestOperationType.class),
|
||||
Mockito.anyString(),
|
||||
Mockito.any(URL.class),
|
||||
Mockito.anyList(),
|
||||
Mockito.any(byte[].class),
|
||||
Mockito.anyInt(),
|
||||
Mockito.anyInt(),
|
||||
Mockito.nullable(String.class));
|
||||
|
||||
final OutputStream os = fs2.create(path);
|
||||
fs.delete(path, true);
|
||||
intercept(FileNotFoundException.class, () -> {
|
||||
/*
|
||||
* This would lead to two server calls.
|
||||
* First call would be with expect headers, and expect 100 continue
|
||||
* assertion has to happen which would fail with 404.
|
||||
* Second call would be a retry from AbfsOutputStream, and would not be using expect headers.
|
||||
*/
|
||||
os.write(1);
|
||||
os.close();
|
||||
});
|
||||
|
||||
final OutputStream os2 = fs2.create(path);
|
||||
/*
|
||||
* This would lead to third server call. This would be with expect headers,
|
||||
* and the expect 100 continue assertion would pass.
|
||||
*/
|
||||
os2.write(1);
|
||||
os2.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a mock of HttpOperation that would be returned for AbfsRestOperation
|
||||
* to use to execute server call. To make call via ApacheHttpClient, an object
|
||||
* of {@link HttpClientContext} is required. This method would create a mock
|
||||
* of HttpClientContext that would be able to register the actions taken on
|
||||
* {@link HttpClientConnection} object. This would help in asserting the
|
||||
* order of actions taken on the connection object for making an append call with
|
||||
* expect100 header.
|
||||
*/
|
||||
private void mockHttpOperationBehavior(final ConnectionInfo connectionInfo,
|
||||
final AbfsRestOperation op) throws IOException {
|
||||
Mockito.doAnswer(httpOpCreationAnswer -> {
|
||||
AbfsAHCHttpOperation httpOperation = Mockito.spy(
|
||||
(AbfsAHCHttpOperation) httpOpCreationAnswer.callRealMethod());
|
||||
|
||||
Mockito.doAnswer(createContextAnswer -> {
|
||||
AbfsManagedHttpClientContext context = Mockito.spy(
|
||||
(AbfsManagedHttpClientContext) createContextAnswer.callRealMethod());
|
||||
Mockito.doAnswer(connectionSpyIntercept -> {
|
||||
return interceptedConn(connectionInfo,
|
||||
(HttpClientConnection) connectionSpyIntercept.getArgument(0));
|
||||
}).when(context).interceptConnectionActivity(Mockito.any(
|
||||
HttpClientConnection.class));
|
||||
return context;
|
||||
})
|
||||
.when(httpOperation).getHttpClientContext();
|
||||
return httpOperation;
|
||||
}).when(op).createHttpOperation();
|
||||
}
|
||||
|
||||
private HttpClientConnection interceptedConn(final ConnectionInfo connectionInfo,
|
||||
final HttpClientConnection connection) throws IOException, HttpException {
|
||||
HttpClientConnection interceptedConn = Mockito.spy(connection);
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
connectionInfo.incrementSendHeaderInvocation();
|
||||
long start = System.currentTimeMillis();
|
||||
Object result = answer.callRealMethod();
|
||||
connectionInfo.addSendTime(System.currentTimeMillis() - start);
|
||||
return result;
|
||||
}).when(interceptedConn).sendRequestHeader(Mockito.any(HttpRequest.class));
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
connectionInfo.incrementSendBodyInvocation();
|
||||
long start = System.currentTimeMillis();
|
||||
Object result = answer.callRealMethod();
|
||||
connectionInfo.addSendTime(System.currentTimeMillis() - start);
|
||||
return result;
|
||||
}).when(interceptedConn).sendRequestEntity(Mockito.any(
|
||||
HttpEntityEnclosingRequest.class));
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
connectionInfo.incrementReceiveResponseInvocation();
|
||||
long start = System.currentTimeMillis();
|
||||
Object result = answer.callRealMethod();
|
||||
connectionInfo.addReadTime(System.currentTimeMillis() - start);
|
||||
return result;
|
||||
}).when(interceptedConn).receiveResponseHeader();
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
connectionInfo.incrementReceiveResponseBodyInvocation();
|
||||
long start = System.currentTimeMillis();
|
||||
Object result = answer.callRealMethod();
|
||||
connectionInfo.addReadTime(System.currentTimeMillis() - start);
|
||||
return result;
|
||||
}).when(interceptedConn).receiveResponseEntity(Mockito.any(
|
||||
HttpResponse.class));
|
||||
return interceptedConn;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConnectionReadRecords() throws Exception {
|
||||
AzureBlobFileSystem fs = getFileSystem();
|
||||
Path path = new Path("/testConnectionRecords");
|
||||
|
||||
Configuration conf = new Configuration(fs.getConf());
|
||||
conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
|
||||
AzureBlobFileSystem fs2 = Mockito.spy(
|
||||
(AzureBlobFileSystem) FileSystem.newInstance(conf));
|
||||
|
||||
AzureBlobFileSystemStore store = Mockito.spy(fs2.getAbfsStore());
|
||||
Mockito.doReturn(store).when(fs2).getAbfsStore();
|
||||
|
||||
AbfsClient client = Mockito.spy(store.getClient());
|
||||
Mockito.doReturn(client).when(store).getClient();
|
||||
|
||||
try (OutputStream os = fs.create(path)) {
|
||||
os.write(1);
|
||||
}
|
||||
|
||||
InputStream is = fs2.open(path);
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
AbfsRestOperation op = Mockito.spy(
|
||||
(AbfsRestOperation) answer.callRealMethod());
|
||||
final ConnectionInfo connectionInfo = new ConnectionInfo();
|
||||
mockHttpOperationBehavior(connectionInfo, op);
|
||||
Mockito.doAnswer(executeAnswer -> {
|
||||
executeAnswer.callRealMethod();
|
||||
Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getSendBodyInvocation())
|
||||
.isEqualTo(0);
|
||||
Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
|
||||
.isEqualTo(1);
|
||||
Assertions.assertThat(connectionInfo.getReceiveResponseBodyInvocation())
|
||||
.isEqualTo(1);
|
||||
return null;
|
||||
}).when(op).execute(Mockito.any(TracingContext.class));
|
||||
return op;
|
||||
}).when(client).getAbfsRestOperation(
|
||||
Mockito.any(AbfsRestOperationType.class),
|
||||
Mockito.anyString(),
|
||||
Mockito.any(URL.class),
|
||||
Mockito.anyList(),
|
||||
Mockito.any(byte[].class),
|
||||
Mockito.anyInt(),
|
||||
Mockito.anyInt(),
|
||||
Mockito.nullable(String.class));
|
||||
|
||||
is.read();
|
||||
is.close();
|
||||
}
|
||||
|
||||
private static class ConnectionInfo {
|
||||
|
||||
private long connectTime;
|
||||
|
||||
private long readTime;
|
||||
|
||||
private long sendTime;
|
||||
|
||||
private int sendHeaderInvocation;
|
||||
|
||||
private int sendBodyInvocation;
|
||||
|
||||
private int receiveResponseInvocation;
|
||||
|
||||
private int receiveResponseBodyInvocation;
|
||||
|
||||
private void incrementSendHeaderInvocation() {
|
||||
sendHeaderInvocation++;
|
||||
}
|
||||
|
||||
private void incrementSendBodyInvocation() {
|
||||
sendBodyInvocation++;
|
||||
}
|
||||
|
||||
private void incrementReceiveResponseInvocation() {
|
||||
receiveResponseInvocation++;
|
||||
}
|
||||
|
||||
private void incrementReceiveResponseBodyInvocation() {
|
||||
receiveResponseBodyInvocation++;
|
||||
}
|
||||
|
||||
private void addConnectTime(long connectTime) {
|
||||
this.connectTime += connectTime;
|
||||
}
|
||||
|
||||
private void addReadTime(long readTime) {
|
||||
this.readTime += readTime;
|
||||
}
|
||||
|
||||
private void addSendTime(long sendTime) {
|
||||
this.sendTime += sendTime;
|
||||
}
|
||||
|
||||
private long getConnectTime() {
|
||||
return connectTime;
|
||||
}
|
||||
|
||||
private long getReadTime() {
|
||||
return readTime;
|
||||
}
|
||||
|
||||
private long getSendTime() {
|
||||
return sendTime;
|
||||
}
|
||||
|
||||
private int getSendHeaderInvocation() {
|
||||
return sendHeaderInvocation;
|
||||
}
|
||||
|
||||
private int getSendBodyInvocation() {
|
||||
return sendBodyInvocation;
|
||||
}
|
||||
|
||||
private int getReceiveResponseInvocation() {
|
||||
return receiveResponseInvocation;
|
||||
}
|
||||
|
||||
private int getReceiveResponseBodyInvocation() {
|
||||
return receiveResponseBodyInvocation;
|
||||
}
|
||||
}
|
||||
}
|
@ -23,9 +23,12 @@
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.net.URL;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@ -36,6 +39,7 @@
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
|
||||
@ -43,15 +47,36 @@
|
||||
/**
|
||||
* Test create operation.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
|
||||
|
||||
private static final int TEST_EXECUTION_TIMEOUT = 2 * 60 * 1000;
|
||||
private static final String TEST_FILE_PATH = "testfile";
|
||||
|
||||
@Parameterized.Parameter
|
||||
public HttpOperationType httpOperationType;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Iterable<Object[]> params() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{HttpOperationType.JDK_HTTP_URL_CONNECTION},
|
||||
{HttpOperationType.APACHE_HTTP_CLIENT}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
public ITestAbfsOutputStream() throws Exception {
|
||||
super();
|
||||
}
|
||||
|
||||
@Override
|
||||
public AzureBlobFileSystem getFileSystem(final Configuration configuration)
|
||||
throws Exception {
|
||||
Configuration conf = new Configuration(configuration);
|
||||
conf.set(ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY, httpOperationType.toString());
|
||||
return (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxRequestsAndQueueCapacityDefaults() throws Exception {
|
||||
Configuration conf = getRawConfiguration();
|
||||
@ -158,8 +183,7 @@ public void testAbfsOutputStreamClosingFsBeforeStream()
|
||||
public void testExpect100ContinueFailureInAppend() throws Exception {
|
||||
Configuration configuration = new Configuration(getRawConfiguration());
|
||||
configuration.set(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, "true");
|
||||
AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
|
||||
configuration);
|
||||
AzureBlobFileSystem fs = getFileSystem(configuration);
|
||||
Path path = new Path("/testFile");
|
||||
AbfsOutputStream os = Mockito.spy(
|
||||
(AbfsOutputStream) fs.create(path).getWrappedStream());
|
||||
@ -175,17 +199,23 @@ public void testExpect100ContinueFailureInAppend() throws Exception {
|
||||
Assertions.assertThat(httpOpForAppendTest[0].getConnectionDisconnectedOnError())
|
||||
.describedAs("First try from AbfsClient will have expect-100 "
|
||||
+ "header and should fail with expect-100 error.").isTrue();
|
||||
Mockito.verify(httpOpForAppendTest[0], Mockito.times(0))
|
||||
.processConnHeadersAndInputStreams(Mockito.any(byte[].class),
|
||||
Mockito.anyInt(), Mockito.anyInt());
|
||||
if (httpOpForAppendTest[0] instanceof AbfsJdkHttpOperation) {
|
||||
Mockito.verify((AbfsJdkHttpOperation) httpOpForAppendTest[0],
|
||||
Mockito.times(0))
|
||||
.processConnHeadersAndInputStreams(Mockito.any(byte[].class),
|
||||
Mockito.anyInt(), Mockito.anyInt());
|
||||
}
|
||||
|
||||
Assertions.assertThat(httpOpForAppendTest[1].getConnectionDisconnectedOnError())
|
||||
.describedAs("The retried operation from AbfsClient should not "
|
||||
+ "fail with expect-100 error. The retried operation does not have"
|
||||
+ "expect-100 header.").isFalse();
|
||||
Mockito.verify(httpOpForAppendTest[1], Mockito.times(1))
|
||||
.processConnHeadersAndInputStreams(Mockito.any(byte[].class),
|
||||
Mockito.anyInt(), Mockito.anyInt());
|
||||
if (httpOpForAppendTest[1] instanceof AbfsJdkHttpOperation) {
|
||||
Mockito.verify((AbfsJdkHttpOperation) httpOpForAppendTest[1],
|
||||
Mockito.times(1))
|
||||
.processConnHeadersAndInputStreams(Mockito.any(byte[].class),
|
||||
Mockito.anyInt(), Mockito.anyInt());
|
||||
}
|
||||
}
|
||||
|
||||
private void mockSetupForAppend(final AbfsHttpOperation[] httpOpForAppendTest,
|
||||
|
@ -20,7 +20,6 @@
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.ProtocolException;
|
||||
import java.net.URL;
|
||||
import java.util.Arrays;
|
||||
@ -34,16 +33,21 @@
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
import org.apache.http.HttpResponse;
|
||||
|
||||
import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
|
||||
import static java.net.HttpURLConnection.HTTP_OK;
|
||||
@ -53,17 +57,16 @@
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CONNECTION_TIMEOUT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_READ_TIMEOUT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
|
||||
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.times;
|
||||
|
||||
@ -98,6 +101,9 @@ public enum ErrorType {OUTPUTSTREAM, WRITE};
|
||||
@Parameterized.Parameter(3)
|
||||
public ErrorType errorType;
|
||||
|
||||
@Parameterized.Parameter(4)
|
||||
public HttpOperationType httpOperationType;
|
||||
|
||||
// The intercept.
|
||||
private AbfsThrottlingIntercept intercept;
|
||||
|
||||
@ -108,15 +114,26 @@ public enum ErrorType {OUTPUTSTREAM, WRITE};
|
||||
HTTP_EXPECTATION_FAILED = 417,
|
||||
HTTP_ERROR = 0.
|
||||
*/
|
||||
@Parameterized.Parameters(name = "expect={0}-code={1}-ErrorType={3}")
|
||||
@Parameterized.Parameters(name = "expect={0}-code={1}-ErrorType={3}=NetLib={4}")
|
||||
public static Iterable<Object[]> params() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{true, HTTP_OK, "OK", ErrorType.WRITE},
|
||||
{false, HTTP_OK, "OK", ErrorType.WRITE},
|
||||
{true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM},
|
||||
{true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM},
|
||||
{true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM},
|
||||
{true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM}
|
||||
{true, HTTP_OK, "OK", ErrorType.WRITE, JDK_HTTP_URL_CONNECTION},
|
||||
{true, HTTP_OK, "OK", ErrorType.WRITE, APACHE_HTTP_CLIENT},
|
||||
|
||||
{false, HTTP_OK, "OK", ErrorType.WRITE, JDK_HTTP_URL_CONNECTION},
|
||||
{false, HTTP_OK, "OK", ErrorType.WRITE, APACHE_HTTP_CLIENT},
|
||||
|
||||
{true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
|
||||
{true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT},
|
||||
|
||||
{true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
|
||||
{true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT},
|
||||
|
||||
{true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
|
||||
{true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT},
|
||||
|
||||
{true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
|
||||
{true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT}
|
||||
});
|
||||
}
|
||||
|
||||
@ -135,15 +152,23 @@ private byte[] getRandomBytesArray(int length) {
|
||||
return b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AzureBlobFileSystem getFileSystem(final Configuration configuration)
|
||||
throws Exception {
|
||||
Configuration conf = new Configuration(configuration);
|
||||
conf.set(ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY, httpOperationType.toString());
|
||||
return (AzureBlobFileSystem) FileSystem.newInstance(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gives the AbfsRestOperation.
|
||||
* @return abfsRestOperation.
|
||||
*/
|
||||
private AbfsRestOperation getRestOperation() throws Exception {
|
||||
// Get the filesystem.
|
||||
final AzureBlobFileSystem fs = getFileSystem();
|
||||
final AzureBlobFileSystem fs = getFileSystem(getRawConfiguration());
|
||||
|
||||
final Configuration configuration = new Configuration();
|
||||
final Configuration configuration = fs.getConf();
|
||||
configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
||||
AbfsClient abfsClient = fs.getAbfsStore().getClient();
|
||||
|
||||
@ -196,36 +221,38 @@ private AbfsRestOperation getRestOperation() throws Exception {
|
||||
URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString());
|
||||
|
||||
// Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation.
|
||||
AbfsRestOperation op = Mockito.spy(new AbfsRestOperation(
|
||||
final AbfsRestOperation op = Mockito.spy(new AbfsRestOperation(
|
||||
AbfsRestOperationType.Append,
|
||||
testClient,
|
||||
HTTP_METHOD_PUT,
|
||||
url,
|
||||
requestHeaders, buffer,
|
||||
appendRequestParameters.getoffset(),
|
||||
appendRequestParameters.getLength(), null));
|
||||
appendRequestParameters.getLength(), null, abfsConfig));
|
||||
|
||||
AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders,
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT));
|
||||
Mockito.doAnswer(answer -> {
|
||||
AbfsHttpOperation httpOperation = Mockito.spy(
|
||||
(AbfsHttpOperation) answer.callRealMethod());
|
||||
mockHttpOperation(appendRequestParameters, buffer, url, httpOperation);
|
||||
Mockito.doReturn(httpOperation).when(op).getResult();
|
||||
return httpOperation;
|
||||
}).when(op).createHttpOperation();
|
||||
return op;
|
||||
}
|
||||
|
||||
private void mockHttpOperation(final AppendRequestParameters appendRequestParameters,
|
||||
final byte[] buffer,
|
||||
final URL url,
|
||||
final AbfsHttpOperation httpOperation) throws IOException {
|
||||
// Sets the expect request property if expect header is enabled.
|
||||
if (expectHeaderEnabled) {
|
||||
Mockito.doReturn(HUNDRED_CONTINUE)
|
||||
.when(abfsHttpOperation)
|
||||
.when(httpOperation)
|
||||
.getConnProperty(EXPECT);
|
||||
}
|
||||
|
||||
HttpURLConnection urlConnection = mock(HttpURLConnection.class);
|
||||
Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito
|
||||
Mockito.doNothing().when(httpOperation).setRequestProperty(Mockito
|
||||
.any(), Mockito.any());
|
||||
Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod();
|
||||
Mockito.doReturn(url).when(urlConnection).getURL();
|
||||
Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection();
|
||||
|
||||
Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito
|
||||
.any(), Mockito.any());
|
||||
Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl();
|
||||
Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod();
|
||||
|
||||
switch (errorType) {
|
||||
case OUTPUTSTREAM:
|
||||
@ -233,28 +260,51 @@ private AbfsRestOperation getRestOperation() throws Exception {
|
||||
// enabled, it returns back to processResponse and hence we have
|
||||
// mocked the response code and the response message to check different
|
||||
// behaviour based on response code.
|
||||
Mockito.doReturn(responseCode).when(abfsHttpOperation).getConnResponseCode();
|
||||
|
||||
Mockito.doReturn(responseCode).when(httpOperation).getStatusCode();
|
||||
if (responseCode == HTTP_UNAVAILABLE) {
|
||||
Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())
|
||||
.when(abfsHttpOperation)
|
||||
.when(httpOperation)
|
||||
.getStorageErrorMessage();
|
||||
}
|
||||
Mockito.doReturn(responseMessage)
|
||||
.when(abfsHttpOperation)
|
||||
.when(httpOperation)
|
||||
.getConnResponseMessage();
|
||||
Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
|
||||
.when(abfsHttpOperation)
|
||||
.getConnOutputStream();
|
||||
if (httpOperation instanceof AbfsJdkHttpOperation) {
|
||||
Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
|
||||
.when((AbfsJdkHttpOperation) httpOperation)
|
||||
.getConnOutputStream();
|
||||
}
|
||||
if (httpOperation instanceof AbfsAHCHttpOperation) {
|
||||
Mockito.doNothing()
|
||||
.when((AbfsAHCHttpOperation) httpOperation)
|
||||
.parseResponseHeaderAndBody(Mockito.any(byte[].class),
|
||||
Mockito.anyInt(), Mockito.anyInt());
|
||||
Mockito.doReturn(HTTP_NOT_FOUND)
|
||||
.when((AbfsAHCHttpOperation) httpOperation)
|
||||
.parseStatusCode(Mockito.nullable(
|
||||
HttpResponse.class));
|
||||
Mockito.doThrow(
|
||||
new AbfsApacheHttpExpect100Exception(Mockito.mock(HttpResponse.class)))
|
||||
.when((AbfsAHCHttpOperation) httpOperation).executeRequest();
|
||||
}
|
||||
break;
|
||||
case WRITE:
|
||||
// If write() throws IOException and Expect Header is
|
||||
// enabled or not, it should throw back the exception.
|
||||
if (httpOperation instanceof AbfsAHCHttpOperation) {
|
||||
Mockito.doThrow(new IOException())
|
||||
.when((AbfsAHCHttpOperation) httpOperation).executeRequest();
|
||||
return;
|
||||
}
|
||||
OutputStream outputStream = Mockito.spy(new OutputStream() {
|
||||
@Override
|
||||
public void write(final int i) throws IOException {
|
||||
}
|
||||
});
|
||||
Mockito.doReturn(outputStream).when(abfsHttpOperation).getConnOutputStream();
|
||||
Mockito.doReturn(outputStream)
|
||||
.when((AbfsJdkHttpOperation) httpOperation)
|
||||
.getConnOutputStream();
|
||||
Mockito.doThrow(new IOException())
|
||||
.when(outputStream)
|
||||
.write(buffer, appendRequestParameters.getoffset(),
|
||||
@ -263,12 +313,6 @@ public void write(final int i) throws IOException {
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
// Sets the httpOperation for the rest operation.
|
||||
Mockito.doReturn(abfsHttpOperation)
|
||||
.when(op)
|
||||
.createHttpOperation();
|
||||
return op;
|
||||
}
|
||||
|
||||
void assertTraceContextState(int retryCount, int assertRetryCount, int bytesSent, int assertBytesSent,
|
||||
@ -295,8 +339,6 @@ void assertTraceContextState(int retryCount, int assertRetryCount, int bytesSent
|
||||
public void testExpectHundredContinue() throws Exception {
|
||||
// Gets the AbfsRestOperation.
|
||||
AbfsRestOperation op = getRestOperation();
|
||||
AbfsHttpOperation httpOperation = op.createHttpOperation();
|
||||
|
||||
TracingContext tracingContext = Mockito.spy(new TracingContext("abcd",
|
||||
"abcde", FSOperationType.APPEND,
|
||||
TracingHeaderFormat.ALL_ID_FORMAT, null));
|
||||
@ -311,7 +353,7 @@ public void testExpectHundredContinue() throws Exception {
|
||||
() -> op.execute(tracingContext));
|
||||
|
||||
// Asserting update of metrics and retries.
|
||||
assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), BUFFER_LENGTH,
|
||||
assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, op.getResult().getBytesSent(), BUFFER_LENGTH,
|
||||
0, 0);
|
||||
break;
|
||||
case OUTPUTSTREAM:
|
||||
@ -322,8 +364,8 @@ public void testExpectHundredContinue() throws Exception {
|
||||
() -> op.execute(tracingContext));
|
||||
|
||||
// Asserting update of metrics and retries.
|
||||
assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), ZERO,
|
||||
httpOperation.getExpectedBytesToBeSent(), BUFFER_LENGTH);
|
||||
assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, op.getResult().getBytesSent(), ZERO,
|
||||
op.getResult().getExpectedBytesToBeSent(), BUFFER_LENGTH);
|
||||
|
||||
// Verifies that update Metrics call is made for throttle case and for the first without retry +
|
||||
// for the retried cases as well.
|
||||
@ -336,7 +378,7 @@ public void testExpectHundredContinue() throws Exception {
|
||||
() -> op.execute(tracingContext));
|
||||
|
||||
// Asserting update of metrics and retries.
|
||||
assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(),
|
||||
assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, op.getResult().getBytesSent(),
|
||||
ZERO, 0, 0);
|
||||
|
||||
// Verifies that update Metrics call is made for ErrorType case and for the first without retry +
|
||||
|
@ -0,0 +1,63 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.ClosedIOException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.verifyCause;
|
||||
|
||||
/**
|
||||
* This test class tests the exception handling in ABFS thrown by the
|
||||
* {@link KeepAliveCache}.
|
||||
*/
|
||||
public class ITestApacheClientConnectionPool extends
|
||||
AbstractAbfsIntegrationTest {
|
||||
|
||||
public ITestApacheClientConnectionPool() throws Exception {
|
||||
super();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKacIsClosed() throws Throwable {
|
||||
Configuration configuration = new Configuration(getRawConfiguration());
|
||||
configuration.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.name());
|
||||
try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
|
||||
configuration)) {
|
||||
KeepAliveCache kac = fs.getAbfsStore().getClient().getKeepAliveCache();
|
||||
kac.close();
|
||||
AbfsDriverException ex = intercept(AbfsDriverException.class,
|
||||
KEEP_ALIVE_CACHE_CLOSED, () -> {
|
||||
fs.create(new Path("/test"));
|
||||
});
|
||||
verifyCause(ClosedIOException.class, ex);
|
||||
}
|
||||
}
|
||||
}
|
@ -76,8 +76,8 @@ public void verifyDisablingOfTracker() throws Exception {
|
||||
|
||||
try (AbfsPerfInfo tracker = new AbfsPerfInfo(abfsPerfTracker, "disablingCaller",
|
||||
"disablingCallee")) {
|
||||
AbfsHttpOperation op = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
tracker.registerResult(op).registerSuccess(true);
|
||||
}
|
||||
|
||||
@ -95,8 +95,8 @@ public void verifyTrackingForSingletonLatencyRecords() throws Exception {
|
||||
assertThat(latencyDetails).describedAs("AbfsPerfTracker should be empty").isNull();
|
||||
|
||||
List<Callable<Integer>> tasks = new ArrayList<>();
|
||||
AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
tasks.add(() -> {
|
||||
@ -135,8 +135,8 @@ public void verifyTrackingForAggregateLatencyRecords() throws Exception {
|
||||
assertThat(latencyDetails).describedAs("AbfsPerfTracker should be empty").isNull();
|
||||
|
||||
List<Callable<Integer>> tasks = new ArrayList<>();
|
||||
AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
tasks.add(() -> {
|
||||
@ -175,8 +175,8 @@ public void verifyRecordingSingletonLatencyIsCheapWhenDisabled() throws Exceptio
|
||||
long aggregateLatency = 0;
|
||||
AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, false);
|
||||
List<Callable<Long>> tasks = new ArrayList<>();
|
||||
final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
tasks.add(() -> {
|
||||
@ -211,8 +211,8 @@ public void verifyRecordingAggregateLatencyIsCheapWhenDisabled() throws Exceptio
|
||||
long aggregateLatency = 0;
|
||||
AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, false);
|
||||
List<Callable<Long>> tasks = new ArrayList<>();
|
||||
final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
tasks.add(() -> {
|
||||
@ -276,8 +276,8 @@ public void verifyRecordingSingletonLatencyIsCheapWhenEnabled() throws Exception
|
||||
long aggregateLatency = 0;
|
||||
AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, true);
|
||||
List<Callable<Long>> tasks = new ArrayList<>();
|
||||
final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
tasks.add(() -> {
|
||||
@ -311,8 +311,8 @@ public void verifyRecordingAggregateLatencyIsCheapWhenEnabled() throws Exception
|
||||
long aggregateLatency = 0;
|
||||
AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, true);
|
||||
List<Callable<Long>> tasks = new ArrayList<>();
|
||||
final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
tasks.add(() -> {
|
||||
@ -372,8 +372,8 @@ public void verifyNoExceptionOnInvalidInput() throws Exception {
|
||||
Instant testInstant = Instant.now();
|
||||
AbfsPerfTracker abfsPerfTrackerDisabled = new AbfsPerfTracker(accountName, filesystemName, false);
|
||||
AbfsPerfTracker abfsPerfTrackerEnabled = new AbfsPerfTracker(accountName, filesystemName, true);
|
||||
final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
verifyNoException(abfsPerfTrackerDisabled);
|
||||
verifyNoException(abfsPerfTrackerEnabled);
|
||||
@ -381,8 +381,8 @@ public void verifyNoExceptionOnInvalidInput() throws Exception {
|
||||
|
||||
private void verifyNoException(AbfsPerfTracker abfsPerfTracker) throws Exception {
|
||||
Instant testInstant = Instant.now();
|
||||
final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
|
||||
DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
|
||||
final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
|
||||
Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
|
||||
|
||||
try (
|
||||
AbfsPerfInfo tracker01 = new AbfsPerfInfo(abfsPerfTracker, null, null);
|
||||
|
@ -99,14 +99,14 @@ public void testRenameFailuresDueToIncompleteMetadata() throws Exception {
|
||||
// SuccessFul Result.
|
||||
AbfsRestOperation successOp =
|
||||
new AbfsRestOperation(AbfsRestOperationType.RenamePath, mockClient,
|
||||
HTTP_METHOD_PUT, null, null);
|
||||
HTTP_METHOD_PUT, null, null, mockClient.getAbfsConfiguration());
|
||||
AbfsClientRenameResult successResult = mock(AbfsClientRenameResult.class);
|
||||
doReturn(successOp).when(successResult).getOp();
|
||||
when(successResult.isIncompleteMetadataState()).thenReturn(false);
|
||||
|
||||
// Failed Result.
|
||||
AbfsRestOperation failedOp = new AbfsRestOperation(AbfsRestOperationType.RenamePath, mockClient,
|
||||
HTTP_METHOD_PUT, null, null);
|
||||
HTTP_METHOD_PUT, null, null, mockClient.getAbfsConfiguration());
|
||||
AbfsClientRenameResult recoveredMetaDataIncompleteResult =
|
||||
mock(AbfsClientRenameResult.class);
|
||||
|
||||
@ -167,12 +167,17 @@ AbfsClient getMockAbfsClient() throws IOException {
|
||||
Mockito.doReturn(spiedConf).when(spyClient).getAbfsConfiguration();
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
AbfsRestOperation op = new AbfsRestOperation(AbfsRestOperationType.RenamePath,
|
||||
spyClient, HTTP_METHOD_PUT, answer.getArgument(0), answer.getArgument(1));
|
||||
AbfsRestOperation spiedOp = Mockito.spy(op);
|
||||
addSpyBehavior(spiedOp, op, spyClient);
|
||||
return spiedOp;
|
||||
}).when(spyClient).createRenameRestOperation(Mockito.any(URL.class), anyList());
|
||||
AbfsRestOperation op = new AbfsRestOperation(
|
||||
AbfsRestOperationType.RenamePath,
|
||||
spyClient, HTTP_METHOD_PUT, answer.getArgument(0),
|
||||
answer.getArgument(1),
|
||||
spyClient.getAbfsConfiguration());
|
||||
AbfsRestOperation spiedOp = Mockito.spy(op);
|
||||
addSpyBehavior(spiedOp, op, spyClient);
|
||||
return spiedOp;
|
||||
})
|
||||
.when(spyClient)
|
||||
.createRenameRestOperation(Mockito.any(URL.class), anyList());
|
||||
|
||||
return spyClient;
|
||||
|
||||
@ -195,7 +200,7 @@ private void addSpyBehavior(final AbfsRestOperation spiedRestOp,
|
||||
AbfsHttpOperation normalOp1 = normalRestOp.createHttpOperation();
|
||||
executeThenFail(client, normalRestOp, failingOperation, normalOp1);
|
||||
AbfsHttpOperation normalOp2 = normalRestOp.createHttpOperation();
|
||||
normalOp2.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
|
||||
normalOp2.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
|
||||
client.getAccessToken());
|
||||
|
||||
Mockito.doReturn(failingOperation).doReturn(normalOp2).when(spiedRestOp).createHttpOperation();
|
||||
@ -221,14 +226,14 @@ private void executeThenFail(final AbfsClient client,
|
||||
final int offset = answer.getArgument(1);
|
||||
final int length = answer.getArgument(2);
|
||||
normalRestOp.signRequest(normalOp, length);
|
||||
normalOp.sendRequest(buffer, offset, length);
|
||||
normalOp.sendPayload(buffer, offset, length);
|
||||
normalOp.processResponse(buffer, offset, length);
|
||||
LOG.info("Actual outcome is {} \"{}\" \"{}\"; injecting failure",
|
||||
normalOp.getStatusCode(),
|
||||
normalOp.getStorageErrorCode(),
|
||||
normalOp.getStorageErrorMessage());
|
||||
throw new SocketException("connection-reset");
|
||||
}).when(failingOperation).sendRequest(Mockito.nullable(byte[].class),
|
||||
}).when(failingOperation).sendPayload(Mockito.nullable(byte[].class),
|
||||
Mockito.nullable(int.class), Mockito.nullable(int.class));
|
||||
|
||||
}
|
||||
|
@ -24,8 +24,8 @@
|
||||
import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
|
||||
import org.junit.Test;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT;
|
||||
import static org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.DeletePath;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT;
|
||||
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
||||
import java.util.ArrayList;
|
||||
@ -58,8 +58,9 @@ public void testBackoffRetryMetrics() throws Exception {
|
||||
// Get an instance of AbfsClient and AbfsRestOperation.
|
||||
AbfsClient testClient = super.getAbfsClient(super.getAbfsStore(fs));
|
||||
AbfsRestOperation op = ITestAbfsClient.getRestOp(
|
||||
DeletePath, testClient, HTTP_METHOD_DELETE,
|
||||
ITestAbfsClient.getTestUrl(testClient, "/NonExistingPath"), ITestAbfsClient.getTestRequestHeaders(testClient));
|
||||
DeletePath, testClient, HTTP_METHOD_DELETE,
|
||||
ITestAbfsClient.getTestUrl(testClient, "/NonExistingPath"),
|
||||
ITestAbfsClient.getTestRequestHeaders(testClient), getConfiguration());
|
||||
|
||||
// Mock retry counts and status code.
|
||||
ArrayList<String> retryCounts = new ArrayList<>(Arrays.asList("35", "28", "31", "45", "10", "2", "9"));
|
||||
|
@ -28,6 +28,7 @@
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.stubbing.Stubber;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
|
||||
@ -207,7 +208,8 @@ public void testRetryPolicyWithDifferentFailureReasons() throws Exception {
|
||||
abfsClient,
|
||||
"PUT",
|
||||
null,
|
||||
new ArrayList<>()
|
||||
new ArrayList<>(),
|
||||
Mockito.mock(AbfsConfiguration.class)
|
||||
));
|
||||
|
||||
AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
|
||||
@ -224,6 +226,8 @@ public void testRetryPolicyWithDifferentFailureReasons() throws Exception {
|
||||
Mockito.doReturn("").when(httpOperation).getStorageErrorMessage();
|
||||
Mockito.doReturn("").when(httpOperation).getStorageErrorCode();
|
||||
Mockito.doReturn("HEAD").when(httpOperation).getMethod();
|
||||
Mockito.doReturn("").when(httpOperation).getMaskedUrl();
|
||||
Mockito.doReturn("").when(httpOperation).getRequestId();
|
||||
Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage()).when(httpOperation).getStorageErrorMessage();
|
||||
Mockito.doReturn(tracingContext).when(abfsRestOperation).createNewTracingContext(any());
|
||||
|
||||
@ -270,7 +274,8 @@ public void testRetryPolicyWithDifferentFailureReasons() throws Exception {
|
||||
|
||||
// Assert that intercept.updateMetrics was called 2 times. Both the retried request fails with EGR.
|
||||
Mockito.verify(intercept, Mockito.times(2))
|
||||
.updateMetrics(nullable(AbfsRestOperationType.class), nullable(AbfsHttpOperation.class));
|
||||
.updateMetrics(nullable(AbfsRestOperationType.class), nullable(
|
||||
AbfsHttpOperation.class));
|
||||
}
|
||||
|
||||
private void testClientRequestIdForStatusRetry(int status,
|
||||
@ -292,7 +297,8 @@ private void testClientRequestIdForStatusRetry(int status,
|
||||
abfsClient,
|
||||
"PUT",
|
||||
null,
|
||||
new ArrayList<>()
|
||||
new ArrayList<>(),
|
||||
Mockito.mock(AbfsConfiguration.class)
|
||||
));
|
||||
|
||||
AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
|
||||
@ -357,7 +363,8 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions,
|
||||
abfsClient,
|
||||
"PUT",
|
||||
null,
|
||||
new ArrayList<>()
|
||||
new ArrayList<>(),
|
||||
Mockito.mock(AbfsConfiguration.class)
|
||||
));
|
||||
|
||||
AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
|
||||
|
@ -0,0 +1,290 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.ClosedIOException;
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsTestWithTimeout;
|
||||
|
||||
import org.apache.http.HttpClientConnection;
|
||||
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_MAX_CONN_SYS_PROP;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.HUNDRED;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
|
||||
public class TestApacheClientConnectionPool extends
|
||||
AbstractAbfsTestWithTimeout {
|
||||
|
||||
public TestApacheClientConnectionPool() throws Exception {
|
||||
super();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getTestTimeoutMillis() {
|
||||
return (int) DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME * 4;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBasicPool() throws Exception {
|
||||
System.clearProperty(HTTP_MAX_CONN_SYS_PROP);
|
||||
validatePoolSize(DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSysPropAppliedPool() throws Exception {
|
||||
final String customPoolSize = "10";
|
||||
System.setProperty(HTTP_MAX_CONN_SYS_PROP, customPoolSize);
|
||||
validatePoolSize(Integer.parseInt(customPoolSize));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPoolWithZeroSysProp() throws Exception {
|
||||
final String customPoolSize = "0";
|
||||
System.setProperty(HTTP_MAX_CONN_SYS_PROP, customPoolSize);
|
||||
validatePoolSize(DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptySizePool() throws Exception {
|
||||
Configuration configuration = new Configuration();
|
||||
configuration.set(FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE,
|
||||
"0");
|
||||
AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
|
||||
EMPTY_STRING);
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
abfsConfiguration)) {
|
||||
assertCachePutFail(keepAliveCache,
|
||||
Mockito.mock(HttpClientConnection.class));
|
||||
assertCacheGetIsNull(keepAliveCache);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertCacheGetIsNull(final KeepAliveCache keepAliveCache)
|
||||
throws IOException {
|
||||
Assertions.assertThat(keepAliveCache.get())
|
||||
.describedAs("cache.get()")
|
||||
.isNull();
|
||||
}
|
||||
|
||||
private void assertCacheGetIsNonNull(final KeepAliveCache keepAliveCache)
|
||||
throws IOException {
|
||||
Assertions.assertThat(keepAliveCache.get())
|
||||
.describedAs("cache.get()")
|
||||
.isNotNull();
|
||||
}
|
||||
|
||||
private void assertCachePutFail(final KeepAliveCache keepAliveCache,
|
||||
final HttpClientConnection mock) {
|
||||
Assertions.assertThat(keepAliveCache.put(mock))
|
||||
.describedAs("cache.put()")
|
||||
.isFalse();
|
||||
}
|
||||
|
||||
private void assertCachePutSuccess(final KeepAliveCache keepAliveCache,
|
||||
final HttpClientConnection connections) {
|
||||
Assertions.assertThat(keepAliveCache.put(connections))
|
||||
.describedAs("cache.put()")
|
||||
.isTrue();
|
||||
}
|
||||
|
||||
private void validatePoolSize(int size) throws Exception {
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
|
||||
keepAliveCache.clear();
|
||||
final HttpClientConnection[] connections = new HttpClientConnection[size
|
||||
* 2];
|
||||
|
||||
for (int i = 0; i < size * 2; i++) {
|
||||
connections[i] = Mockito.mock(HttpClientConnection.class);
|
||||
}
|
||||
|
||||
for (int i = 0; i < size; i++) {
|
||||
assertCachePutSuccess(keepAliveCache, connections[i]);
|
||||
Mockito.verify(connections[i], Mockito.times(0)).close();
|
||||
}
|
||||
|
||||
for (int i = size; i < size * 2; i++) {
|
||||
assertCachePutSuccess(keepAliveCache, connections[i]);
|
||||
Mockito.verify(connections[i - size], Mockito.times(1)).close();
|
||||
}
|
||||
|
||||
for (int i = 0; i < size * 2; i++) {
|
||||
if (i < size) {
|
||||
assertCacheGetIsNonNull(keepAliveCache);
|
||||
} else {
|
||||
assertCacheGetIsNull(keepAliveCache);
|
||||
}
|
||||
}
|
||||
System.clearProperty(HTTP_MAX_CONN_SYS_PROP);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepAliveCache() throws Exception {
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
|
||||
keepAliveCache.clear();
|
||||
HttpClientConnection connection = Mockito.mock(
|
||||
HttpClientConnection.class);
|
||||
|
||||
keepAliveCache.put(connection);
|
||||
|
||||
assertCacheGetIsNonNull(keepAliveCache);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepAliveCacheCleanup() throws Exception {
|
||||
Configuration configuration = new Configuration();
|
||||
configuration.set(FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL,
|
||||
HUNDRED + EMPTY_STRING);
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
new AbfsConfiguration(configuration, EMPTY_STRING))) {
|
||||
keepAliveCache.clear();
|
||||
HttpClientConnection connection = Mockito.mock(
|
||||
HttpClientConnection.class);
|
||||
|
||||
|
||||
// Eviction thread would close the TTL-elapsed connection and remove it from cache.
|
||||
AtomicBoolean isConnClosed = new AtomicBoolean(false);
|
||||
Mockito.doAnswer(closeInvocation -> {
|
||||
isConnClosed.set(true);
|
||||
return null;
|
||||
}).when(connection).close();
|
||||
keepAliveCache.put(connection);
|
||||
|
||||
while (!isConnClosed.get()) {
|
||||
Thread.sleep(HUNDRED);
|
||||
}
|
||||
|
||||
// Assert that the closed connection is removed from the cache.
|
||||
assertCacheGetIsNull(keepAliveCache);
|
||||
Mockito.verify(connection, Mockito.times(1)).close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepAliveCacheCleanupWithConnections() throws Exception {
|
||||
Configuration configuration = new Configuration();
|
||||
configuration.set(FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL,
|
||||
HUNDRED + EMPTY_STRING);
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
new AbfsConfiguration(configuration, EMPTY_STRING))) {
|
||||
keepAliveCache.pauseThread();
|
||||
keepAliveCache.clear();
|
||||
HttpClientConnection connection = Mockito.mock(
|
||||
HttpClientConnection.class);
|
||||
keepAliveCache.put(connection);
|
||||
|
||||
Thread.sleep(2 * keepAliveCache.getConnectionIdleTTL());
|
||||
/*
|
||||
* Eviction thread is switched off, the get() on the cache would close and
|
||||
* remove the TTL-elapsed connection.
|
||||
*/
|
||||
Mockito.verify(connection, Mockito.times(0)).close();
|
||||
assertCacheGetIsNull(keepAliveCache);
|
||||
Mockito.verify(connection, Mockito.times(1)).close();
|
||||
keepAliveCache.resumeThread();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepAliveCacheConnectionRecache() throws Exception {
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
|
||||
keepAliveCache.clear();
|
||||
HttpClientConnection connection = Mockito.mock(
|
||||
HttpClientConnection.class);
|
||||
keepAliveCache.put(connection);
|
||||
|
||||
assertCacheGetIsNonNull(keepAliveCache);
|
||||
keepAliveCache.put(connection);
|
||||
assertCacheGetIsNonNull(keepAliveCache);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepAliveCacheRemoveStaleConnection() throws Exception {
|
||||
try (KeepAliveCache keepAliveCache = new KeepAliveCache(
|
||||
new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
|
||||
keepAliveCache.clear();
|
||||
HttpClientConnection[] connections = new HttpClientConnection[5];
|
||||
|
||||
// Fill up the cache.
|
||||
for (int i = 0;
|
||||
i < DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
|
||||
i++) {
|
||||
connections[i] = Mockito.mock(HttpClientConnection.class);
|
||||
keepAliveCache.put(connections[i]);
|
||||
}
|
||||
|
||||
// Mark all but the last two connections as stale.
|
||||
for (int i = 0;
|
||||
i < DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS - 2;
|
||||
i++) {
|
||||
Mockito.doReturn(true).when(connections[i]).isStale();
|
||||
}
|
||||
|
||||
// Verify that the stale connections are removed.
|
||||
for (int i = DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS - 1;
|
||||
i >= 0;
|
||||
i--) {
|
||||
// The last two connections are not stale and would be returned.
|
||||
if (i >= (DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS - 2)) {
|
||||
assertCacheGetIsNonNull(keepAliveCache);
|
||||
} else {
|
||||
// Stale connections are closed and removed.
|
||||
assertCacheGetIsNull(keepAliveCache);
|
||||
Mockito.verify(connections[i], Mockito.times(1)).close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKeepAliveCacheClosed() throws Exception {
|
||||
KeepAliveCache keepAliveCache = Mockito.spy(new KeepAliveCache(
|
||||
new AbfsConfiguration(new Configuration(), EMPTY_STRING)));
|
||||
keepAliveCache.put(Mockito.mock(HttpClientConnection.class));
|
||||
keepAliveCache.close();
|
||||
intercept(ClosedIOException.class,
|
||||
KEEP_ALIVE_CACHE_CLOSED,
|
||||
() -> keepAliveCache.get());
|
||||
|
||||
HttpClientConnection conn = Mockito.mock(HttpClientConnection.class);
|
||||
assertCachePutFail(keepAliveCache, conn);
|
||||
Mockito.verify(conn, Mockito.times(1)).close();
|
||||
keepAliveCache.close();
|
||||
Mockito.verify(keepAliveCache, Mockito.times(1)).closeInternal();
|
||||
}
|
||||
}
|
@ -0,0 +1,226 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.azurebfs.services;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
||||
import org.apache.hadoop.fs.azurebfs.AbstractAbfsTestWithTimeout;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
||||
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
||||
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
||||
|
||||
import static java.net.HttpURLConnection.HTTP_OK;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APACHE_IMPL;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_FALLBACK;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_IMPL;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES;
|
||||
import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
|
||||
|
||||
public class TestApacheHttpClientFallback extends AbstractAbfsTestWithTimeout {
|
||||
|
||||
public TestApacheHttpClientFallback() throws Exception {
|
||||
super();
|
||||
}
|
||||
|
||||
private TracingContext getSampleTracingContext(int[] jdkCallsRegister,
|
||||
int[] apacheCallsRegister) {
|
||||
String correlationId, fsId;
|
||||
TracingHeaderFormat format;
|
||||
correlationId = "test-corr-id";
|
||||
fsId = "test-filesystem-id";
|
||||
format = TracingHeaderFormat.ALL_ID_FORMAT;
|
||||
TracingContext tc = Mockito.spy(new TracingContext(correlationId, fsId,
|
||||
FSOperationType.TEST_OP, true, format, null));
|
||||
Mockito.doAnswer(answer -> {
|
||||
answer.callRealMethod();
|
||||
AbfsHttpOperation op = answer.getArgument(0);
|
||||
if (op instanceof AbfsAHCHttpOperation) {
|
||||
Assertions.assertThat(tc.getHeader()).endsWith(APACHE_IMPL);
|
||||
apacheCallsRegister[0]++;
|
||||
}
|
||||
if (op instanceof AbfsJdkHttpOperation) {
|
||||
jdkCallsRegister[0]++;
|
||||
if (AbfsApacheHttpClient.usable()) {
|
||||
Assertions.assertThat(tc.getHeader()).endsWith(JDK_IMPL);
|
||||
} else {
|
||||
Assertions.assertThat(tc.getHeader()).endsWith(JDK_FALLBACK);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
})
|
||||
.when(tc)
|
||||
.constructHeader(Mockito.any(AbfsHttpOperation.class),
|
||||
Mockito.nullable(String.class), Mockito.nullable(String.class));
|
||||
return tc;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleFailureLeadToFallback()
|
||||
throws Exception {
|
||||
int[] apacheCallsTest1 = {0};
|
||||
int[] jdkCallsTest1 = {0};
|
||||
TracingContext tcTest1 = getSampleTracingContext(jdkCallsTest1,
|
||||
apacheCallsTest1);
|
||||
int[] retryIterationTest1 = {0};
|
||||
intercept(IOException.class, () -> {
|
||||
getMockRestOperation(retryIterationTest1).execute(tcTest1);
|
||||
});
|
||||
Assertions.assertThat(apacheCallsTest1[0])
|
||||
.isEqualTo(DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES);
|
||||
Assertions.assertThat(jdkCallsTest1[0]).isEqualTo(1);
|
||||
|
||||
int[] retryIteration1 = {0};
|
||||
int[] apacheCallsTest2 = {0};
|
||||
int[] jdkCallsTest2 = {0};
|
||||
TracingContext tcTest2 = getSampleTracingContext(jdkCallsTest2,
|
||||
apacheCallsTest2);
|
||||
intercept(IOException.class, () -> {
|
||||
getMockRestOperation(retryIteration1).execute(tcTest2);
|
||||
});
|
||||
Assertions.assertThat(apacheCallsTest2[0]).isEqualTo(0);
|
||||
Assertions.assertThat(jdkCallsTest2[0])
|
||||
.isEqualTo(DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES + 1);
|
||||
}
|
||||
|
||||
private AbfsRestOperation getMockRestOperation(int[] retryIteration)
|
||||
throws IOException {
|
||||
AbfsConfiguration configuration = Mockito.mock(AbfsConfiguration.class);
|
||||
Mockito.doReturn(APACHE_HTTP_CLIENT)
|
||||
.when(configuration)
|
||||
.getPreferredHttpOperationType();
|
||||
Mockito.doReturn(DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES)
|
||||
.when(configuration)
|
||||
.getMaxApacheHttpClientIoExceptionsRetries();
|
||||
AbfsClient client = Mockito.mock(AbfsClient.class);
|
||||
Mockito.doReturn(Mockito.mock(ExponentialRetryPolicy.class))
|
||||
.when(client)
|
||||
.getExponentialRetryPolicy();
|
||||
|
||||
AbfsRetryPolicy retryPolicy = Mockito.mock(AbfsRetryPolicy.class);
|
||||
Mockito.doReturn(retryPolicy)
|
||||
.when(client)
|
||||
.getRetryPolicy(Mockito.nullable(String.class));
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
if (retryIteration[0]
|
||||
< DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES) {
|
||||
retryIteration[0]++;
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
})
|
||||
.when(retryPolicy)
|
||||
.shouldRetry(Mockito.anyInt(), Mockito.nullable(Integer.class));
|
||||
|
||||
AbfsThrottlingIntercept abfsThrottlingIntercept = Mockito.mock(
|
||||
AbfsThrottlingIntercept.class);
|
||||
Mockito.doNothing()
|
||||
.when(abfsThrottlingIntercept)
|
||||
.updateMetrics(Mockito.any(AbfsRestOperationType.class),
|
||||
Mockito.any(AbfsHttpOperation.class));
|
||||
Mockito.doNothing()
|
||||
.when(abfsThrottlingIntercept)
|
||||
.sendingRequest(Mockito.any(AbfsRestOperationType.class),
|
||||
Mockito.nullable(AbfsCounters.class));
|
||||
Mockito.doReturn(abfsThrottlingIntercept).when(client).getIntercept();
|
||||
|
||||
|
||||
AbfsRestOperation op = Mockito.spy(new AbfsRestOperation(
|
||||
AbfsRestOperationType.ReadFile,
|
||||
client,
|
||||
AbfsHttpConstants.HTTP_METHOD_GET,
|
||||
new URL("http://localhost"),
|
||||
new ArrayList<>(),
|
||||
null,
|
||||
configuration
|
||||
));
|
||||
|
||||
Mockito.doReturn(null).when(op).getClientLatency();
|
||||
|
||||
Mockito.doReturn(createApacheHttpOp())
|
||||
.when(op)
|
||||
.createAbfsHttpOperation();
|
||||
Mockito.doReturn(createAhcHttpOp())
|
||||
.when(op)
|
||||
.createAbfsAHCHttpOperation();
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
return answer.getArgument(0);
|
||||
}).when(op).createNewTracingContext(Mockito.nullable(TracingContext.class));
|
||||
|
||||
Mockito.doNothing()
|
||||
.when(op)
|
||||
.signRequest(Mockito.any(AbfsHttpOperation.class), Mockito.anyInt());
|
||||
|
||||
Mockito.doAnswer(answer -> {
|
||||
AbfsHttpOperation operation = Mockito.spy(
|
||||
(AbfsHttpOperation) answer.callRealMethod());
|
||||
Assertions.assertThat(operation).isInstanceOf(
|
||||
(retryIteration[0]
|
||||
< DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES
|
||||
&& AbfsApacheHttpClient.usable())
|
||||
? AbfsAHCHttpOperation.class
|
||||
: AbfsJdkHttpOperation.class);
|
||||
Mockito.doReturn(HTTP_OK).when(operation).getStatusCode();
|
||||
Mockito.doThrow(new IOException("Test Exception"))
|
||||
.when(operation)
|
||||
.processResponse(Mockito.nullable(byte[].class), Mockito.anyInt(),
|
||||
Mockito.anyInt());
|
||||
Mockito.doCallRealMethod().when(operation).getTracingContextSuffix();
|
||||
return operation;
|
||||
}).when(op).createHttpOperation();
|
||||
return op;
|
||||
}
|
||||
|
||||
private AbfsAHCHttpOperation createAhcHttpOp() {
|
||||
AbfsAHCHttpOperation ahcOp = Mockito.mock(AbfsAHCHttpOperation.class);
|
||||
Mockito.doCallRealMethod().when(ahcOp).getTracingContextSuffix();
|
||||
return ahcOp;
|
||||
}
|
||||
|
||||
private AbfsJdkHttpOperation createApacheHttpOp() {
|
||||
AbfsJdkHttpOperation httpOperationMock = Mockito.mock(AbfsJdkHttpOperation.class);
|
||||
Mockito.doCallRealMethod()
|
||||
.when(httpOperationMock)
|
||||
.getTracingContextSuffix();
|
||||
return httpOperationMock;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTcHeaderOnJDKClientUse() {
|
||||
int[] jdkCallsRegister = {0};
|
||||
int[] apacheCallsRegister = {0};
|
||||
TracingContext tc = getSampleTracingContext(jdkCallsRegister,
|
||||
apacheCallsRegister);
|
||||
AbfsJdkHttpOperation op = Mockito.mock(AbfsJdkHttpOperation.class);
|
||||
Mockito.doCallRealMethod().when(op).getTracingContextSuffix();
|
||||
tc.constructHeader(op, null, null);
|
||||
}
|
||||
}
|
@ -93,7 +93,7 @@ private void validateTracingHeader(String tracingContextHeader) {
|
||||
private void validateBasicFormat(String[] idList) {
|
||||
if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
|
||||
Assertions.assertThat(idList)
|
||||
.describedAs("header should have 7 elements").hasSize(7);
|
||||
.describedAs("header should have 8 elements").hasSize(8);
|
||||
} else if (format == TracingHeaderFormat.TWO_ID_FORMAT) {
|
||||
Assertions.assertThat(idList)
|
||||
.describedAs("header should have 2 elements").hasSize(2);
|
||||
|
Loading…
Reference in New Issue
Block a user