|
24 | 24 | import java.time.Duration;
|
25 | 25 | import java.util.Map;
|
26 | 26 | import java.util.Optional;
|
27 |
| -import java.util.concurrent.CancellationException; |
28 | 27 |
|
29 | 28 | import software.amazon.awssdk.core.ResponseInputStream;
|
30 | 29 | import software.amazon.awssdk.core.exception.SdkException;
|
|
34 | 33 | import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse;
|
35 | 34 | import software.amazon.awssdk.services.s3.model.GetObjectResponse;
|
36 | 35 | import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
|
37 |
| -import software.amazon.awssdk.services.s3.model.PutObjectRequest; |
38 |
| -import software.amazon.awssdk.services.s3.model.PutObjectResponse; |
39 |
| -import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload; |
40 | 36 |
|
41 | 37 | import org.apache.hadoop.classification.InterfaceAudience;
|
42 | 38 | import org.apache.hadoop.classification.InterfaceStability;
|
43 |
| -import org.apache.hadoop.classification.VisibleForTesting; |
44 | 39 | import org.apache.hadoop.conf.Configuration;
|
45 | 40 | import org.apache.hadoop.fs.LocalDirAllocator;
|
46 | 41 | import org.apache.hadoop.fs.PathCapabilities;
|
47 | 42 | import org.apache.hadoop.fs.s3a.api.RequestFactory;
|
48 | 43 | import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
49 | 44 | import org.apache.hadoop.fs.s3a.impl.ClientManager;
|
50 | 45 | import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException;
|
51 |
| -import org.apache.hadoop.fs.s3a.impl.MultipartIO; |
52 |
| -import org.apache.hadoop.fs.s3a.impl.MultipartIOService; |
53 |
| -import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; |
| 46 | +import org.apache.hadoop.fs.s3a.impl.StoreWriterService; |
| 47 | +import org.apache.hadoop.fs.s3a.impl.write.StoreWriter; |
54 | 48 | import org.apache.hadoop.fs.s3a.impl.S3AFileSystemOperations;
|
55 | 49 | import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
56 | 50 | import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory;
|
@@ -118,11 +112,11 @@ public interface S3AStore extends
|
118 | 112 | ClientManager clientManager();
|
119 | 113 |
|
120 | 114 | /**
|
121 |
| - * Get the Multipart IO operations. |
122 |
| - * @return an instance of multipart IO. |
| 115 | + * Get the store writer operations. |
| 116 | + * @return an instance of StoreWriterService. |
123 | 117 | */
|
124 |
| - default MultipartIOService getMultipartIO() { |
125 |
| - return lookupService(MultipartIO.MULTIPART_IO, MultipartIOService.class); |
| 118 | + default StoreWriterService getStoreWriter() { |
| 119 | + return lookupService(StoreWriter.STORE_WRITER, StoreWriterService.class); |
126 | 120 | }
|
127 | 121 |
|
128 | 122 | /**
|
@@ -321,71 +315,6 @@ void deleteObjectAtPath(
|
321 | 315 | boolean isFile)
|
322 | 316 | throws SdkException, UncheckedIOException;
|
323 | 317 |
|
324 |
| - /** |
325 |
| - * Start a transfer-manager managed async PUT of an object, |
326 |
| - * incrementing the put requests and put bytes |
327 |
| - * counters. |
328 |
| - * <p> |
329 |
| - * It does not update the other counters, |
330 |
| - * as existing code does that as progress callbacks come in. |
331 |
| - * Byte length is calculated from the file length, or, if there is no |
332 |
| - * file, from the content length of the header. |
333 |
| - * <p> |
334 |
| - * Because the operation is async, any stream supplied in the request |
335 |
| - * must reference data (files, buffers) which stay valid until the upload |
336 |
| - * completes. |
337 |
| - * Retry policy: N/A: the transfer manager is performing the upload. |
338 |
| - * Auditing: must be inside an audit span. |
339 |
| - * @param putObjectRequest the request |
340 |
| - * @param file the file to be uploaded |
341 |
| - * @param listener the progress listener for the request |
342 |
| - * @return the upload initiated |
343 |
| - * @throws IOException if transfer manager creation failed. |
344 |
| - */ |
345 |
| - @Retries.OnceRaw |
346 |
| - UploadInfo putObject( |
347 |
| - PutObjectRequest putObjectRequest, |
348 |
| - File file, |
349 |
| - ProgressableProgressListener listener) throws IOException; |
350 |
| - |
351 |
| - /** |
352 |
| - * PUT an object directly (i.e. not via the transfer manager). |
353 |
| - * Byte length is calculated from the file length, or, if there is no |
354 |
| - * file, from the content length of the header. |
355 |
| - * |
356 |
| - * Retry Policy: none. |
357 |
| - * Auditing: must be inside an audit span. |
358 |
| - * <i>Important: this call will close any input stream in the request.</i> |
359 |
| - * @param putObjectRequest the request |
360 |
| - * @param putOptions put object options |
361 |
| - * @param uploadData data to be uploaded |
362 |
| - * @param durationTrackerFactory factory for duration tracking |
363 |
| - * @return the upload initiated |
364 |
| - * @throws SdkException on problems |
365 |
| - */ |
366 |
| - @VisibleForTesting |
367 |
| - @Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed") |
368 |
| - PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest, |
369 |
| - PutObjectOptions putOptions, |
370 |
| - S3ADataBlocks.BlockUploadData uploadData, |
371 |
| - DurationTrackerFactory durationTrackerFactory) |
372 |
| - throws SdkException; |
373 |
| - |
374 |
| - /** |
375 |
| - * Wait for an upload to complete. |
376 |
| - * If the upload (or its result collection) failed, this is where |
377 |
| - * the failure is raised as an AWS exception. |
378 |
| - * Calls {@link S3AStore#incrementPutCompletedStatistics(boolean, long)} |
379 |
| - * to update the statistics. |
380 |
| - * @param key destination key |
381 |
| - * @param uploadInfo upload to wait for |
382 |
| - * @return the upload result |
383 |
| - * @throws IOException IO failure |
384 |
| - * @throws CancellationException if the wait() was cancelled |
385 |
| - */ |
386 |
| - @Retries.OnceTranslated |
387 |
| - CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo) |
388 |
| - throws IOException; |
389 | 318 |
|
390 | 319 | /**
|
391 | 320 | * Get the directory allocator.
|
|
0 commit comments