Skip to content

Commit c4080aa

Browse files
committed
renames
1 parent 6374dbb commit c4080aa

File tree

5 files changed

+53
-62
lines changed

5 files changed

+53
-62
lines changed

aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIO.java

Lines changed: 37 additions & 42 deletions
Original file line numberDiff line numberDiff line change
@@ -494,27 +494,26 @@ public void fileIOWithPrefixedS3ClientKryoSerialization() throws IOException {
494494

495495
// there should be a client for the generic and specific storage prefix available
496496
assertThat(io.clientForStoragePath("s3")).isInstanceOf(PrefixedS3Client.class);
497-
assertThat(io.clientForStoragePath("s3").s3Client()).isInstanceOf(S3Client.class);
498-
assertThat(io.clientForStoragePath("s3").s3AsyncClient()).isInstanceOf(S3AsyncClient.class);
497+
assertThat(io.clientForStoragePath("s3").s3()).isInstanceOf(S3Client.class);
498+
assertThat(io.clientForStoragePath("s3").s3Async()).isInstanceOf(S3AsyncClient.class);
499499
assertThat(io.clientForStoragePath("s3://my-bucket/my-path"))
500500
.isInstanceOf(PrefixedS3Client.class);
501-
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3Client())
502-
.isInstanceOf(S3Client.class);
503-
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3AsyncClient())
501+
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3()).isInstanceOf(S3Client.class);
502+
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3Async())
504503
.isInstanceOf(S3AsyncClient.class);
505504

506505
S3FileIO fileIO = TestHelpers.KryoHelpers.roundTripSerialize(io);
507506
assertThat(fileIO.credentials()).isEqualTo(io.credentials());
508507

509508
// make sure there's a client for the generic and specific storage prefix available after ser/de
510509
assertThat(fileIO.clientForStoragePath("s3")).isInstanceOf(PrefixedS3Client.class);
511-
assertThat(fileIO.clientForStoragePath("s3").s3Client()).isInstanceOf(S3Client.class);
512-
assertThat(fileIO.clientForStoragePath("s3").s3AsyncClient()).isInstanceOf(S3AsyncClient.class);
510+
assertThat(fileIO.clientForStoragePath("s3").s3()).isInstanceOf(S3Client.class);
511+
assertThat(fileIO.clientForStoragePath("s3").s3Async()).isInstanceOf(S3AsyncClient.class);
513512
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path"))
514513
.isInstanceOf(PrefixedS3Client.class);
515-
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3Client())
514+
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3())
516515
.isInstanceOf(S3Client.class);
517-
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3AsyncClient())
516+
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3Async())
518517
.isInstanceOf(S3AsyncClient.class);
519518
}
520519

@@ -529,27 +528,26 @@ public void fileIOWithPrefixedS3ClientJavaSerialization()
529528

530529
// there should be a client for the generic and specific storage prefix available
531530
assertThat(io.clientForStoragePath("s3")).isInstanceOf(PrefixedS3Client.class);
532-
assertThat(io.clientForStoragePath("s3").s3Client()).isInstanceOf(S3Client.class);
533-
assertThat(io.clientForStoragePath("s3").s3AsyncClient()).isInstanceOf(S3AsyncClient.class);
531+
assertThat(io.clientForStoragePath("s3").s3()).isInstanceOf(S3Client.class);
532+
assertThat(io.clientForStoragePath("s3").s3Async()).isInstanceOf(S3AsyncClient.class);
534533
assertThat(io.clientForStoragePath("s3://my-bucket/my-path"))
535534
.isInstanceOf(PrefixedS3Client.class);
536-
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3Client())
537-
.isInstanceOf(S3Client.class);
538-
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3AsyncClient())
535+
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3()).isInstanceOf(S3Client.class);
536+
assertThat(io.clientForStoragePath("s3://my-bucket/my-path").s3Async())
539537
.isInstanceOf(S3AsyncClient.class);
540538

541539
S3FileIO fileIO = TestHelpers.roundTripSerialize(io);
542540
assertThat(fileIO.credentials()).isEqualTo(io.credentials());
543541

