Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.FileIOTracker;
import org.apache.iceberg.io.StorageCredential;
import org.apache.iceberg.io.SupportsStorageCredentials;
import org.apache.iceberg.metrics.MetricsReporter;
import org.apache.iceberg.metrics.MetricsReporters;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -1175,7 +1176,15 @@ private FileIO newFileIO(SessionContext context, Map<String, String> properties)
private FileIO newFileIO(
SessionContext context, Map<String, String> properties, List<Credential> storageCredentials) {
if (null != ioBuilder) {
return ioBuilder.apply(context, properties);
FileIO fileIO = ioBuilder.apply(context, properties);
if (!storageCredentials.isEmpty()
&& fileIO instanceof SupportsStorageCredentials ioWithCredentials) {
ioWithCredentials.setCredentials(
storageCredentials.stream()
.map(c -> StorageCredential.create(c.prefix(), c.config()))
.collect(Collectors.toList()));
}
return fileIO;
Comment on lines +1179 to +1187
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This approach is not right :(

With the statement

 ioBuilder.apply(context, properties);

we already create a file system factory in Trino (and also wrap it into a ForwardingFileIo).

https://github.com/trinodb/trino/blob/ae8daa0a0a07d43c6c36ce8085f7bbe7dd599d03/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoIcebergRestCatalogFactory.java#L117-L122

Ideally we would need to give the storageCredentials at the same time as the properties to the ioBuilder

e.g.

ioBuilder.apply(context, properties, storageCredentials);

(side note, there is no TriFunction in JDK and we'll likely need to introduce an interface if we agree on making such a change)

cc @nastra @kaveti

} else {
String ioImpl = properties.getOrDefault(CatalogProperties.FILE_IO_IMPL, DEFAULT_FILE_IO_IMPL);
return CatalogUtil.loadFileIO(
Expand Down
71 changes: 71 additions & 0 deletions core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.nio.file.Path;
Expand Down Expand Up @@ -67,6 +68,7 @@
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.TestCatalogUtil;
import org.apache.iceberg.Transaction;
import org.apache.iceberg.UpdatePartitionSpec;
import org.apache.iceberg.UpdateSchema;
Expand All @@ -86,6 +88,8 @@
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.inmemory.InMemoryCatalog;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.StorageCredential;
import org.apache.iceberg.io.SupportsStorageCredentials;
import org.apache.iceberg.metrics.CommitReport;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
Expand All @@ -99,6 +103,8 @@
import org.apache.iceberg.rest.auth.AuthSessionUtil;
import org.apache.iceberg.rest.auth.OAuth2Properties;
import org.apache.iceberg.rest.auth.OAuth2Util;
import org.apache.iceberg.rest.credentials.Credential;
import org.apache.iceberg.rest.credentials.ImmutableCredential;
import org.apache.iceberg.rest.requests.CreateTableRequest;
import org.apache.iceberg.rest.requests.ReportMetricsRequest;
import org.apache.iceberg.rest.requests.UpdateTableRequest;
Expand Down Expand Up @@ -3740,6 +3746,71 @@ public void testSequenceNumberConflictThrowsCommitFailed() {
.hasMessageContaining("Validation failed, please retry");
}

@Test
public void testIoBuilderReceivesStorageCredentials() {
Credential credential =
ImmutableCredential.builder()
.prefix("s3://test-bucket/")
.putConfig("s3.access-key-id", "test-access-key")
.putConfig("s3.secret-access-key", "test-secret-key")
.build();

// Adapter that injects storage credentials into LoadTableResponse
RESTCatalogAdapter adapter =
new RESTCatalogAdapter(backendCatalog) {
@SuppressWarnings("unchecked")
@Override
public <T extends RESTResponse> T handleRequest(
Route route,
Map<String, String> vars,
HTTPRequest httpRequest,
Class<T> responseType,
Consumer<Map<String, String>> responseHeaders) {
T response =
super.handleRequest(route, vars, httpRequest, responseType, responseHeaders);
if (route == Route.LOAD_TABLE && response instanceof LoadTableResponse loadResponse) {
return (T)
LoadTableResponse.builder()
.withTableMetadata(loadResponse.tableMetadata())
.addAllConfig(loadResponse.config())
.addCredential(credential)
.build();
}
return response;
}
};

AtomicReference<FileIO> createdFileIO = new AtomicReference<>();

try (RESTCatalog catalog =
catalog(
adapter,
clientBuilder ->
new RESTSessionCatalog(
clientBuilder,
(context, config) -> {
TestCatalogUtil.TestFileIOWithStorageCredentials fileIO =
new TestCatalogUtil.TestFileIOWithStorageCredentials();
createdFileIO.set(fileIO);
return fileIO;
}))) {
catalog.createNamespace(NS);
catalog.createTable(TABLE, SCHEMA);
catalog.loadTable(TABLE);

assertThat(createdFileIO.get()).isInstanceOf(SupportsStorageCredentials.class);
List<StorageCredential> creds =
((SupportsStorageCredentials) createdFileIO.get()).credentials();
assertThat(creds).hasSize(1);
assertThat(creds.get(0).prefix()).isEqualTo("s3://test-bucket/");
assertThat(creds.get(0).config())
.containsEntry("s3.access-key-id", "test-access-key")
.containsEntry("s3.secret-access-key", "test-secret-key");
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}

private RESTCatalog catalog(RESTCatalogAdapter adapter) {
RESTCatalog catalog =
new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter);
Expand Down
Loading