2929import java .lang .annotation .Retention ;
3030import java .lang .annotation .RetentionPolicy ;
3131import java .lang .reflect .Method ;
32+ import java .net .URI ;
33+ import java .nio .file .Path ;
3234import java .util .HashMap ;
3335import java .util .List ;
3436import java .util .Map ;
7981import org .apache .polaris .service .it .env .CatalogApi ;
8082import org .apache .polaris .service .it .env .ClientCredentials ;
8183import org .apache .polaris .service .it .env .IcebergHelper ;
84+ import org .apache .polaris .service .it .env .IntegrationTestsHelper ;
8285import org .apache .polaris .service .it .env .ManagementApi ;
8386import org .apache .polaris .service .it .env .PolarisApiEndpoints ;
8487import org .apache .polaris .service .it .env .PolarisClient ;
9295import org .junit .jupiter .api .Test ;
9396import org .junit .jupiter .api .TestInfo ;
9497import org .junit .jupiter .api .extension .ExtendWith ;
98+ import org .junit .jupiter .api .io .TempDir ;
9599
96100/**
97101 * Import the full core Iceberg catalog tests by hitting the REST service via the RESTCatalog
@@ -108,9 +112,9 @@ public class PolarisRestCatalogIntegrationTest extends CatalogTests<RESTCatalog>
108112 private static final String TEST_ROLE_ARN =
109113 Optional .ofNullable (System .getenv ("INTEGRATION_TEST_ROLE_ARN" ))
110114 .orElse ("arn:aws:iam::123456789012:role/my-role" );
111- private static final String S3_BUCKET_BASE =
112- Optional . ofNullable ( System . getenv ( "INTEGRATION_TEST_S3_PATH" ))
113- . orElse ( "file:///tmp/buckets/my-bucket" ) ;
115+
116+ private static URI s3BucketBase ;
117+ private static URI externalCatalogBase ;
114118
115119 protected static final String VIEW_QUERY = "select * from ns1.layer1_table" ;
116120 private static String principalRoleName ;
@@ -125,7 +129,7 @@ public class PolarisRestCatalogIntegrationTest extends CatalogTests<RESTCatalog>
125129 private String currentCatalogName ;
126130
127131 private final String catalogBaseLocation =
128- S3_BUCKET_BASE + "/" + System .getenv ("USER" ) + "/path/to/data" ;
132+ s3BucketBase + "/" + System .getenv ("USER" ) + "/path/to/data" ;
129133
130134 private static final String [] DEFAULT_CATALOG_PROPERTIES = {
131135 "allow.unstructured.table.location" , "true" ,
@@ -148,7 +152,8 @@ String[] properties() default {
148152 }
149153
150154 @ BeforeAll
151- static void setup (PolarisApiEndpoints apiEndpoints , ClientCredentials credentials ) {
155+ static void setup (
156+ PolarisApiEndpoints apiEndpoints , ClientCredentials credentials , @ TempDir Path tempDir ) {
152157 adminCredentials = credentials ;
153158 endpoints = apiEndpoints ;
154159 client = polarisClient (endpoints );
@@ -157,6 +162,9 @@ static void setup(PolarisApiEndpoints apiEndpoints, ClientCredentials credential
157162 principalRoleName = client .newEntityName ("rest-admin" );
158163 principalCredentials = managementApi .createPrincipalWithRole (principalName , principalRoleName );
159164 catalogApi = client .catalogApi (principalCredentials );
165+ URI testRootUri = IntegrationTestsHelper .getTemporaryDirectory (tempDir );
166+ s3BucketBase = testRootUri .resolve ("my-bucket" );
167+ externalCatalogBase = testRootUri .resolve ("external-catalog" );
160168 }
161169
162170 @ AfterAll
@@ -192,7 +200,7 @@ public void before(TestInfo testInfo) {
192200 for (int i = 0 ; i < properties .length ; i += 2 ) {
193201 catalogPropsBuilder .addProperty (properties [i ], properties [i + 1 ]);
194202 }
195- if (!S3_BUCKET_BASE . startsWith ( "file:/ " )) {
203+ if (!s3BucketBase . getScheme (). equals ( "file" )) {
196204 catalogPropsBuilder .addProperty (
197205 CatalogEntity .REPLACE_NEW_LOCATION_PREFIX_WITH_CATALOG_DEFAULT_KEY , "file:" );
198206 }
@@ -202,7 +210,7 @@ public void before(TestInfo testInfo) {
202210 .setName (currentCatalogName )
203211 .setProperties (catalogPropsBuilder .build ())
204212 .setStorageConfigInfo (
205- S3_BUCKET_BASE . startsWith ( "file:/ " )
213+ s3BucketBase . getScheme (). equals ( "file" )
206214 ? new FileStorageConfigInfo (
207215 StorageConfigInfo .StorageTypeEnum .FILE , List .of ("file://" ))
208216 : awsConfigModel )
@@ -541,12 +549,12 @@ public void testLoadTableWithAccessDelegationForExternalCatalogWithConfigDisable
541549 TableMetadata .newTableMetadata (
542550 new Schema (List .of (Types .NestedField .of (1 , false , "col1" , new Types .StringType ()))),
543551 PartitionSpec .unpartitioned (),
544- "file:///tmp /ns1/my_table" ,
552+ externalCatalogBase + " /ns1/my_table" ,
545553 Map .of ());
546554 try (ResolvingFileIO resolvingFileIO = new ResolvingFileIO ()) {
547555 resolvingFileIO .initialize (Map .of ());
548556 resolvingFileIO .setConf (new Configuration ());
549- String fileLocation = "file:///tmp /ns1/my_table/metadata/v1.metadata.json" ;
557+ String fileLocation = externalCatalogBase + " /ns1/my_table/metadata/v1.metadata.json" ;
550558 TableMetadataParser .write (tableMetadata , resolvingFileIO .newOutputFile (fileLocation ));
551559 restCatalog .registerTable (TableIdentifier .of (ns1 , "my_table" ), fileLocation );
552560 try {
@@ -576,12 +584,12 @@ public void testLoadTableWithoutAccessDelegationForExternalCatalogWithConfigDisa
576584 TableMetadata .newTableMetadata (
577585 new Schema (List .of (Types .NestedField .of (1 , false , "col1" , new Types .StringType ()))),
578586 PartitionSpec .unpartitioned (),
579- "file:///tmp /ns1/my_table" ,
587+ externalCatalogBase + " /ns1/my_table" ,
580588 Map .of ());
581589 try (ResolvingFileIO resolvingFileIO = new ResolvingFileIO ()) {
582590 resolvingFileIO .initialize (Map .of ());
583591 resolvingFileIO .setConf (new Configuration ());
584- String fileLocation = "file:///tmp /ns1/my_table/metadata/v1.metadata.json" ;
592+ String fileLocation = externalCatalogBase + " /ns1/my_table/metadata/v1.metadata.json" ;
585593 TableMetadataParser .write (tableMetadata , resolvingFileIO .newOutputFile (fileLocation ));
586594 restCatalog .registerTable (TableIdentifier .of (ns1 , "my_table" ), fileLocation );
587595 try {
@@ -610,12 +618,12 @@ public void testLoadTableWithAccessDelegationForExternalCatalogWithConfigEnabled
610618 TableMetadata .newTableMetadata (
611619 new Schema (List .of (Types .NestedField .of (1 , false , "col1" , new Types .StringType ()))),
612620 PartitionSpec .unpartitioned (),
613- "file:///tmp /ns1/my_table" ,
621+ externalCatalogBase + " /ns1/my_table" ,
614622 Map .of ());
615623 try (ResolvingFileIO resolvingFileIO = new ResolvingFileIO ()) {
616624 resolvingFileIO .initialize (Map .of ());
617625 resolvingFileIO .setConf (new Configuration ());
618- String fileLocation = "file:///tmp /ns1/my_table/metadata/v1.metadata.json" ;
626+ String fileLocation = externalCatalogBase + " /ns1/my_table/metadata/v1.metadata.json" ;
619627 TableMetadataParser .write (tableMetadata , resolvingFileIO .newOutputFile (fileLocation ));
620628 restCatalog .registerTable (TableIdentifier .of (ns1 , "my_table" ), fileLocation );
621629 try {
0 commit comments