dannycjones commented on code in PR #4706: URL: https://github.com/apache/hadoop/pull/4706#discussion_r951368873
########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.s3a; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.VersionInfo; +import org.apache.http.client.utils.URIBuilder; + +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_DOMAIN; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PASSWORD; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_USERNAME; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_WORKSTATION; +import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; + +public final class AWSClientConfig { Review Comment: Add JavaDoc. at first glance, I'm thinking this is actually an SDK class so let's make clear its purpose. Or maybe rename if we can think of a better one. ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.s3a; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.VersionInfo; +import org.apache.http.client.utils.URIBuilder; + +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_DOMAIN; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PASSWORD; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_USERNAME; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_WORKSTATION; +import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; + +public final class AWSClientConfig { + private static final Logger LOG = LoggerFactory.getLogger(AWSClientConfig.class); + + private AWSClientConfig() { + } + + public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Configuration conf) { + ClientOverrideConfiguration.Builder overrideConfigBuilder = + ClientOverrideConfiguration.builder(); + + long requestTimeoutMillis = conf.getTimeDuration(REQUEST_TIMEOUT, + DEFAULT_REQUEST_TIMEOUT, TimeUnit.SECONDS, TimeUnit.MILLISECONDS); + + if (requestTimeoutMillis > Integer.MAX_VALUE) { + LOG.debug("Request timeout is too high({} ms). Setting to {} ms instead", + requestTimeoutMillis, Integer.MAX_VALUE); + requestTimeoutMillis = Integer.MAX_VALUE; + } + + if(requestTimeoutMillis > 0) { + overrideConfigBuilder.apiCallAttemptTimeout(Duration.ofMillis(requestTimeoutMillis)); + } Review Comment: Maybe we can move to a private method like `initApiCallTimeout`/`initRequestTimeout` similar to `initUserAgent` since lines 69 thru 80 are all tightly related ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java: ########## @@ -451,21 +452,21 @@ private boolean buildNextStatusBatch(S3ListResult objects) { int added = 0, ignored = 0; // list to fill in with results. Initial size will be list maximum. List<S3AFileStatus> stats = new ArrayList<>( - objects.getObjectSummaries().size() + + objects.getS3Objects().size() + objects.getCommonPrefixes().size()); // objects - for (S3ObjectSummary summary : objects.getObjectSummaries()) { - String key = summary.getKey(); + for (S3Object s3Object : objects.getS3Objects()) { + String key = s3Object.key(); Path keyPath = getStoreContext().getContextAccessors().keyToPath(key); if (LOG.isDebugEnabled()) { - LOG.debug("{}: {}", keyPath, stringify(summary)); + LOG.debug("{}: {}", keyPath, stringify(s3Object)); } // Skip over keys that are ourselves and old S3N _$folder$ files - if (acceptor.accept(keyPath, summary) && filter.accept(keyPath)) { - S3AFileStatus status = createFileStatus(keyPath, summary, + if (acceptor.accept(keyPath, s3Object) && filter.accept(keyPath)) { + S3AFileStatus status = createFileStatus(keyPath, s3Object, listingOperationCallbacks.getDefaultBlockSize(keyPath), getStoreContext().getUsername(), - summary.getETag(), null, isCSEEnabled); + s3Object.eTag(), null, isCSEEnabled); Review Comment: Fix indent while this line is modified ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java: ########## @@ -1203,4 +1203,8 @@ private Constants() { * Default maximum read size in bytes during vectored reads : {@value}. */ public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M + + public static final String HTTP = "http"; + + public static final String HTTPS = "https"; Review Comment: I'd prefer to keep these in DefaultS3ClientFactory rather than add them to Constants now, or not use them at all since they're only referred to once and I don't see much value to them at the moment. ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java: ########## @@ -160,6 +182,84 @@ public AmazonS3 createS3Client( } } + /** + * Creates a new {@link S3Client}. + * + * @param uri S3A file system URI + * @param parameters parameter object + * @return S3 client + * @throws IOException on any IO problem + */ + @Override + public S3Client createS3ClientV2( + final URI uri, + final S3ClientCreationParameters parameters) throws IOException { + + Configuration conf = getConf(); + bucket = uri.getHost(); + + final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder = + AWSClientConfig.createClientConfigBuilder(conf); + + final ApacheHttpClient.Builder httpClientBuilder = + AWSClientConfig.createHttpClientBuilder(conf); + + final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf); + + final ProxyConfiguration.Builder proxyConfigBuilder = + AWSClientConfig.createProxyConfigurationBuilder(conf, bucket); + + S3ClientBuilder s3ClientBuilder = S3Client.builder(); + + // build a s3 client with region eu-west-2 that can be used to get the region of the bucket. + S3Client defaultS3Client = S3Client.builder().region(Region.EU_WEST_2) + .credentialsProvider(V1V2AwsCredentialProviderAdapter.adapt(parameters.getCredentialSet())) + .build(); Review Comment: Move this into `getS3Region`? It's only used for determining region - we don't even need to create the client if the S3A user already specifies `fs.s3a.endpoint.region`. ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java: ########## @@ -0,0 +1,231 @@ +/* + * 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.s3a; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.VersionInfo; +import org.apache.http.client.utils.URIBuilder; + +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_DOMAIN; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PASSWORD; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_USERNAME; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_WORKSTATION; +import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; + +public final class AWSClientConfig { + private static final Logger LOG = LoggerFactory.getLogger(AWSClientConfig.class); + + private AWSClientConfig() { + } + + public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Configuration conf) { + ClientOverrideConfiguration.Builder overrideConfigBuilder = + ClientOverrideConfiguration.builder(); + + long requestTimeoutMillis = conf.getTimeDuration(REQUEST_TIMEOUT, + DEFAULT_REQUEST_TIMEOUT, TimeUnit.SECONDS, TimeUnit.MILLISECONDS); + + if (requestTimeoutMillis > Integer.MAX_VALUE) { + LOG.debug("Request timeout is too high({} ms). Setting to {} ms instead", + requestTimeoutMillis, Integer.MAX_VALUE); + requestTimeoutMillis = Integer.MAX_VALUE; + } + + if(requestTimeoutMillis > 0) { + overrideConfigBuilder.apiCallAttemptTimeout(Duration.ofMillis(requestTimeoutMillis)); + } + + initUserAgent(conf, overrideConfigBuilder); + + // TODO: Look at signers. See issue https://github.com/aws/aws-sdk-java-v2/issues/1024 + // String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, ""); + // if (!signerOverride.isEmpty()) { + // LOG.debug("Signer override = {}", signerOverride); + // overrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.SIGNER) + // } + + return overrideConfigBuilder; + } + + /** + * Configures the http client. + * + * @param conf The Hadoop configuration + * @return Http client builder + */ + public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration conf) { + ApacheHttpClient.Builder httpClientBuilder = + ApacheHttpClient.builder(); + + httpClientBuilder.maxConnections(S3AUtils.intOption(conf, MAXIMUM_CONNECTIONS, + DEFAULT_MAXIMUM_CONNECTIONS, 1)); + + httpClientBuilder.connectionTimeout(Duration.ofSeconds( + S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0))); Review Comment: Consider moving the intOption call to a temporary variable. ```suggestion int connTimeoutMillis = S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0); httpClientBuilder.connectionTimeout(Duration.ofSeconds(connTimeoutMillis)); ``` Also applicable on a few other lines, where we have nested method calls in the args. ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java: ########## @@ -563,14 +563,16 @@ public ListObjectsRequest newListObjectsV1Request( final String key, final String delimiter, final int maxKeys) { - ListObjectsRequest request = new ListObjectsRequest() - .withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); + + ListObjectsRequest.Builder requestBuilder = + ListObjectsRequest.builder().bucket(bucket).maxKeys(maxKeys).prefix(key); + if (delimiter != null) { - request.setDelimiter(delimiter); + requestBuilder.delimiter(delimiter); } - return prepareRequest(request); + + //TODO: add call to prepareRequest + return requestBuilder.build(); Review Comment: Should we add a no-op implementation of `<T extends V2Request> T prepareRequest(T t)` to return and implement later? This way, we have the correct final code here we don't have to return to and then leave a single to-do to implement which will solve all calls across the code base. ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java: ########## @@ -1183,12 +1191,26 @@ AmazonS3 getAmazonS3Client() { * @param reason a justification for requesting access. * @return AmazonS3Client */ + // TODO: Remove when we remove S3V1 client @VisibleForTesting public AmazonS3 getAmazonS3ClientForTesting(String reason) { LOG.warn("Access to S3A client requested, reason {}", reason); return s3; } + /** + * Returns the S3 client used by this filesystem. + * <i>Warning: this must only be used for testing, as it bypasses core + * S3A operations. </i> + * @param reason a justification for requesting access. + * @return S3Client + */ + @VisibleForTesting + public S3Client getAmazonS3V2ClientForTesting(String reason) { + LOG.warn("Access to S3A client requested, reason {}", reason); Review Comment: while we are here... ```suggestion LOG.warn("Access to S3 client requested, reason {}", reason); ``` ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java: ########## @@ -104,7 +104,7 @@ public interface S3ATestConstants { * Default path for the multi MB test file: {@value}. */ String DEFAULT_CSVTEST_FILE = LANDSAT_BUCKET + "scene_list.gz"; - + Review Comment: Drop this ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java: ########## @@ -34,8 +35,7 @@ public class MockS3ClientFactory implements S3ClientFactory { @Override - public AmazonS3 createS3Client(URI uri, - final S3ClientCreationParameters parameters) { + public AmazonS3 createS3Client(URI uri, final S3ClientCreationParameters parameters) { Review Comment: No need to touch this method. We can revert to reduce diff for reviewers ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java: ########## @@ -170,9 +170,11 @@ private void createFactoryObjects(RequestFactory factory) { a(factory.newGetObjectRequest(path)); a(factory.newGetObjectMetadataRequest(path)); a(factory.newListMultipartUploadsRequest(path)); - a(factory.newListObjectsV1Request(path, "/", 1)); + //TODO: Commenting out for now, new request extends AwsRequest, this can be updated once all + // request factory operations are updated. Review Comment: similar suggestion to RequestPreparer... add no-op implementation, come back later to add logic. no change needed here later. ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java: ########## @@ -160,6 +182,84 @@ public AmazonS3 createS3Client( } } + /** + * Creates a new {@link S3Client}. + * + * @param uri S3A file system URI + * @param parameters parameter object + * @return S3 client + * @throws IOException on any IO problem + */ + @Override + public S3Client createS3ClientV2( + final URI uri, + final S3ClientCreationParameters parameters) throws IOException { + + Configuration conf = getConf(); + bucket = uri.getHost(); + + final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder = + AWSClientConfig.createClientConfigBuilder(conf); + + final ApacheHttpClient.Builder httpClientBuilder = + AWSClientConfig.createHttpClientBuilder(conf); + + final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf); + + final ProxyConfiguration.Builder proxyConfigBuilder = + AWSClientConfig.createProxyConfigurationBuilder(conf, bucket); + + S3ClientBuilder s3ClientBuilder = S3Client.builder(); + + // build a s3 client with region eu-west-2 that can be used to get the region of the bucket. + S3Client defaultS3Client = S3Client.builder().region(Region.EU_WEST_2) + .credentialsProvider(V1V2AwsCredentialProviderAdapter.adapt(parameters.getCredentialSet())) + .build(); + + // add any headers + parameters.getHeaders().forEach((h, v) -> clientOverrideConfigBuilder.putHeader(h, v)); + + if (parameters.isRequesterPays()) { + // All calls must acknowledge requester will pay via header. + clientOverrideConfigBuilder.putHeader(REQUESTER_PAYS_HEADER, REQUESTER_PAYS_HEADER_VALUE); + } + + if (!StringUtils.isEmpty(parameters.getUserAgentSuffix())) { + clientOverrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_SUFFIX, + parameters.getUserAgentSuffix()); + } + + clientOverrideConfigBuilder.retryPolicy(retryPolicyBuilder.build()); + httpClientBuilder.proxyConfiguration(proxyConfigBuilder.build()); + + s3ClientBuilder.httpClientBuilder(httpClientBuilder) + .overrideConfiguration(clientOverrideConfigBuilder.build()); + + // use adapter classes so V1 credential providers continue to work. This will be moved to + // AWSCredentialProviderList.add() when that class is updated. + s3ClientBuilder.credentialsProvider( + V1V2AwsCredentialProviderAdapter.adapt(parameters.getCredentialSet())); + + URI endpoint = getS3Endpoint(parameters.getEndpoint(), conf); + + Region region = + getS3Region(conf.getTrimmed(AWS_REGION), defaultS3Client); + + LOG.debug("Using endpoint {}; and region {}", endpoint, region); + + s3ClientBuilder.endpointOverride(endpoint).region(region); + + s3ClientBuilder.serviceConfiguration( + S3Configuration.builder().pathStyleAccessEnabled(parameters.isPathStyleAccess()).build()); Review Comment: Move S3Configuration to temporary variable. There's a bunch of config in there that we may want to let users leverage in the future which could then just be one line changes. ```suggestion S3Configuration s3Configuration = S3Configuration.builder() .pathStyleAccessEnabled(parameters.isPathStyleAccess()) .build(); s3ClientBuilder.serviceConfiguration(s3Configuration); ``` ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java: ########## @@ -584,14 +586,17 @@ public ListObjectsV2Request newListObjectsV2Request( final String key, final String delimiter, final int maxKeys) { - final ListObjectsV2Request request = new ListObjectsV2Request() - .withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); + + final ListObjectsV2Request.Builder requestBuilder = + ListObjectsV2Request.builder().bucket(bucket).maxKeys(maxKeys).prefix(key); Review Comment: I'd recommend in general to keep each builder arg on a new line similar to how it was previously. ```suggestion final ListObjectsV2Request.Builder requestBuilder = ListObjectsV2Request.builder() .bucket(bucket) .maxKeys(maxKeys) .prefix(key); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
