23
23
import java .io .FileNotFoundException ;
24
24
import java .io .IOException ;
25
25
import java .io .InputStream ;
26
- import java .util .ArrayList ;
27
26
import java .util .List ;
28
27
import java .util .Map ;
29
28
import java .util .concurrent .atomic .AtomicInteger ;
30
29
31
- import com .amazonaws .services .s3 .model .AmazonS3Exception ;
32
30
import com .amazonaws .services .s3 .model .CompleteMultipartUploadRequest ;
33
31
import com .amazonaws .services .s3 .model .CompleteMultipartUploadResult ;
34
32
import com .amazonaws .services .s3 .model .InitiateMultipartUploadRequest ;
51
49
import org .apache .hadoop .conf .Configuration ;
52
50
import org .apache .hadoop .fs .Path ;
53
51
import org .apache .hadoop .fs .PathIOException ;
52
+ import org .apache .hadoop .fs .s3a .impl .RawS3A ;
54
53
import org .apache .hadoop .fs .s3a .impl .StoreContext ;
55
54
import org .apache .hadoop .fs .s3a .statistics .S3AStatisticsContext ;
56
55
import org .apache .hadoop .fs .s3a .s3guard .BulkOperationState ;
57
56
import org .apache .hadoop .fs .s3a .s3guard .S3Guard ;
58
- import org .apache .hadoop .fs .s3a .select .SelectBinding ;
59
- import org .apache .hadoop .util .DurationInfo ;
60
57
import org .apache .hadoop .util .functional .CallableRaisingIOE ;
61
58
62
- import static org .apache .hadoop .thirdparty .com .google .common .base .Preconditions .checkArgument ;
63
59
import static org .apache .hadoop .thirdparty .com .google .common .base .Preconditions .checkNotNull ;
64
60
import static org .apache .hadoop .fs .s3a .Invoker .*;
65
- import static org .apache .hadoop .fs .s3a .S3AUtils .longOption ;
66
- import static org .apache .hadoop .fs .s3a .impl .InternalConstants .DEFAULT_UPLOAD_PART_COUNT_LIMIT ;
67
- import static org .apache .hadoop .fs .s3a .impl .InternalConstants .UPLOAD_PART_COUNT_LIMIT ;
68
61
69
62
/**
70
63
* Helper for low-level operations against an S3 Bucket for writing data,
@@ -112,6 +105,9 @@ public class WriteOperationHelper implements WriteOperations {
112
105
/** Bucket of the owner FS. */
113
106
private final String bucket ;
114
107
108
+ /** Raw S3A implementation to invoke. */
109
+ private final RawS3A rawS3A ;
110
+
115
111
/**
116
112
* statistics context.
117
113
*/
@@ -124,18 +120,20 @@ public class WriteOperationHelper implements WriteOperations {
124
120
* @param owner owner FS creating the helper
125
121
* @param conf Configuration object
126
122
* @param statisticsContext statistics context
127
- *
123
+ * @param rawS3A raw S3A implementation.
128
124
*/
129
125
protected WriteOperationHelper (S3AFileSystem owner ,
130
126
Configuration conf ,
131
- S3AStatisticsContext statisticsContext ) {
127
+ S3AStatisticsContext statisticsContext ,
128
+ RawS3A rawS3A ) {
132
129
this .owner = owner ;
133
130
this .invoker = new Invoker (new S3ARetryPolicy (conf ),
134
131
this ::operationRetried );
135
132
this .conf = conf ;
136
133
this .statisticsContext = statisticsContext ;
137
134
this .context = owner .createStoreContext ();
138
135
this .bucket = owner .getBucket ();
136
+ this .rawS3A = rawS3A ;
139
137
}
140
138
141
139
/**
@@ -231,7 +229,7 @@ public void writeFailed(Exception ex) {
231
229
* @return a new metadata instance
232
230
*/
233
231
public ObjectMetadata newObjectMetadata (long length ) {
234
- return owner .newObjectMetadata (length );
232
+ return context . getRequestFactory () .newObjectMetadata (length );
235
233
}
236
234
237
235
/**
@@ -249,7 +247,7 @@ public String initiateMultiPartUpload(String destKey) throws IOException {
249
247
destKey );
250
248
251
249
return retry ("initiate MultiPartUpload" , destKey , true ,
252
- () -> owner .initiateMultipartUpload (initiateMPURequest ).getUploadId ());
250
+ () -> rawS3A .initiateMultipartUpload (initiateMPURequest ).getUploadId ());
253
251
}
254
252
255
253
/**
@@ -279,18 +277,16 @@ private CompleteMultipartUploadResult finalizeMultipartUpload(
279
277
throw new PathIOException (destKey ,
280
278
"No upload parts in multipart upload" );
281
279
}
280
+
281
+ CompleteMultipartUploadRequest request =
282
+ context .getRequestFactory ().newCompleteMultipartUploadRequest (
283
+ destKey , uploadId , partETags );
282
284
CompleteMultipartUploadResult uploadResult =
283
285
invoker .retry ("Completing multipart upload" , destKey ,
284
286
true ,
285
287
retrying ,
286
288
() -> {
287
- // a copy of the list is required, so that the AWS SDK doesn't
288
- // attempt to sort an unmodifiable list.
289
- return owner .getAmazonS3Client ().completeMultipartUpload (
290
- new CompleteMultipartUploadRequest (bucket ,
291
- destKey ,
292
- uploadId ,
293
- new ArrayList <>(partETags )));
289
+ return rawS3A .completeMultipartUpload (request );
294
290
}
295
291
);
296
292
owner .finishedWrite (destKey , length , uploadResult .getETag (),
@@ -432,9 +428,9 @@ public UploadPartRequest newUploadPartRequest(
432
428
InputStream uploadStream ,
433
429
File sourceFile ,
434
430
Long offset ) throws PathIOException {
435
- checkNotNull (uploadId );
436
431
437
- return context .getRequestFactory ().newUploadPartRequest (destKey , uploadId , partNumber , size , uploadStream , sourceFile , offset );
432
+ return context .getRequestFactory ().newUploadPartRequest (
433
+ destKey , uploadId , partNumber , size , uploadStream , sourceFile , offset );
438
434
}
439
435
440
436
/**
@@ -610,33 +606,6 @@ public SelectObjectContentResult select(
610
606
final SelectObjectContentRequest request ,
611
607
final String action )
612
608
throws IOException {
613
- String bucketName = request .getBucketName ();
614
- Preconditions .checkArgument (bucket .equals (bucketName ),
615
- "wrong bucket: %s" , bucketName );
616
- if (LOG .isDebugEnabled ()) {
617
- LOG .debug ("Initiating select call {} {}" ,
618
- source , request .getExpression ());
619
- LOG .debug (SelectBinding .toString (request ));
620
- }
621
- return invoker .retry (
622
- action ,
623
- source .toString (),
624
- true ,
625
- () -> {
626
- try (DurationInfo ignored =
627
- new DurationInfo (LOG , "S3 Select operation" )) {
628
- try {
629
- return owner .getAmazonS3Client ().selectObjectContent (request );
630
- } catch (AmazonS3Exception e ) {
631
- LOG .error ("Failure of S3 Select request against {}" ,
632
- source );
633
- LOG .debug ("S3 Select request against {}:\n {}" ,
634
- source ,
635
- SelectBinding .toString (request ),
636
- e );
637
- throw e ;
638
- }
639
- }
640
- });
609
+ return owner .store ().select (source , request , action );
641
610
}
642
611
}
0 commit comments