diff --git a/LICENSE-binary b/LICENSE-binary index 917aa4751fc9f..604b745d49512 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -363,7 +363,7 @@ org.xerial.snappy:snappy-java:1.1.10.4 org.yaml:snakeyaml:2.0 org.wildfly.openssl:wildfly-openssl:2.1.4.Final ro.isdc.wro4j:wro4j-maven-plugin:1.8.0 -software.amazon.awssdk:bundle:2.25.53 +software.amazon.awssdk:bundle:2.30.27 net.jodah:failsafe:2.4.4 -------------------------------------------------------------------------------- diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index f6dc71288ed15..8b53b0599fe12 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -204,7 +204,7 @@ 1.0-beta-1 900 1.12.720 - 2.25.53 + 2.30.27 3.1.1 0.0.4 1.0.1 diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 7b686130ce30c..db1f56a60e329 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.event.Level; import software.amazon.awssdk.awscore.util.AwsHostNameUtils; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; @@ -38,6 +39,7 @@ import software.amazon.awssdk.http.auth.spi.scheme.AuthScheme; import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity; +import software.amazon.awssdk.metrics.LoggingMetricPublisher; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.s3accessgrants.plugin.S3AccessGrantsPlugin; import software.amazon.awssdk.services.s3.S3AsyncClient; @@ -53,6 +55,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.s3a.impl.MD5RequiredOperationInterceptor; import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; import org.apache.hadoop.fs.store.LogExactlyOnce; @@ -211,12 +214,25 @@ private , ClientT> Build final ClientOverrideConfiguration.Builder override = createClientOverrideConfiguration(parameters, conf); - S3BaseClientBuilder s3BaseClientBuilder = builder + S3BaseClientBuilder s3BaseClientBuilder = builder .overrideConfiguration(override.build()) .credentialsProvider(parameters.getCredentialSet()) .disableS3ExpressSessionAuth(!parameters.isExpressCreateSession()) .serviceConfiguration(serviceConfiguration); + if (LOG.isTraceEnabled()) { + // if this log is set to debug then we turn on logging of SDK metrics. + // The metrics itself will log at info; it is just that reflection work + // would be needed to change that setting safely for shaded and unshaded aws artifacts. + s3BaseClientBuilder.overrideConfiguration(o -> + o.addMetricPublisher(LoggingMetricPublisher.create())); + } + + // Force adding MD5 checksums to requests which need it now that SDK doesn't. + s3BaseClientBuilder.overrideConfiguration(o -> + override.addExecutionInterceptor(new MD5RequiredOperationInterceptor())); + + if (conf.getBoolean(HTTP_SIGNER_ENABLED, HTTP_SIGNER_ENABLED_DEFAULT)) { // use an http signer through an AuthScheme final AuthScheme signer = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java index b9a7c776b1405..796b820e79439 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.io.IOException; +import java.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +63,12 @@ public class IAMInstanceCredentialsProvider private static final Logger LOG = LoggerFactory.getLogger(IAMInstanceCredentialsProvider.class); + /** + * How far in advance of credential expiry must IAM credentials be refreshed. + * See HADOOP-19181. S3A: IAMCredentialsProvider throttling results in AWS auth failures + */ + public static final Duration TIME_BEFORE_EXPIRY = Duration.ofMinutes(1); + /** * The credentials provider. * Initially a container credentials provider, but if that fails @@ -130,8 +137,12 @@ private synchronized AwsCredentials getCredentials() { // close it to shut down any thread iamCredentialsProvider.close(); isContainerCredentialsProvider = false; + + // create an async credentials provider with a safe credential + // expiry time. iamCredentialsProvider = InstanceProfileCredentialsProvider.builder() .asyncCredentialUpdateEnabled(true) + .staleTime(TIME_BEFORE_EXPIRY) .build(); return iamCredentialsProvider.resolveCredentials(); } else { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AwsSdkWorkarounds.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AwsSdkWorkarounds.java index a0673b123b2b1..8b8ba0c16cfd0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AwsSdkWorkarounds.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AwsSdkWorkarounds.java @@ -43,8 +43,7 @@ private AwsSdkWorkarounds() { * @return true if the log tuning operation took place. */ public static boolean prepareLogging() { - return LogControllerFactory.createController(). - setLogLevel(TRANSFER_MANAGER, LogControl.LogLevel.ERROR); + return true; } /** @@ -53,7 +52,6 @@ public static boolean prepareLogging() { */ @VisibleForTesting static boolean restoreNoisyLogging() { - return LogControllerFactory.createController(). - setLogLevel(TRANSFER_MANAGER, LogControl.LogLevel.INFO); + return true; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MD5RequiredOperationInterceptor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MD5RequiredOperationInterceptor.java new file mode 100644 index 0000000000000..2ae298b4950b4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MD5RequiredOperationInterceptor.java @@ -0,0 +1,138 @@ +/* + * 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. + */ +/* + * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/apache2.0 + * + * or in the "license" file accompanying this file. This file 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.impl; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Optional; + +import software.amazon.awssdk.checksums.DefaultChecksumAlgorithm; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.core.checksums.ChecksumSpecs; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.core.interceptor.SdkExecutionAttribute; +import software.amazon.awssdk.core.interceptor.SdkInternalExecutionAttribute; +import software.amazon.awssdk.core.interceptor.trait.HttpChecksum; +import software.amazon.awssdk.core.internal.util.HttpChecksumUtils; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.http.Header; +import software.amazon.awssdk.http.SdkHttpRequest; +import software.amazon.awssdk.utils.Md5Utils; + + +/** + * Taken from AWS engineer discussion on how to address incompatible changes + * in SDKs. + * @see https://github.com/aws/aws-sdk-java-v2/discussions/5802 + */ + +public final class MD5RequiredOperationInterceptor implements ExecutionInterceptor { + + @Override + public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, + ExecutionAttributes executionAttributes) { + boolean isHttpChecksumRequired = isHttpChecksumRequired(executionAttributes); + boolean requestAlreadyHasMd5 = + context.httpRequest().firstMatchingHeader(Header.CONTENT_MD5).isPresent(); + + Optional syncContent = context.requestBody(); + Optional asyncContent = context.asyncRequestBody(); + + if (!isHttpChecksumRequired || requestAlreadyHasMd5) { + return context.httpRequest(); + } + + if (asyncContent.isPresent()) { + throw new IllegalStateException("This operation requires a content-MD5 checksum, " + + "but one cannot be calculated for non-blocking content."); + } + + if (syncContent.isPresent()) { + try { + String payloadMd5 = + Md5Utils.md5AsBase64(syncContent.get().contentStreamProvider().newStream()); + return context.httpRequest().copy(r -> r.putHeader(Header.CONTENT_MD5, payloadMd5)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + return context.httpRequest(); + } + + private boolean isHttpChecksumRequired(ExecutionAttributes executionAttributes) { + return executionAttributes.getAttribute(SdkInternalExecutionAttribute.HTTP_CHECKSUM_REQUIRED) + != null + || isMd5ChecksumRequired(executionAttributes); + } + + public static boolean isMd5ChecksumRequired(ExecutionAttributes executionAttributes) { + ChecksumSpecs resolvedChecksumSpecs = getResolvedChecksumSpecs(executionAttributes); + if (resolvedChecksumSpecs == null) { + return false; + } else { + return resolvedChecksumSpecs.algorithm() == null + && resolvedChecksumSpecs.isRequestChecksumRequired(); + } + } + + public static ChecksumSpecs getResolvedChecksumSpecs(ExecutionAttributes executionAttributes) { + ChecksumSpecs checksumSpecs = + executionAttributes.getAttribute(SdkExecutionAttribute.RESOLVED_CHECKSUM_SPECS); + return checksumSpecs != null ? checksumSpecs : resolveChecksumSpecs(executionAttributes); + } + + public static ChecksumSpecs resolveChecksumSpecs(ExecutionAttributes executionAttributes) { + HttpChecksum httpChecksumTraitInOperation = + executionAttributes.getAttribute(SdkInternalExecutionAttribute.HTTP_CHECKSUM); + if (httpChecksumTraitInOperation == null) { + return null; + } else { + boolean hasRequestValidation = httpChecksumTraitInOperation.requestValidationMode() != null; + String requestAlgorithm = httpChecksumTraitInOperation.requestAlgorithm(); + String checksumHeaderName = + requestAlgorithm != null ? HttpChecksumUtils.httpChecksumHeader(requestAlgorithm) : null; + return ChecksumSpecs.builder() + .algorithmV2(DefaultChecksumAlgorithm.fromValue(requestAlgorithm)) + .headerName(checksumHeaderName) + .responseValidationAlgorithmsV2(httpChecksumTraitInOperation.responseAlgorithmsV2()) + .isValidationEnabled(hasRequestValidation) + .isRequestChecksumRequired(httpChecksumTraitInOperation.isRequestChecksumRequired()) + .isRequestStreaming(httpChecksumTraitInOperation.isRequestStreaming()) + .requestAlgorithmHeader(httpChecksumTraitInOperation.requestAlgorithmHeader()) + .build(); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index 55ebf7614c17f..2b39f0572123a 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -1359,6 +1359,48 @@ execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be ``` + +To log the output of the AWS SDK metrics, set the log +`org.apache.hadoop.fs.s3a.DefaultS3ClientFactory` to `TRACE`. +This will then turn on logging of the internal SDK metrics.4 + +These will actually be logged at INFO in the log +``` +software.amazon.awssdk.metrics.LoggingMetricPublisher +``` + +```text +INFO metrics.LoggingMetricPublisher (LoggerAdapter.java:info(165)) - Metrics published: +MetricCollection(name=ApiCall, metrics=[ +MetricRecord(metric=MarshallingDuration, value=PT0.000092041S), +MetricRecord(metric=RetryCount, value=0), +MetricRecord(metric=ApiCallSuccessful, value=true), +MetricRecord(metric=OperationName, value=DeleteObject), +MetricRecord(metric=EndpointResolveDuration, value=PT0.000132792S), +MetricRecord(metric=ApiCallDuration, value=PT0.064890875S), +MetricRecord(metric=CredentialsFetchDuration, value=PT0.000017458S), +MetricRecord(metric=ServiceEndpoint, value=https://buckets3.eu-west-2.amazonaws.com), +MetricRecord(metric=ServiceId, value=S3)], children=[ +MetricCollection(name=ApiCallAttempt, metrics=[ + MetricRecord(metric=TimeToFirstByte, value=PT0.06260225S), + MetricRecord(metric=SigningDuration, value=PT0.000293083S), + MetricRecord(metric=ReadThroughput, value=0.0), + MetricRecord(metric=ServiceCallDuration, value=PT0.06260225S), + MetricRecord(metric=HttpStatusCode, value=204), + MetricRecord(metric=BackoffDelayDuration, value=PT0S), + MetricRecord(metric=TimeToLastByte, value=PT0.064313667S), + MetricRecord(metric=AwsRequestId, value=RKZD44SE5DW91K1G)], children=[ + MetricCollection(name=HttpClient, metrics=[ + MetricRecord(metric=AvailableConcurrency, value=1), + MetricRecord(metric=LeasedConcurrency, value=0), + MetricRecord(metric=ConcurrencyAcquireDuration, value=PT0S), + MetricRecord(metric=PendingConcurrencyAcquires, value=0), + MetricRecord(metric=MaxConcurrency, value=512), + MetricRecord(metric=HttpClientName, value=Apache)], children=[]) + ]) + ]) +``` + ### Enable S3 Server-side Logging The [Auditing](auditing) feature of the S3A connector can be used to generate diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java index 07caeb02f416a..1ba7f1848aa82 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java @@ -24,9 +24,9 @@ import java.nio.file.AccessDeniedException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; import org.assertj.core.api.Assertions; -import org.junit.Ignore; import org.junit.Test; import software.amazon.awssdk.awscore.AwsExecutionAttribute; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -106,6 +106,10 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase { public static final String EXCEPTION_THROWN_BY_INTERCEPTOR = "Exception thrown by interceptor"; + /** + * Text to include in asseertions + */ + private static final AtomicReference expectedMessage = new AtomicReference<>(); /** * New FS instance which will be closed in teardown. */ @@ -576,7 +580,7 @@ private void assertOpsUsingNewFs() throws IOException { .isFalse(); } - private final class RegionInterceptor implements ExecutionInterceptor { + private static final class RegionInterceptor implements ExecutionInterceptor { private final String endpoint; private final String region; private final boolean isFips; @@ -591,28 +595,50 @@ private final class RegionInterceptor implements ExecutionInterceptor { public void beforeExecution(Context.BeforeExecution context, ExecutionAttributes executionAttributes) { + + // extract state from the execution attributes. + final Boolean endpointOveridden = + executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN); + final String clientEndpoint = + executionAttributes.getAttribute(AwsExecutionAttribute.CLIENT_ENDPOINT).toString(); + final Boolean fipsEnabled = executionAttributes.getAttribute( + AwsExecutionAttribute.FIPS_ENDPOINT_ENABLED); + final String reg = executionAttributes.getAttribute(AwsExecutionAttribute.AWS_REGION).toString(); + + String state = "SDK beforeExecution callback; " + + "endpointOveridden=" + endpointOveridden + + "; clientEndpoint=" + clientEndpoint + + "; fipsEnabled=" + fipsEnabled + + "; region=" + reg; + if (endpoint != null && !endpoint.endsWith(CENTRAL_ENDPOINT)) { Assertions.assertThat( - executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN)) - .describedAs("Endpoint not overridden").isTrue(); + endpointOveridden) + .describedAs("Endpoint not overridden in %s. Client Config=%s", state, expectedMessage.get()) + .isTrue(); Assertions.assertThat( - executionAttributes.getAttribute(AwsExecutionAttribute.CLIENT_ENDPOINT).toString()) - .describedAs("There is an endpoint mismatch").isEqualTo("https://" + endpoint); + clientEndpoint) + .describedAs("There is an endpoint mismatch in %s. Client Config=%s" + , state, expectedMessage.get()) + .isEqualTo("https://" + endpoint); } else { Assertions.assertThat( - executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN)) - .describedAs("Endpoint is overridden").isEqualTo(null); + endpointOveridden) + .describedAs("Attribute endpointOveridden is null in %s. Client Config=%s", + state, expectedMessage.get()) + .isEqualTo(false); } - Assertions.assertThat( - executionAttributes.getAttribute(AwsExecutionAttribute.AWS_REGION).toString()) - .describedAs("Incorrect region set").isEqualTo(region); + Assertions.assertThat(reg) + .describedAs("Incorrect region set in %s. Client Config=%s", + state, expectedMessage.get()) + .isEqualTo(region); // verify the fips state matches expectation. - Assertions.assertThat(executionAttributes.getAttribute( - AwsExecutionAttribute.FIPS_ENDPOINT_ENABLED)) - .describedAs("Incorrect FIPS flag set in execution attributes") + Assertions.assertThat(fipsEnabled) + .describedAs("Incorrect FIPS flag set in %s; Client Config=%s", + state, expectedMessage.get()) .isNotNull() .isEqualTo(isFips); @@ -637,6 +663,11 @@ private S3Client createS3Client(Configuration conf, String endpoint, String configuredRegion, String expectedRegion, boolean isFips) throws IOException { + String expected = + "endpoint=" + endpoint + "; region=" + configuredRegion + + "; expectedRegion=" + expectedRegion + "; isFips=" + isFips; + LOG.info("Creating S3 client with {}", expected); + expectedMessage.set(expected); List interceptors = new ArrayList<>(); interceptors.add(new RegionInterceptor(endpoint, expectedRegion, isFips)); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestAwsSdkWorkarounds.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestAwsSdkWorkarounds.java index ed7a32928b8bf..d18a722a0e2cc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestAwsSdkWorkarounds.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestAwsSdkWorkarounds.java @@ -32,12 +32,9 @@ import static org.apache.hadoop.test.GenericTestUtils.LogCapturer.captureLogs; /** - * Verify that noisy transfer manager logs are turned off. + * Tests for any AWS SDK workaround code. *

- * This is done by creating new FS instances and then - * requesting an on-demand transfer manager from the store. - * As this is only done once per FS instance, a new FS is - * required per test case. + * These tests are inevitably brittle against SDK updates. */ public class ITestAwsSdkWorkarounds extends AbstractS3ATestBase { @@ -53,13 +50,6 @@ public class ITestAwsSdkWorkarounds extends AbstractS3ATestBase { private static final Logger XFER_LOG = LoggerFactory.getLogger(AwsSdkWorkarounds.TRANSFER_MANAGER); - /** - * This is the string which keeps being printed. - * {@value}. - */ - private static final String FORBIDDEN = - "The provided S3AsyncClient is an instance of MultipartS3AsyncClient"; - /** * Marginal test run speedup by skipping needless test dir cleanup. * @throws IOException failure @@ -70,23 +60,7 @@ protected void deleteTestDirInTeardown() throws IOException { } /** - * Test instantiation with logging disabled. - */ - @Test - public void testQuietLogging() throws Throwable { - // simulate the base state of logging - noisyLogging(); - // creating a new FS switches to quiet logging - try (S3AFileSystem newFs = newFileSystem()) { - String output = createAndLogTransferManager(newFs); - Assertions.assertThat(output) - .describedAs("LOG output") - .doesNotContain(FORBIDDEN); - } - } - - /** - * Test instantiation with logging disabled. + * Test instantiation with logging enabled. */ @Test public void testNoisyLogging() throws Throwable { @@ -95,9 +69,8 @@ public void testNoisyLogging() throws Throwable { noisyLogging(); String output = createAndLogTransferManager(newFs); Assertions.assertThat(output) - .describedAs("LOG output does not contain the forbidden text." - + " Has the SDK been fixed?") - .contains(FORBIDDEN); + .describedAs("LOG output") + .isEmpty(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 7b8dd3c11fcdc..737e4f7eef863 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -102,3 +102,6 @@ log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO # services it launches itself. # log4.logger.org.apache.hadoop.service=DEBUG +# log this at trace to trigger enabling the +# log4j.logger.org.apache.hadoop.fs.s3a.DefaultS3ClientFactory=TRACE +