53
53
54
54
import software .amazon .awssdk .core .ResponseInputStream ;
55
55
import software .amazon .awssdk .core .exception .SdkException ;
56
+ import software .amazon .awssdk .services .s3 .S3AsyncClient ;
56
57
import software .amazon .awssdk .services .s3 .S3Client ;
58
+ import software .amazon .awssdk .services .s3 .internal .crt .S3CrtAsyncClient ;
57
59
import software .amazon .awssdk .services .s3 .model .CompleteMultipartUploadRequest ;
58
60
import software .amazon .awssdk .services .s3 .model .CompleteMultipartUploadResponse ;
59
61
import software .amazon .awssdk .services .s3 .model .GetBucketLocationRequest ;
86
88
import software .amazon .awssdk .transfer .s3 .model .Copy ;
87
89
import software .amazon .awssdk .transfer .s3 .model .CopyRequest ;
88
90
91
+ import software .amazon .s3 .analyticsaccelerator .S3SdkObjectClient ;
92
+ import software .amazon .s3 .analyticsaccelerator .S3SeekableInputStreamConfiguration ;
93
+ import software .amazon .s3 .analyticsaccelerator .S3SeekableInputStreamFactory ;
94
+ import software .amazon .s3 .analyticsaccelerator .common .ConnectorConfiguration ;
95
+
89
96
import org .apache .hadoop .fs .impl .prefetch .ExecutorServiceFuturePool ;
90
97
import org .slf4j .Logger ;
91
98
import org .slf4j .LoggerFactory ;
@@ -313,6 +320,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
313
320
*/
314
321
private S3Client s3Client ;
315
322
323
+ /**
324
+ * CRT-Based S3Client created of analytics accelerator library is enabled
325
+ * and managed by the S3AStoreImpl. Analytics accelerator library can be
326
+ * enabled with {@link Constants#ANALYTICS_ACCELERATOR_ENABLED_KEY}
327
+ */
328
+ private S3AsyncClient s3AsyncClient ;
329
+
316
330
// initial callback policy is fail-once; it's there just to assist
317
331
// some mock tests and other codepaths trying to call the low level
318
332
// APIs on an uninitialized filesystem.
@@ -340,6 +354,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
340
354
// If true, the prefetching input stream is used for reads.
341
355
private boolean prefetchEnabled ;
342
356
357
+ // If true, S3SeekableInputStream from Analytics Accelerator for Amazon S3 will be used.
358
+ private boolean analyticsAcceleratorEnabled ;
359
+
360
+ private boolean analyticsAcceleratorCRTEnabled ;
361
+
343
362
// Size in bytes of a single prefetch block.
344
363
private int prefetchBlockSize ;
345
364
@@ -515,6 +534,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
515
534
*/
516
535
private boolean s3AccessGrantsEnabled ;
517
536
537
+ /**
538
+ * Factory to create S3SeekableInputStream if {@link this#analyticsAcceleratorEnabled} is true.
539
+ */
540
+ private S3SeekableInputStreamFactory s3SeekableInputStreamFactory ;
541
+
518
542
/** Add any deprecated keys. */
519
543
@ SuppressWarnings ("deprecation" )
520
544
private static void addDeprecatedKeys () {
@@ -670,8 +694,21 @@ public void initialize(URI name, Configuration originalConf)
670
694
this .prefetchBlockSize = (int ) prefetchBlockSizeLong ;
671
695
this .prefetchBlockCount =
672
696
intOption (conf , PREFETCH_BLOCK_COUNT_KEY , PREFETCH_BLOCK_DEFAULT_COUNT , 1 );
697
+
698
+ this .analyticsAcceleratorEnabled =
699
+ conf .getBoolean (ANALYTICS_ACCELERATOR_ENABLED_KEY , ANALYTICS_ACCELERATOR_ENABLED_DEFAULT );
700
+ this .analyticsAcceleratorCRTEnabled =
701
+ conf .getBoolean (ANALYTICS_ACCELERATOR_CRT_ENABLED ,
702
+ ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT );
703
+
673
704
this .isMultipartUploadEnabled = conf .getBoolean (MULTIPART_UPLOADS_ENABLED ,
674
- DEFAULT_MULTIPART_UPLOAD_ENABLED );
705
+ DEFAULT_MULTIPART_UPLOAD_ENABLED );
706
+
707
+ if (this .analyticsAcceleratorEnabled && !analyticsAcceleratorCRTEnabled ) {
708
+ // Temp change: Analytics Accelerator with S3AsyncClient do not support Multi-part upload.
709
+ this .isMultipartUploadEnabled = false ;
710
+ }
711
+
675
712
// multipart copy and upload are the same; this just makes it explicit
676
713
this .isMultipartCopyEnabled = isMultipartUploadEnabled ;
677
714
@@ -794,6 +831,27 @@ public void initialize(URI name, Configuration originalConf)
794
831
// directly through the client manager.
795
832
// this is to aid mocking.
796
833
s3Client = store .getOrCreateS3Client ();
834
+
835
+ if (this .analyticsAcceleratorEnabled ) {
836
+ LOG .info ("Using S3SeekableInputStream" );
837
+ if (this .analyticsAcceleratorCRTEnabled ) {
838
+ LOG .info ("Using S3 CRT client for analytics accelerator S3" );
839
+ this .s3AsyncClient = S3CrtAsyncClient .builder ().maxConcurrency (600 ).build ();
840
+ } else {
841
+ LOG .info ("Using S3 async client for analytics accelerator S3" );
842
+ this .s3AsyncClient = store .getOrCreateAsyncClient ();
843
+ }
844
+
845
+ ConnectorConfiguration configuration = new ConnectorConfiguration (conf ,
846
+ ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX );
847
+ S3SeekableInputStreamConfiguration seekableInputStreamConfiguration =
848
+ S3SeekableInputStreamConfiguration .fromConfiguration (configuration );
849
+ this .s3SeekableInputStreamFactory =
850
+ new S3SeekableInputStreamFactory (
851
+ new S3SdkObjectClient (this .s3AsyncClient ),
852
+ seekableInputStreamConfiguration );
853
+ }
854
+
797
855
// The filesystem is now ready to perform operations against
798
856
// S3
799
857
// This initiates a probe against S3 for the bucket existing.
@@ -1861,6 +1919,8 @@ private FSDataInputStream executeOpen(
1861
1919
final Path path ,
1862
1920
final OpenFileSupport .OpenFileInformation fileInformation )
1863
1921
throws IOException {
1922
+
1923
+
1864
1924
// create the input stream statistics before opening
1865
1925
// the file so that the time to prepare to open the file is included.
1866
1926
S3AInputStreamStatistics inputStreamStats =
@@ -1877,6 +1937,14 @@ private FSDataInputStream executeOpen(
1877
1937
fileInformation .applyOptions (readContext );
1878
1938
LOG .debug ("Opening '{}'" , readContext );
1879
1939
1940
+ if (this .analyticsAcceleratorEnabled ) {
1941
+ return new FSDataInputStream (
1942
+ new S3ASeekableStream (
1943
+ this .bucket ,
1944
+ pathToKey (path ),
1945
+ s3SeekableInputStreamFactory ));
1946
+ }
1947
+
1880
1948
if (this .prefetchEnabled ) {
1881
1949
Configuration configuration = getConf ();
1882
1950
initLocalDirAllocatorIfNotInitialized (configuration );
@@ -4354,9 +4422,11 @@ public void close() throws IOException {
4354
4422
protected synchronized void stopAllServices () {
4355
4423
try {
4356
4424
trackDuration (getDurationTrackerFactory (), FILESYSTEM_CLOSE .getSymbol (), () -> {
4357
- closeAutocloseables (LOG , store );
4425
+ closeAutocloseables (LOG , store , s3SeekableInputStreamFactory );
4358
4426
store = null ;
4359
4427
s3Client = null ;
4428
+ s3AsyncClient = null ;
4429
+ s3SeekableInputStreamFactory = null ;
4360
4430
4361
4431
// At this point the S3A client is shut down,
4362
4432
// now the executor pools are closed
0 commit comments