-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Support Iceberg REST catalog storage credentials in ForwardingFileIo #28425
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
b7a09db
d2a4f53
7535bd4
1548fc3
08583ff
66a19cc
515a7c8
3af270d
c64e9e2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,6 +35,7 @@ | |
| import org.apache.iceberg.catalog.Namespace; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
| import org.apache.iceberg.rest.HTTPClient; | ||
| import org.apache.iceberg.rest.RESTCatalogProperties; | ||
| import org.apache.iceberg.rest.RESTSessionCatalog; | ||
| import org.apache.iceberg.rest.RESTUtil; | ||
|
|
||
|
|
@@ -129,7 +130,7 @@ public synchronized TrinoCatalog create(ConnectorIdentity identity) | |
| properties.put(CatalogProperties.URI, serverUri.toString()); | ||
| warehouse.ifPresent(location -> properties.put(CatalogProperties.WAREHOUSE_LOCATION, location)); | ||
| prefix.ifPresent(prefix -> properties.put("prefix", prefix)); | ||
| properties.put("view-endpoints-supported", Boolean.toString(viewEndpointsEnabled)); | ||
| properties.put(RESTCatalogProperties.VIEW_ENDPOINTS_SUPPORTED, Boolean.toString(viewEndpointsEnabled)); | ||
| properties.put("trino-version", trinoVersion); | ||
| properties.put(AUTH_SESSION_TIMEOUT_MS, String.valueOf(sessionTimeout.toMillis())); | ||
| connectionTimeout.ifPresent(duration -> properties.put("rest.client.connection-timeout-ms", String.valueOf(duration.toMillis()))); | ||
|
|
@@ -148,8 +149,13 @@ public synchronized TrinoCatalog create(ConnectorIdentity identity) | |
| (context, config) -> { | ||
| ConnectorIdentity currentIdentity = (context.wrappedIdentity() != null) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems that the changes from apache/iceberg#12591 disregarded the fact that the @nastra was it intentional to skip it during the implementation phase?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @findinpath this was not intentional. I believe Trino is the only engine that goes through that path. Can you please open a PR with a fix?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. opened PR: apache/iceberg#15752 , cc @nastra |
||
| ? ((ConnectorIdentity) context.wrappedIdentity()) | ||
| : ConnectorIdentity.ofUser("fake"); | ||
| return fileIoFactory.create(fileSystemFactory.create(currentIdentity, config), true, config); | ||
| : ConnectorIdentity.ofUser("trino"); | ||
| return fileIoFactory.create( | ||
| fileSystemFactory.create(currentIdentity, config), | ||
| true, | ||
| config, | ||
| fileSystemFactory, | ||
| currentIdentity); | ||
| }); | ||
| icebergCatalogInstance.initialize(catalogName.toString(), properties.buildOrThrow()); | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -14,22 +14,30 @@ | |
| package io.trino.plugin.iceberg.fileio; | ||
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
|
kaveti marked this conversation as resolved.
|
||
| import com.google.common.collect.ImmutableList; | ||
| import com.google.common.collect.ImmutableMap; | ||
|
kaveti marked this conversation as resolved.
|
||
| import com.google.common.collect.Iterables; | ||
| import com.google.common.collect.Streams; | ||
| import io.trino.filesystem.Location; | ||
| import io.trino.filesystem.TrinoFileSystem; | ||
| import io.trino.plugin.iceberg.IcebergFileSystemFactory; | ||
| import io.trino.spi.TrinoException; | ||
| import io.trino.spi.security.ConnectorIdentity; | ||
| import org.apache.iceberg.DataFile; | ||
| import org.apache.iceberg.DeleteFile; | ||
| import org.apache.iceberg.ManifestFile; | ||
| import org.apache.iceberg.ManifestListFile; | ||
| import org.apache.iceberg.io.BulkDeletionFailureException; | ||
| import org.apache.iceberg.io.InputFile; | ||
| import org.apache.iceberg.io.OutputFile; | ||
| import org.apache.iceberg.io.StorageCredential; | ||
|
kaveti marked this conversation as resolved.
|
||
| import org.apache.iceberg.io.SupportsBulkOperations; | ||
| import org.apache.iceberg.io.SupportsStorageCredentials; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.UncheckedIOException; | ||
| import java.util.ArrayList; | ||
| import java.util.IdentityHashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.concurrent.Callable; | ||
|
|
@@ -45,7 +53,7 @@ | |
| import static java.util.stream.Collectors.joining; | ||
|
|
||
| public class ForwardingFileIo | ||
| implements SupportsBulkOperations | ||
| implements SupportsBulkOperations, SupportsStorageCredentials | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm looking at and am rather thinking that we should have a prefixed map of |
||
| { | ||
| private static final int DELETE_BATCH_SIZE = 1000; | ||
| private static final int BATCH_DELETE_PATHS_MESSAGE_LIMIT = 5; | ||
|
|
@@ -54,6 +62,11 @@ public class ForwardingFileIo | |
| private final Map<String, String> properties; | ||
| private final boolean useFileSizeFromMetadata; | ||
| private final ExecutorService deleteExecutor; | ||
| private final IcebergFileSystemFactory storageCredentialFileSystemFactory; | ||
| private final ConnectorIdentity storageCredentialIdentity; | ||
|
|
||
| private volatile List<StorageCredential> storageCredentials = ImmutableList.of(); | ||
| private volatile Map<String, TrinoFileSystem> prefixedFileSystems = ImmutableMap.of(); | ||
|
|
||
| @VisibleForTesting | ||
| public ForwardingFileIo(TrinoFileSystem fileSystem, boolean useFileSizeFromMetadata) | ||
|
|
@@ -62,22 +75,36 @@ public ForwardingFileIo(TrinoFileSystem fileSystem, boolean useFileSizeFromMetad | |
| } | ||
|
|
||
| public ForwardingFileIo(TrinoFileSystem fileSystem, Map<String, String> properties, boolean useFileSizeFromMetadata, ExecutorService deleteExecutor) | ||
| { | ||
| this(fileSystem, properties, useFileSizeFromMetadata, deleteExecutor, null, null); | ||
| } | ||
|
|
||
| public ForwardingFileIo( | ||
| TrinoFileSystem fileSystem, | ||
| Map<String, String> properties, | ||
| boolean useFileSizeFromMetadata, | ||
| ExecutorService deleteExecutor, | ||
| IcebergFileSystemFactory storageCredentialFileSystemFactory, | ||
| ConnectorIdentity storageCredentialIdentity) | ||
| { | ||
| this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); | ||
| this.deleteExecutor = requireNonNull(deleteExecutor, "executorService is null"); | ||
| this.properties = ImmutableMap.copyOf(requireNonNull(properties, "properties is null")); | ||
| this.useFileSizeFromMetadata = useFileSizeFromMetadata; | ||
| this.storageCredentialFileSystemFactory = storageCredentialFileSystemFactory; | ||
| this.storageCredentialIdentity = storageCredentialIdentity; | ||
| } | ||
|
|
||
| @Override | ||
| public InputFile newInputFile(String path) | ||
| { | ||
| return new ForwardingInputFile(fileSystem.newInputFile(Location.of(path))); | ||
| return new ForwardingInputFile(fileSystemForPath(path).newInputFile(Location.of(path))); | ||
| } | ||
|
|
||
| @Override | ||
| public InputFile newInputFile(String path, long length) | ||
| { | ||
| TrinoFileSystem fileSystem = fileSystemForPath(path); | ||
| if (!useFileSizeFromMetadata) { | ||
| return new ForwardingInputFile(fileSystem.newInputFile(Location.of(path))); | ||
| } | ||
|
|
@@ -88,14 +115,14 @@ public InputFile newInputFile(String path, long length) | |
| @Override | ||
| public OutputFile newOutputFile(String path) | ||
| { | ||
| return new ForwardingOutputFile(fileSystem, Location.of(path)); | ||
| return new ForwardingOutputFile(fileSystemForPath(path), Location.of(path)); | ||
| } | ||
|
|
||
| @Override | ||
| public void deleteFile(String path) | ||
| { | ||
| try { | ||
| fileSystem.deleteFile(Location.of(path)); | ||
| fileSystemForPath(path).deleteFile(Location.of(path)); | ||
| } | ||
| catch (IOException e) { | ||
| throw new UncheckedIOException("Failed to delete file: " + path, e); | ||
|
|
@@ -149,10 +176,24 @@ public InputFile newInputFile(DeleteFile file) | |
| return SupportsBulkOperations.super.newInputFile(file); | ||
| } | ||
|
|
||
| @Override | ||
| public InputFile newInputFile(ManifestListFile manifestList) | ||
| { | ||
| return SupportsBulkOperations.super.newInputFile(manifestList); | ||
| } | ||
|
|
||
| private void deleteBatch(List<String> filesToDelete) | ||
| { | ||
| try { | ||
| fileSystem.deleteFiles(filesToDelete.stream().map(Location::of).toList()); | ||
| Map<TrinoFileSystem, List<Location>> locationsByFileSystem = new IdentityHashMap<>(); | ||
| for (String path : filesToDelete) { | ||
| TrinoFileSystem fileSystem = fileSystemForPath(path); | ||
| locationsByFileSystem.computeIfAbsent(fileSystem, ignored -> new ArrayList<>()) | ||
| .add(Location.of(path)); | ||
| } | ||
| for (Map.Entry<TrinoFileSystem, List<Location>> entry : locationsByFileSystem.entrySet()) { | ||
| entry.getKey().deleteFiles(entry.getValue()); | ||
| } | ||
| } | ||
| catch (IOException e) { | ||
| throw new UncheckedIOException( | ||
|
|
@@ -172,6 +213,51 @@ public Map<String, String> properties() | |
| return properties; | ||
| } | ||
|
|
||
| @Override | ||
| public void setCredentials(List<StorageCredential> credentials) | ||
| { | ||
| storageCredentials = ImmutableList.copyOf(requireNonNull(credentials, "credentials is null")); | ||
| rebuildPrefixedFileSystems(); | ||
| } | ||
|
|
||
| @Override | ||
| public List<StorageCredential> credentials() | ||
| { | ||
| return storageCredentials; | ||
| } | ||
|
|
||
| private TrinoFileSystem fileSystemForPath(String path) | ||
| { | ||
| TrinoFileSystem matchingFileSystem = fileSystem; | ||
| int matchingPrefixLength = -1; | ||
| for (Map.Entry<String, TrinoFileSystem> prefixedFileSystem : prefixedFileSystems.entrySet()) { | ||
| String prefix = prefixedFileSystem.getKey(); | ||
| if (path.startsWith(prefix) && prefix.length() > matchingPrefixLength) { | ||
| matchingPrefixLength = prefix.length(); | ||
| matchingFileSystem = prefixedFileSystem.getValue(); | ||
| } | ||
| } | ||
| return matchingFileSystem; | ||
| } | ||
|
|
||
| private void rebuildPrefixedFileSystems() | ||
| { | ||
| if (storageCredentials.isEmpty() || storageCredentialFileSystemFactory == null || storageCredentialIdentity == null) { | ||
| prefixedFileSystems = ImmutableMap.of(); | ||
| return; | ||
| } | ||
|
|
||
| ImmutableMap.Builder<String, TrinoFileSystem> rebuiltFileSystems = ImmutableMap.builder(); | ||
| for (StorageCredential storageCredential : storageCredentials) { | ||
| Map<String, String> mergedProperties = ImmutableMap.<String, String>builder() | ||
| .putAll(properties) | ||
| .putAll(storageCredential.config()) | ||
| .buildKeepingLast(); | ||
| rebuiltFileSystems.put(storageCredential.prefix(), storageCredentialFileSystemFactory.create(storageCredentialIdentity, mergedProperties)); | ||
| } | ||
| prefixedFileSystems = rebuiltFileSystems.buildKeepingLast(); | ||
| } | ||
|
|
||
| @Override | ||
| public void initialize(Map<String, String> properties) | ||
| { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.