544542
// make sure there's a client for the generic and specific storage prefix available after ser/de
545543
assertThat(fileIO.clientForStoragePath("s3")).isInstanceOf(PrefixedS3Client.class);
546-
assertThat(fileIO.clientForStoragePath("s3").s3Client()).isInstanceOf(S3Client.class);
547-
assertThat(fileIO.clientForStoragePath("s3").s3AsyncClient()).isInstanceOf(S3AsyncClient.class);
544+
assertThat(fileIO.clientForStoragePath("s3").s3()).isInstanceOf(S3Client.class);
545+
assertThat(fileIO.clientForStoragePath("s3").s3Async()).isInstanceOf(S3AsyncClient.class);
548546
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path"))
549547
.isInstanceOf(PrefixedS3Client.class);
550-
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3Client())
548+
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3())
551549
.isInstanceOf(S3Client.class);
552-
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3AsyncClient())
550+
assertThat(fileIO.clientForStoragePath("s3://my-bucket/my-path").s3Async())
553551
.isInstanceOf(S3AsyncClient.class);
554552
}
555553

@@ -651,70 +649,67 @@ public void resolvingFileIOLoadWithStorageCredentials()
651649
.hiddenImpl(ResolvingFileIO.class, String.class)
652650
.build(resolvingFileIO)
653651
.invoke("s3://foo/bar");
654-
ObjectAssert<S3FileIO> s3FileIO =
652+
ObjectAssert<S3FileIO> io =
655653
assertThat(result)
656654
.isInstanceOf(S3FileIO.class)
657655
.asInstanceOf(InstanceOfAssertFactories.type(S3FileIO.class));
658-
s3FileIO.extracting(S3FileIO::credentials).isEqualTo(storageCredentials);
659-
s3FileIO.satisfies(
656+
io.extracting(S3FileIO::credentials).isEqualTo(storageCredentials);
657+
io.satisfies(
660658
fileIO -> {
661659
// make sure there are two separate S3 clients for different prefixes and that the
662660
// underlying sync/async client is set
663661
assertThat(fileIO.clientForStoragePath("s3://foo/bar"))
664662
.isNotSameAs(fileIO.clientForStoragePath("s3"));
665-
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3Client())
666-
.isInstanceOf(S3Client.class);
667-
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3AsyncClient())
663+
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3()).isInstanceOf(S3Client.class);
664+
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3Async())
668665
.isInstanceOf(S3AsyncClient.class);
669666
});
670667

671668
// make sure credentials are still present after kryo serde
672-
ResolvingFileIO io = TestHelpers.KryoHelpers.roundTripSerialize(resolvingFileIO);
673-
assertThat(io.credentials()).isEqualTo(storageCredentials);
669+
ResolvingFileIO resolvingIO = TestHelpers.KryoHelpers.roundTripSerialize(resolvingFileIO);
670+
assertThat(resolvingIO.credentials()).isEqualTo(storageCredentials);
674671
result =
675672
DynMethods.builder("io")
676673
.hiddenImpl(ResolvingFileIO.class, String.class)
677-
.build(io)
674+
.build(resolvingIO)
678675
.invoke("s3://foo/bar");
679-
s3FileIO =
676+
io =
680677
assertThat(result)
681678
.isInstanceOf(S3FileIO.class)
682679
.asInstanceOf(InstanceOfAssertFactories.type(S3FileIO.class));
683-
s3FileIO.extracting(S3FileIO::credentials).isEqualTo(storageCredentials);
684-
s3FileIO.satisfies(
680+
io.extracting(S3FileIO::credentials).isEqualTo(storageCredentials);
681+
io.satisfies(
685682
fileIO -> {
686683
// make sure there are two separate S3 clients for different prefixes and that the
687684
// underlying sync/async client is set
688685
assertThat(fileIO.clientForStoragePath("s3://foo/bar"))
689686
.isNotSameAs(fileIO.clientForStoragePath("s3"));
690-
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3Client())
691-
.isInstanceOf(S3Client.class);
692-
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3AsyncClient())
687+
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3()).isInstanceOf(S3Client.class);
688+
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3Async())
693689
.isInstanceOf(S3AsyncClient.class);
694690
});
695691

