Skip to content

Commit c771a1f

Browse files
passaroahmarsuhail
authored andcommitted
Upgrade GetObject to use SDK v2.
Key change: `getObject` now returns a `ResponseInputStream<GetObjectResponse>` rather than a `S3Object`. This makes it simpler to handle the input stream lifetime in various classes such as `S3AInputStream`, `S3ARemoteObject`, or `SDKStreamDrainer`.
1 parent 4085a8d commit c771a1f

17 files changed

+321
-363
lines changed

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -59,13 +59,11 @@
5959
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
6060
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
6161
import com.amazonaws.services.s3.model.DeleteObjectsResult;
62-
import com.amazonaws.services.s3.model.GetObjectRequest;
6362
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
6463
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
6564
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
6665
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
6766
import com.amazonaws.services.s3.model.MultipartUpload;
68-
import com.amazonaws.services.s3.model.S3Object;
6967
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
7068
import com.amazonaws.services.s3.model.SelectObjectContentResult;
7169
import com.amazonaws.services.s3.model.StorageClass;
@@ -78,7 +76,10 @@
7876
import org.slf4j.Logger;
7977
import org.slf4j.LoggerFactory;
8078

79+
import software.amazon.awssdk.core.ResponseInputStream;
8180
import software.amazon.awssdk.services.s3.S3Client;
81+
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
82+
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
8283
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
8384
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
8485
import software.amazon.awssdk.awscore.exception.AwsServiceException;
@@ -1643,18 +1644,18 @@ public void close() {
16431644
}
16441645

