|
66 | 66 | import org.apache.iceberg.io.FileIOParser;
|
67 | 67 | import org.apache.iceberg.io.FileInfo;
|
68 | 68 | import org.apache.iceberg.io.IOUtil;
|
| 69 | +import org.apache.iceberg.io.ImmutableStorageCredential; |
69 | 70 | import org.apache.iceberg.io.InputFile;
|
70 | 71 | import org.apache.iceberg.io.OutputFile;
|
71 | 72 | import org.apache.iceberg.io.ResolvingFileIO;
|
| 73 | +import org.apache.iceberg.io.StorageCredential; |
72 | 74 | import org.apache.iceberg.jdbc.JdbcCatalog;
|
73 | 75 | import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
|
74 | 76 | import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
|
|
77 | 79 | import org.apache.iceberg.relocated.com.google.common.collect.Streams;
|
78 | 80 | import org.apache.iceberg.types.Types;
|
79 | 81 | import org.apache.iceberg.util.SerializableSupplier;
|
| 82 | +import org.assertj.core.api.InstanceOfAssertFactories; |
| 83 | +import org.assertj.core.api.ObjectAssert; |
80 | 84 | import org.junit.jupiter.api.AfterEach;
|
81 | 85 | import org.junit.jupiter.api.BeforeEach;
|
82 | 86 | import org.junit.jupiter.api.Disabled;
|
@@ -539,6 +543,115 @@ public void testInputFileWithManifest() throws IOException {
|
539 | 543 | verify(s3mock, never()).headObject(any(HeadObjectRequest.class));
|
540 | 544 | }
|
541 | 545 |
|
| 546 | + @Test |
| 547 | + public void noStorageCredentialConfigured() { |
| 548 | + S3FileIO fileIO = new S3FileIO(); |
| 549 | + fileIO.setCredentials(ImmutableList.of()); |
| 550 | + fileIO.initialize( |
| 551 | + ImmutableMap.of( |
| 552 | + "s3.access-key-id", |
| 553 | + "keyIdFromProperties", |
| 554 | + "s3.secret-access-key", |
| 555 | + "accessKeyFromProperties", |
| 556 | + "s3.session-token", |
| 557 | + "sessionTokenFromProperties")); |
| 558 | + |
| 559 | + ObjectAssert<S3FileIOProperties> s3FileIOProperties = |
| 560 | + assertThat(fileIO) |
| 561 | + .extracting("s3FileIOProperties") |
| 562 | + .asInstanceOf(InstanceOfAssertFactories.type(S3FileIOProperties.class)); |
| 563 | + s3FileIOProperties.extracting(S3FileIOProperties::accessKeyId).isEqualTo("keyIdFromProperties"); |
| 564 | + s3FileIOProperties |
| 565 | + .extracting(S3FileIOProperties::secretAccessKey) |
| 566 | + .isEqualTo("accessKeyFromProperties"); |
| 567 | + s3FileIOProperties |
| 568 | + .extracting(S3FileIOProperties::sessionToken) |
| 569 | + .isEqualTo("sessionTokenFromProperties"); |
| 570 | + } |
| 571 | + |
| 572 | + @Test |
| 573 | + public void singleStorageCredentialConfigured() { |
| 574 | + StorageCredential s3Credential = |
| 575 | + ImmutableStorageCredential.builder() |
| 576 | + .prefix("s3://custom-uri") |
| 577 | + .config( |
| 578 | + ImmutableMap.of( |
| 579 | + "s3.access-key-id", |
| 580 | + "keyIdFromCredential", |
| 581 | + "s3.secret-access-key", |
| 582 | + "accessKeyFromCredential", |
| 583 | + "s3.session-token", |
| 584 | + "sessionTokenFromCredential")) |
| 585 | + .build(); |
| 586 | + |
| 587 | + S3FileIO fileIO = new S3FileIO(); |
| 588 | + fileIO.setCredentials(ImmutableList.of(s3Credential)); |
| 589 | + fileIO.initialize( |
| 590 | + ImmutableMap.of( |
| 591 | + "s3.access-key-id", |
| 592 | + "keyIdFromProperties", |
| 593 | + "s3.secret-access-key", |
| 594 | + "accessKeyFromProperties", |
| 595 | + "s3.session-token", |
| 596 | + "sessionTokenFromProperties")); |
| 597 | + |
| 598 | + ObjectAssert<S3FileIOProperties> s3FileIOProperties = |
| 599 | + assertThat(fileIO) |
| 600 | + .extracting("s3FileIOProperties") |
| 601 | + .asInstanceOf(InstanceOfAssertFactories.type(S3FileIOProperties.class)); |
| 602 | + s3FileIOProperties.extracting(S3FileIOProperties::accessKeyId).isEqualTo("keyIdFromCredential"); |
| 603 | + s3FileIOProperties |
| 604 | + .extracting(S3FileIOProperties::secretAccessKey) |
| 605 | + .isEqualTo("accessKeyFromCredential"); |
| 606 | + s3FileIOProperties |
| 607 | + .extracting(S3FileIOProperties::sessionToken) |
| 608 | + .isEqualTo("sessionTokenFromCredential"); |
| 609 | + } |
| 610 | + |
| 611 | + @Test |
| 612 | + public void multipleStorageCredentialsConfigured() { |
| 613 | + StorageCredential s3Credential1 = |
| 614 | + ImmutableStorageCredential.builder() |
| 615 | + .prefix("s3://custom-uri/1") |
| 616 | + .config( |
| 617 | + ImmutableMap.of( |
| 618 | + "s3.access-key-id", |
| 619 | + "keyIdFromCredential1", |
| 620 | + "s3.secret-access-key", |
| 621 | + "accessKeyFromCredential1", |
| 622 | + "s3.session-token", |
| 623 | + "sessionTokenFromCredential1")) |
| 624 | + .build(); |
| 625 | + |
| 626 | + StorageCredential s3Credential2 = |
| 627 | + ImmutableStorageCredential.builder() |
| 628 | + .prefix("s3://custom-uri/2") |
| 629 | + .config( |
| 630 | + ImmutableMap.of( |
| 631 | + "s3.access-key-id", |
| 632 | + "keyIdFromCredential2", |
| 633 | + "s3.secret-access-key", |
| 634 | + "accessKeyFromCredential2", |
| 635 | + "s3.session-token", |
| 636 | + "sessionTokenFromCredential2")) |
| 637 | + .build(); |
| 638 | + |
| 639 | + S3FileIO fileIO = new S3FileIO(); |
| 640 | + fileIO.setCredentials(ImmutableList.of(s3Credential1, s3Credential2)); |
| 641 | + assertThatThrownBy( |
| 642 | + () -> |
| 643 | + fileIO.initialize( |
| 644 | + ImmutableMap.of( |
| 645 | + "s3.access-key-id", |
| 646 | + "keyIdFromProperties", |
| 647 | + "s3.secret-access-key", |
| 648 | + "accessKeyFromProperties", |
| 649 | + "s3.session-token", |
| 650 | + "sessionTokenFromProperties"))) |
| 651 | + .isInstanceOf(IllegalStateException.class) |
| 652 | + .hasMessage("Invalid S3 Credentials: only one S3 credential should exist"); |
| 653 | + } |
| 654 | + |
542 | 655 | private void createRandomObjects(String prefix, int count) {
|
543 | 656 | S3URI s3URI = new S3URI(prefix);
|
544 | 657 |
|
|
0 commit comments