696692
// make sure credentials are still present after java serde
697-
io = TestHelpers.roundTripSerialize(resolvingFileIO);
698-
assertThat(io.credentials()).isEqualTo(storageCredentials);
693+
resolvingIO = TestHelpers.roundTripSerialize(resolvingFileIO);
694+
assertThat(resolvingIO.credentials()).isEqualTo(storageCredentials);
699695
result =
700696
DynMethods.builder("io")
701697
.hiddenImpl(ResolvingFileIO.class, String.class)
702-
.build(io)
698+
.build(resolvingIO)
703699
.invoke("s3://foo/bar");
704-
s3FileIO =
700+
io =
705701
assertThat(result)
706702
.isInstanceOf(S3FileIO.class)
707703
.asInstanceOf(InstanceOfAssertFactories.type(S3FileIO.class));
708-
s3FileIO.extracting(S3FileIO::credentials).isEqualTo(storageCredentials);
709-
s3FileIO.satisfies(
704+
io.extracting(S3FileIO::credentials).isEqualTo(storageCredentials);
705+
io.satisfies(
710706
fileIO -> {
711707
// make sure there are two separate S3 clients for different prefixes and that the
712708
// underlying sync/async client is set
713709
assertThat(fileIO.clientForStoragePath("s3://foo/bar"))
714710
.isNotSameAs(fileIO.clientForStoragePath("s3"));
715-
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3Client())
716-
.isInstanceOf(S3Client.class);
717-
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3AsyncClient())
711+
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3()).isInstanceOf(S3Client.class);
712+
assertThat(fileIO.clientForStoragePath("s3://foo/bar").s3Async())
718713
.isInstanceOf(S3AsyncClient.class);
719714
});
720715
}

aws/src/main/java/org/apache/iceberg/aws/s3/PrefixedS3Client.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ public PrefixedS3Client(
5353
this.s3 = ((AwsClientFactory) clientFactory)::s3;
5454
}
5555
if (s3FileIOProperties.isPreloadClientEnabled()) {
56-
s3Client();
56+
s3();
5757
}
5858
}
5959

@@ -73,7 +73,7 @@ public String prefix() {
7373
return prefix;
7474
}
7575