16451646
@Override
1646-
public GetObjectRequest newGetRequest(final String key) {
1647+
public GetObjectRequest.Builder newGetRequestBuilder(final String key) {
16471648
// active the audit span used for the operation
16481649
try (AuditSpan span = auditSpan.activate()) {
1649-
return getRequestFactory().newGetObjectRequest(key);
1650+
return getRequestFactory().newGetObjectRequestBuilder(key);
16501651
}
16511652
}
16521653

16531654
@Override
1654-
public S3Object getObject(GetObjectRequest request) {
1655+
public ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request) {
16551656
// active the audit span used for the operation
16561657
try (AuditSpan span = auditSpan.activate()) {
1657-
return s3.getObject(request);
1658+
return s3V2.getObject(request);
16581659
}
16591660
}
16601661

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java

Lines changed: 42 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.io.Closeable;
2323
import java.io.EOFException;
2424
import java.io.IOException;
25+
import java.io.InputStream;
2526
import java.io.InterruptedIOException;
2627
import java.net.SocketTimeoutException;
2728
import java.nio.ByteBuffer;
@@ -31,9 +32,6 @@
3132
import java.util.concurrent.atomic.AtomicBoolean;
3233
import java.util.function.IntFunction;
3334

34-
import com.amazonaws.services.s3.model.GetObjectRequest;
35-
import com.amazonaws.services.s3.model.S3Object;
36-
import com.amazonaws.services.s3.model.S3ObjectInputStream;
3735
import org.slf4j.Logger;
3836
import org.slf4j.LoggerFactory;
3937

@@ -46,7 +44,6 @@
4644
import org.apache.hadoop.fs.FSExceptionMessages;
4745
import org.apache.hadoop.fs.FSInputStream;
4846
import org.apache.hadoop.fs.FileRange;
49-
import org.apache.hadoop.fs.PathIOException;
5047
import org.apache.hadoop.fs.StreamCapabilities;
5148
import org.apache.hadoop.fs.impl.CombinedFileRange;
5249
import org.apache.hadoop.fs.VectoredReadUtils;
@@ -61,6 +58,10 @@
6158
import org.apache.hadoop.io.IOUtils;
6259
import org.apache.hadoop.util.functional.CallableRaisingIOE;
6360

61+
import software.amazon.awssdk.core.ResponseInputStream;
62+
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
63+
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
64+
6465
import static java.util.Objects.requireNonNull;
6566
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
6667
import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint;
@@ -125,14 +126,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
125126
*/
126127
private volatile boolean closed;
127128
/**
128-
* wrappedStream is associated with an object (instance of S3Object). When
129-
* the object is garbage collected, the associated wrappedStream will be
130-
* closed. Keep a reference to this object to prevent the wrapperStream
131-
* still in use from being closed unexpectedly due to garbage collection.
132-
* See HADOOP-17338 for details.
129+
* Input stream returned by a getObject call.
133130
*/
134-
private S3Object object;
135-
private S3ObjectInputStream wrappedStream;
131+
private ResponseInputStream<GetObjectResponse> wrappedStream;
136132
private final S3AReadOpContext context;
137133
private final InputStreamCallbacks client;
138134

@@ -271,28 +267,22 @@ private synchronized void reopen(String reason, long targetPos, long length,
271267
uri, reason, targetPos, contentRangeFinish, length, pos, nextReadPos,
272268
inputPolicy);
273269

270+
GetObjectRequest request = client.newGetRequestBuilder(key)
271+
.range(S3AUtils.formatRange(targetPos, contentRangeFinish - 1))
272+
.applyMutation(changeTracker::maybeApplyConstraint)
273+
.build();
274274
long opencount = streamStatistics.streamOpened();
275-
GetObjectRequest request = client.newGetRequest(key)
276-
.withRange(targetPos, contentRangeFinish - 1);
277275
String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN;
278276
String text = String.format("%s %s at %d",
279277
operation, uri, targetPos);
280-
changeTracker.maybeApplyConstraint(request);
281-
282-
object = onceTrackingDuration(text, uri,
278+
wrappedStream = onceTrackingDuration(text, uri,
283279
streamStatistics.initiateGetRequest(), () ->
284280
client.getObject(request));
285281

286-
287-
changeTracker.processResponse(object, operation,
282+
changeTracker.processResponse(wrappedStream.response(), operation,
288283
targetPos);
289-
wrappedStream = object.getObjectContent();
290-
contentRangeStart = targetPos;
291-
if (wrappedStream == null) {
292-
throw new PathIOException(uri,
293-
"Null IO stream from " + operation + " of (" + reason + ") ");
294-
}
295284

285+
contentRangeStart = targetPos;
296286
this.pos = targetPos;
297287
}
298288

@@ -505,14 +495,15 @@ public synchronized int read() throws IOException {
505495
*/
506496
@Retries.OnceTranslated
507497
private void onReadFailure(IOException ioe, boolean forceAbort) {
498+
GetObjectResponse objectResponse = wrappedStream == null ? null : wrappedStream.response();
508499
if (LOG.isDebugEnabled()) {
509500
LOG.debug("Got exception while trying to read from stream {}, " +
510501
"client: {} object: {}, trying to recover: ",
511-
uri, client, object, ioe);
502+
uri, client, objectResponse, ioe);
512503
} else {
513504
LOG.info("Got exception while trying to read from stream {}, " +
514505
"client: {} object: {}, trying to recover: " + ioe,
515-
uri, client, object);
506+
uri, client, objectResponse);
516507
}
517508
streamStatistics.readException();
518509
closeStream("failure recovery", forceAbort, false);
@@ -672,7 +663,6 @@ private CompletableFuture<Boolean> closeStream(
672663
CompletableFuture<Boolean> operation;
673664
SDKStreamDrainer drainer = new SDKStreamDrainer(
674665
uri,
675-
object,
676666
wrappedStream,
677667
shouldAbort,
678668
(int) remaining,
@@ -694,7 +684,6 @@ private CompletableFuture<Boolean> closeStream(
694684
// either the stream is closed in the blocking call or the async call is
695685
// submitted with its own copy of the references
696686
wrappedStream = null;
697-
object = null;
698687
return operation;
699688
}
700689

@@ -910,29 +899,21 @@ public void readVectored(List<? extends FileRange> ranges,
910899
private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange,
911900
IntFunction<ByteBuffer> allocate) {
912901
LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr);
913-
// This reference is must be kept till all buffers are populated as this is a
914-
// finalizable object which closes the internal stream when gc triggers.
915-
S3Object objectRange = null;
916-
S3ObjectInputStream objectContent = null;
902+
ResponseInputStream<GetObjectResponse> rangeContent = null;
917903
try {
918904
checkIfVectoredIOStopped();
919905
final String operationName = "readCombinedFileRange";
920-
objectRange = getS3Object(operationName,
906+
rangeContent = getS3Object(operationName,
921907
combinedFileRange.getOffset(),
922908
combinedFileRange.getLength());
923-
objectContent = objectRange.getObjectContent();
924-
if (objectContent == null) {
925-
throw new PathIOException(uri,
926-
"Null IO stream received during " + operationName);
927-
}
928-
populateChildBuffers(combinedFileRange, objectContent, allocate);
909+
populateChildBuffers(combinedFileRange, rangeContent, allocate);
929910
} catch (Exception ex) {
930911
LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex);
931912
for(FileRange child : combinedFileRange.getUnderlying()) {
932913
child.getData().completeExceptionally(ex);
933914
}
934915
} finally {
935-
IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
916+
IOUtils.cleanupWithLogger(LOG, rangeContent);
936917
}
937918
LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr);
938919
}
@@ -945,7 +926,7 @@ private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRa
945926
* @throws IOException any IOE.
946927
*/
947928
private void populateChildBuffers(CombinedFileRange combinedFileRange,
948-
S3ObjectInputStream objectContent,
929+
InputStream objectContent,
949930
IntFunction<ByteBuffer> allocate) throws IOException {
950931
// If the combined file range just contains a single child
951932
// range, we only have to fill that one child buffer else
@@ -977,7 +958,7 @@ private void populateChildBuffers(CombinedFileRange combinedFileRange,
977958
* @param drainQuantity how many bytes to drain.
978959
* @throws IOException any IOE.
979960
*/
980-
private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity)
961+
private void drainUnnecessaryData(InputStream objectContent, long drainQuantity)
981962
throws IOException {
982963
int drainBytes = 0;
983964
int readCount;
@@ -1019,26 +1000,20 @@ private void validateRangeRequest(FileRange range) throws EOFException {
10191000
*/
10201001
private void readSingleRange(FileRange range, ByteBuffer buffer) {
10211002
LOG.debug("Start reading range {} from path {} ", range, pathStr);
1022-
S3Object objectRange = null;
1023-
S3ObjectInputStream objectContent = null;
1003+
ResponseInputStream<GetObjectResponse> objectRange = null;
10241004
try {
10251005
checkIfVectoredIOStopped();
10261006
long position = range.getOffset();
10271007
int length = range.getLength();
10281008
final String operationName = "readRange";
10291009
objectRange = getS3Object(operationName, position, length);
1030-
objectContent = objectRange.getObjectContent();
1031-
if (objectContent == null) {
1032-
throw new PathIOException(uri,
1033-
"Null IO stream received during " + operationName);
1034-
}
1035-
populateBuffer(length, buffer, objectContent);
1010+
populateBuffer(length, buffer, objectRange);
10361011
range.getData().complete(buffer);
10371012
} catch (Exception ex) {
10381013
LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex);
10391014
range.getData().completeExceptionally(ex);
10401015
} finally {
1041-
IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
1016+
IOUtils.cleanupWithLogger(LOG, objectRange);
10421017
}
10431018
LOG.debug("Finished reading range {} from path {} ", range, pathStr);
10441019
}
@@ -1053,7 +1028,7 @@ private void readSingleRange(FileRange range, ByteBuffer buffer) {
10531028
*/
10541029
private void populateBuffer(int length,
10551030
ByteBuffer buffer,
1056-
S3ObjectInputStream objectContent) throws IOException {
1031+
InputStream objectContent) throws IOException {
10571032

10581033
if (buffer.isDirect()) {
10591034
VectoredReadUtils.readInDirectBuffer(length, buffer,
@@ -1078,7 +1053,7 @@ private void populateBuffer(int length,
10781053
* @param length number of bytes to fill in dest.
10791054
* @throws IOException any IOE.
10801055
*/
1081-
private void readByteArray(S3ObjectInputStream objectContent,
1056+
private void readByteArray(InputStream objectContent,
10821057
byte[] dest,
10831058
int offset,
10841059
int length) throws IOException {
@@ -1105,13 +1080,16 @@ private void readByteArray(S3ObjectInputStream objectContent,
11051080
* @return S3Object result s3 object.
11061081
* @throws IOException exception if any.
11071082
*/
1108-
private S3Object getS3Object(String operationName, long position,
1109-
int length) throws IOException {
1110-
final GetObjectRequest request = client.newGetRequest(key)
1111-
.withRange(position, position + length - 1);
1112-
changeTracker.maybeApplyConstraint(request);
1083+
private ResponseInputStream<GetObjectResponse> getS3Object(String operationName,
1084+
long position,
1085+
int length)
1086+
throws IOException {
1087+
final GetObjectRequest request = client.newGetRequestBuilder(key)
1088+
.range(S3AUtils.formatRange(position, position + length - 1))
1089+
.applyMutation(changeTracker::maybeApplyConstraint)
1090+
.build();
11131091
DurationTracker tracker = streamStatistics.initiateGetRequest();
1114-
S3Object objectRange;
1092+
ResponseInputStream<GetObjectResponse> objectRange;
11151093
Invoker invoker = context.getReadInvoker();
11161094
try {
11171095
objectRange = invoker.retry(operationName, pathStr, true,
@@ -1126,7 +1104,7 @@ private S3Object getS3Object(String operationName, long position,
11261104
} finally {
11271105
tracker.close();
11281106
}
1129-
changeTracker.processResponse(objectRange, operationName,
1107+
changeTracker.processResponse(objectRange.response(), operationName,
11301108
position);
11311109
return objectRange;
11321110
}
@@ -1279,19 +1257,19 @@ public IOStatistics getIOStatistics() {
12791257
public interface InputStreamCallbacks extends Closeable {
12801258

12811259
/**
1282-
* Create a GET request.
1260+
* Create a GET request builder.
12831261
* @param key object key
1284-
* @return the request
1262+
* @return the request builder
12851263
*/
1286-
GetObjectRequest newGetRequest(String key);
1264+
GetObjectRequest.Builder newGetRequestBuilder(String key);
12871265

12881266
/**
12891267
* Execute the request.
12901268
* @param request the request
12911269
* @return the response
12921270
*/
12931271
@Retries.OnceRaw
1294-
S3Object getObject(GetObjectRequest request);
1272+
ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request);
12951273

12961274
/**
12971275
* Submit some asynchronous work, for example, draining a stream.

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1932,4 +1932,15 @@ public String toString() {
19321932
}
19331933
};
19341934

1935+
/**
1936+
* Format a byte range for a request header.
1937+
* See https://www.rfc-editor.org/rfc/rfc9110.html#section-14.1.2
1938+
*
1939+
* @param rangeStart the start byte offset
1940+
* @param rangeEnd the end byte offset (inclusive)
1941+
* @return a formatted byte range
1942+
*/
1943+
public static String formatRange(long rangeStart, long rangeEnd) {
1944+
return String.format("bytes={}-{}", rangeStart, rangeEnd);
1945+
}
19351946
}

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -29,30 +29,30 @@
2929
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
3030
import com.amazonaws.services.s3.model.DeleteObjectRequest;
3131
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
32-
import com.amazonaws.services.s3.model.GetObjectRequest;
3332
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
3433
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
3534
import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
36-
37-
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
38-
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
39-
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
40-
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
41-
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
4235
import com.amazonaws.services.s3.model.ObjectListing;
4336
import com.amazonaws.services.s3.model.PartETag;
4437
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
4538
import com.amazonaws.services.s3.model.SSECustomerKey;
4639
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
4740
import com.amazonaws.services.s3.model.StorageClass;
4841
import com.amazonaws.services.s3.model.UploadPartRequest;
49-
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
5042

5143
import org.apache.hadoop.fs.PathIOException;
5244
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
5345
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
5446
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
5547

48+
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
49+
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
50+
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
51+
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
52+
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
53+
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
54+
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
55+
5656
/**
5757
* Factory for S3 objects.
5858
*
@@ -202,11 +202,11 @@ CompleteMultipartUploadRequest newCompleteMultipartUploadRequest(
202202

203203

204204
/**
205-
* Create a GET request.
205+
* Create a GET request builder.
206206
* @param key object key
207-
* @return the request.
207+
* @return the request builder.
208208
*/
209-
GetObjectRequest newGetObjectRequest(String key);
209+
GetObjectRequest.Builder newGetObjectRequestBuilder(String key);
210210

211211
/**
212212
* Create and initialize a part request of a multipart upload.

0 commit comments

Comments
 (0)