steveloughran commented on code in PR #7923: URL: https://github.com/apache/hadoop/pull/7923#discussion_r2547523162
########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java: ########## @@ -25,7 +25,9 @@ import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import java.util.function.Function; - +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream; Review Comment: imports ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestAnalyticsInputStreamRetry.java: ########## @@ -0,0 +1,181 @@ +/* + * 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 org.apache.hadoop.fs.s3a.impl.streams.AnalyticsStreamFactory; +import org.apache.hadoop.fs.s3a.impl.streams.FactoryBindingParameters; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory; +import org.apache.hadoop.service.CompositeService; +import org.junit.jupiter.api.BeforeEach; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3ServiceClientConfiguration; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.CreateBucketResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + +import java.io.IOException; +import java.io.InputStream; +import java.net.ConnectException; +import java.net.URI; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +public class TestAnalyticsInputStreamRetry extends TestS3AInputStreamRetry { + + protected S3AsyncClient s3Async; + private FlakyS3StoreImpl s3Store; + + @BeforeEach + @Override + public void setup() throws Exception { + super.setup(); + conf = createConfiguration(); + fs = new S3AFileSystem(); + URI uri = URI.create(FS_S3A + "://" + BUCKET); + // unset S3CSE property from config to avoid pathIOE. + conf.unset(Constants.S3_ENCRYPTION_ALGORITHM); + conf.set(Constants.INPUT_STREAM_TYPE, Constants.INPUT_STREAM_TYPE_ANALYTICS); + fs.initialize(uri, conf); + s3Async = fs.getS3AInternals().getStore().getOrCreateAsyncClient(); + s3Store = new FlakyS3StoreImpl(); + + } + + @Override + protected ObjectInputStreamFactory getFactory() throws IOException { + return s3Store.getFactory(); + } + + + private class FlakyS3StoreImpl extends CompositeService { + ObjectInputStreamFactory factory; + + public FlakyS3StoreImpl() throws Exception { + super("FlakyS3Store"); + this.factory = new AnalyticsStreamFactory(); + addService(factory); + super.serviceInit(conf); + factory.bind(new FactoryBindingParameters(new FlakyCallbacks())); + } + + public ObjectInputStreamFactory getFactory() { + return factory; + } + + } + /** + * Callbacks from {@link ObjectInputStreamFactory} instances. + * Will throw connection exception twice on client.getObject() and succeed third time. + */ + private class FlakyCallbacks implements ObjectInputStreamFactory.StreamFactoryCallbacks { + AtomicInteger attempts = new AtomicInteger(0); + AtomicInteger fail = new AtomicInteger(2); + ConnectException exception = new ConnectException("Mock Connection Exception"); + @Override + public S3AsyncClient getOrCreateAsyncClient(final boolean requireCRT) throws IOException { + S3AsyncClientWrapper flakyClient = spy(new S3AsyncClientWrapper(s3Async)); + doAnswer( + invocation -> + CompletableFuture.supplyAsync( + () -> { + try { + InputStream flakyInputStream = + mockedInputStream(GetObjectResponse.builder().build(), + attempts.incrementAndGet() < fail.get(), + exception); + + return new ResponseInputStream<>( + GetObjectResponse.builder().build(), flakyInputStream); + } catch (Throwable e) { Review Comment: if it is RuntimeException, don't wrap ########## hadoop-project/pom.xml: ########## @@ -216,7 +216,7 @@ <aws-java-sdk.version>1.12.720</aws-java-sdk.version> <aws-java-sdk-v2.version>2.29.52</aws-java-sdk-v2.version> <amazon-s3-encryption-client-java.version>3.1.1</amazon-s3-encryption-client-java.version> - <amazon-s3-analyticsaccelerator-s3.version>1.2.1</amazon-s3-analyticsaccelerator-s3.version> + <amazon-s3-analyticsaccelerator-s3.version>1.3.0</amazon-s3-analyticsaccelerator-s3.version> Review Comment: rebase to trunk for this ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java: ########## @@ -44,6 +44,7 @@ import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import software.amazon.s3.analyticsaccelerator.util.retry.RetryStrategy; Review Comment: imports ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AnalyticsStreamRetryPolicy.java: ########## @@ -0,0 +1,73 @@ +/* + * 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.impl; + +import com.google.common.collect.ImmutableList; Review Comment: nit: import ordering/separation ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AnalyticsStreamRetryPolicy.java: ########## @@ -0,0 +1,73 @@ +/* + * 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.impl; + +import com.google.common.collect.ImmutableList; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.HttpChannelEOFException; +import org.apache.hadoop.fs.s3a.S3ARetryPolicy; +import org.apache.hadoop.net.ConnectTimeoutException; +import software.amazon.s3.analyticsaccelerator.util.retry.DefaultRetryStrategyImpl; +import software.amazon.s3.analyticsaccelerator.util.retry.RetryPolicy; +import software.amazon.s3.analyticsaccelerator.util.retry.RetryStrategy; + +import java.io.EOFException; +import java.net.ConnectException; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.util.List; + +import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; +import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT_DEFAULT; + +public class AnalyticsStreamRetryPolicy extends S3ARetryPolicy { + + private final RetryStrategy strategy; + + /** + * Instantiate. + * + * @param conf configuration to read. + */ + public AnalyticsStreamRetryPolicy(Configuration conf) { + super(conf); + int limit = conf.getInt(RETRY_LIMIT, RETRY_LIMIT_DEFAULT); + + RetryPolicy connectivityFailure = connectivityFailure(limit); + this.strategy = new DefaultRetryStrategyImpl(connectivityFailure); + } + + public RetryStrategy getAnalyticsRetryStrategy() { + return this.strategy; + } + + private RetryPolicy connectivityFailure(int limit) { + List<Class<? extends Throwable>> retryableExceptions = ImmutableList.of( + HttpChannelEOFException.class, + ConnectTimeoutException.class, + ConnectException.class, + EOFException.class, Review Comment: we need to make sure that a GET past EOF resulting in a 416 response maps to a RangeNotSatisfiableEOFException so it isn't retried. I think I'm with mukund here...we shoudn't need a new policy. If there's something wrong with the main one, then let's fix it. If subclassing is needed, then override `createExceptionMap()` and return that policy map; we used to have a special one for S3Guard and its dynamo db errors included in the normal set of errors. ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestAnalyticsInputStreamRetry.java: ########## @@ -0,0 +1,181 @@ +/* + * 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 org.apache.hadoop.fs.s3a.impl.streams.AnalyticsStreamFactory; Review Comment: imports -- 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]