76-
public S3Client s3Client() {
76+
public S3Client s3() {
7777
if (s3Client == null) {
7878
synchronized (this) {
7979
if (s3Client == null) {
@@ -85,7 +85,7 @@ public S3Client s3Client() {
8585
return s3Client;
8686
}
8787

88-
public S3AsyncClient s3AsyncClient() {
88+
public S3AsyncClient s3Async() {
8989
if (s3AsyncClient == null) {
9090
synchronized (this) {
9191
if (s3AsyncClient == null) {

aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ public void deleteFile(String path) {
203203
DeleteObjectRequest deleteRequest =
204204
DeleteObjectRequest.builder().bucket(location.bucket()).key(location.key()).build();
205205

206-
client.s3Client().deleteObject(deleteRequest);
206+
client.s3().deleteObject(deleteRequest);
207207
}
208208

209209
@Override
@@ -300,7 +300,7 @@ private void tagFileToDelete(PrefixedS3Client client, String path, Set<Tag> dele
300300
GetObjectTaggingRequest getObjectTaggingRequest =
301301
GetObjectTaggingRequest.builder().bucket(bucket).key(objectKey).build();
302302
GetObjectTaggingResponse getObjectTaggingResponse =
303-
client.s3Client().getObjectTagging(getObjectTaggingRequest);
303+
client.s3().getObjectTagging(getObjectTaggingRequest);
304304
// Get existing tags, if any and then add the delete tags
305305
Set<Tag> tags = Sets.newHashSet();
306306
if (getObjectTaggingResponse.hasTagSet()) {
@@ -314,7 +314,7 @@ private void tagFileToDelete(PrefixedS3Client client, String path, Set<Tag> dele
314314
.key(objectKey)
315315
.tagging(Tagging.builder().tagSet(tags).build())
316316
.build();
317-
client.s3Client().putObjectTagging(putObjectTaggingRequest);
317+
client.s3().putObjectTagging(putObjectTaggingRequest);
318318
}
319319

320320
private List<String> deleteBatch(
@@ -330,7 +330,7 @@ private List<String> deleteBatch(
330330
.build();
331331
List<String> failures = Lists.newArrayList();
332332
try {
333-
DeleteObjectsResponse response = client.s3Client().deleteObjects(request);
333+
DeleteObjectsResponse response = client.s3().deleteObjects(request);
334334
if (response.hasErrors()) {
335335
failures.addAll(
336336
response.errors().stream()
@@ -362,7 +362,7 @@ public Iterable<FileInfo> listPrefix(String prefix) {
362362
ListObjectsV2Request.builder().bucket(s3uri.bucket()).prefix(s3uri.key()).build();
363363

364364
return () ->
365-
client.s3Client().listObjectsV2Paginator(request).stream()
365+
client.s3().listObjectsV2Paginator(request).stream()
366366
.flatMap(r -> r.contents().stream())
367367
.map(
368368
o ->
@@ -395,7 +395,7 @@ public void deletePrefix(String prefix) {
395395
*/
396396
@Deprecated
397397
public S3Client client() {
398-
return clientForStoragePath("s3").s3Client();
398+
return clientForStoragePath("s3").s3();
399399
}
400400

401401
/**
@@ -407,7 +407,7 @@ public S3Client client() {
407407
*/
408408
@Deprecated
409409
public S3AsyncClient asyncClient() {
410-
return clientForStoragePath("s3").s3AsyncClient();
410+
return clientForStoragePath("s3").s3Async();
411411
}
412412

413413
public PrefixedS3Client clientForStoragePath(String storagePath) {
@@ -555,7 +555,7 @@ public boolean recoverFile(String path) {
555555
S3URI location = new S3URI(path, client.s3FileIOProperties().bucketToAccessPointMapping());
556556
ListObjectVersionsIterable response =
557557
client
558-
.s3Client()
558+
.s3()
559559
.listObjectVersionsPaginator(
560560
builder -> builder.bucket(location.bucket()).prefix(location.key()));
561561

@@ -579,7 +579,7 @@ private boolean recoverObject(PrefixedS3Client client, ObjectVersion version, St
579579
// Perform a copy instead of deleting the delete marker
580580
// so that recovery does not rely on delete permissions
581581
client
582-
.s3Client()
582+
.s3()
583583
.copyObject(
584584
builder ->
585585
builder

aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -126,10 +126,8 @@ public static S3InputFile fromLocation(
126126
public static S3InputFile fromLocation(
127127
String location, long length, PrefixedS3Client client, MetricsContext metrics) {
128128
return new S3InputFile(
129-
client.s3Client(),
130-
client.s3FileIOProperties().isS3AnalyticsAcceleratorEnabled()
131-
? client.s3AsyncClient()
132-
: null,
129+
client.s3(),
130+
client.s3FileIOProperties().isS3AnalyticsAcceleratorEnabled() ? client.s3Async() : null,
133131
new S3URI(location, client.s3FileIOProperties().bucketToAccessPointMapping()),
134132
length > 0 ? length : null,
135133
client.s3FileIOProperties(),

aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -77,10 +77,8 @@ public static S3OutputFile fromLocation(
7777
public static S3OutputFile fromLocation(
7878
String location, PrefixedS3Client client, MetricsContext metrics) {
7979
return new S3OutputFile(
80-
client.s3Client(),
81-
client.s3FileIOProperties().isS3AnalyticsAcceleratorEnabled()
82-
? client.s3AsyncClient()
83-
: null,
80+
client.s3(),
81+
client.s3FileIOProperties().isS3AnalyticsAcceleratorEnabled() ? client.s3Async() : null,
8482
new S3URI(location, client.s3FileIOProperties().bucketToAccessPointMapping()),
8583
client.s3FileIOProperties(),
8684
metrics);

0 commit comments

Comments
 (0)