Skip to content
Closed
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 @@ -66,7 +66,6 @@
import org.apache.iceberg.io.FileIOParser;
import org.apache.iceberg.io.FileInfo;
import org.apache.iceberg.io.IOUtil;
import org.apache.iceberg.io.ImmutableStorageCredential;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.io.ResolvingFileIO;
Expand Down Expand Up @@ -641,89 +640,6 @@ public void noStorageCredentialConfigured() {
.isEqualTo("sessionTokenFromProperties");
}

@Test
public void singleStorageCredentialConfigured() {
StorageCredential s3Credential =
ImmutableStorageCredential.builder()
.prefix("s3://custom-uri")
.config(
ImmutableMap.of(
"s3.access-key-id",
"keyIdFromCredential",
"s3.secret-access-key",
"accessKeyFromCredential",
"s3.session-token",
"sessionTokenFromCredential"))
.build();

S3FileIO fileIO = new S3FileIO();
fileIO.setCredentials(ImmutableList.of(s3Credential));
fileIO.initialize(
ImmutableMap.of(
"s3.access-key-id",
"keyIdFromProperties",
"s3.secret-access-key",
"accessKeyFromProperties",
"s3.session-token",
"sessionTokenFromProperties"));

ObjectAssert<S3FileIOProperties> s3FileIOProperties =
assertThat(fileIO)
.extracting("s3FileIOProperties")
.asInstanceOf(InstanceOfAssertFactories.type(S3FileIOProperties.class));
s3FileIOProperties.extracting(S3FileIOProperties::accessKeyId).isEqualTo("keyIdFromCredential");
s3FileIOProperties
.extracting(S3FileIOProperties::secretAccessKey)
.isEqualTo("accessKeyFromCredential");
s3FileIOProperties
.extracting(S3FileIOProperties::sessionToken)
.isEqualTo("sessionTokenFromCredential");
}

@Test
public void multipleStorageCredentialsConfigured() {
StorageCredential s3Credential1 =
ImmutableStorageCredential.builder()
.prefix("s3://custom-uri/1")
.config(
ImmutableMap.of(
"s3.access-key-id",
"keyIdFromCredential1",
"s3.secret-access-key",
"accessKeyFromCredential1",
"s3.session-token",
"sessionTokenFromCredential1"))
.build();

StorageCredential s3Credential2 =
ImmutableStorageCredential.builder()
.prefix("s3://custom-uri/2")
.config(
ImmutableMap.of(
"s3.access-key-id",
"keyIdFromCredential2",
"s3.secret-access-key",
"accessKeyFromCredential2",
"s3.session-token",
"sessionTokenFromCredential2"))
.build();

S3FileIO fileIO = new S3FileIO();
fileIO.setCredentials(ImmutableList.of(s3Credential1, s3Credential2));
assertThatThrownBy(
() ->
fileIO.initialize(
ImmutableMap.of(
"s3.access-key-id",
"keyIdFromProperties",
"s3.secret-access-key",
"accessKeyFromProperties",
"s3.session-token",
"sessionTokenFromProperties")))
.isInstanceOf(IllegalStateException.class)
.hasMessage("Invalid S3 Credentials: only one S3 credential should exist");
}

private void createRandomObjects(String prefix, int count) {
S3URI s3URI = new S3URI(prefix);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ public S3Client s3() {
.applyMutation(s3FileIOProperties::applyServiceConfigurations)
.applyMutation(s3FileIOProperties::applySignerConfiguration)
.applyMutation(s3FileIOProperties::applyRetryConfigurations)
.applyMutation(s3FileIOProperties::applyStorageCredentialsInterceptor)
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ public S3Client s3() {
.applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations)
.applyMutation(s3FileIOProperties::applyUserAgentConfigurations)
.applyMutation(s3FileIOProperties::applyRetryConfigurations)
.applyMutation(s3FileIOProperties::applyStorageCredentialsInterceptor)
.build();
}

Expand All @@ -133,6 +134,7 @@ public S3AsyncClient s3Async() {
.applyMutation(awsClientProperties::applyClientRegionConfiguration)
.applyMutation(awsClientProperties::applyClientCredentialConfigurations)
.applyMutation(s3FileIOProperties::applyEndpointConfigurations)
.applyMutation(s3FileIOProperties::applyStorageCredentialsInterceptor)
.build();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.aws.s3;

import java.time.Instant;
import java.util.List;
import java.util.Map;
import org.apache.iceberg.io.StorageCredential;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
import software.amazon.awssdk.http.SdkHttpRequest;

public class S3ExecutionInterceptor implements ExecutionInterceptor {
private final List<StorageCredential> storageCredentials;

public S3ExecutionInterceptor(List<StorageCredential> storageCredentials) {
Preconditions.checkArgument(null != storageCredentials, "Invalid storage credentials: null");
this.storageCredentials = storageCredentials;
}

@Override
public SdkHttpRequest modifyHttpRequest(
Context.ModifyHttpRequest context, ExecutionAttributes executionAttributes) {
if (!storageCredentials.isEmpty()) {
SdkHttpRequest.Builder builder = context.httpRequest().toBuilder();
String requestPath = context.httpRequest().encodedPath();
StorageCredential credentialToUse = storageCredentials.get(0);
Map<String, String> configToUse = null;

for (StorageCredential storageCredential : storageCredentials) {
if (requestPath.startsWith(storageCredential.prefix())
&& storageCredential.prefix().length() >= credentialToUse.prefix().length()) {
configToUse = storageCredential.config();
}
}

if (null != configToUse) {
String accessKeyId = configToUse.get(S3FileIOProperties.ACCESS_KEY_ID);
String secretAccessKey = configToUse.get(S3FileIOProperties.SECRET_ACCESS_KEY);
String sessionToken = configToUse.get(S3FileIOProperties.SESSION_TOKEN);
String tokenExpiresAtMillis =
configToUse.get(S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS);

if (null != accessKeyId && null != secretAccessKey) {
builder.putHeader("X-Amz-Access-Key", accessKeyId);
builder.putHeader("X-Amz-Secret-Key", secretAccessKey);
if (null != sessionToken) {
builder.putHeader("X-Amz-Security-Token", sessionToken);
}

if (null != tokenExpiresAtMillis) {
Instant expiresAt = Instant.ofEpochMilli(Long.parseLong(tokenExpiresAtMillis));
long expiresInSeconds =
Math.max(1, expiresAt.minusMillis(Instant.now().toEpochMilli()).getEpochSecond());
builder.putHeader("X-Amz-Expires", Long.toString(expiresInSeconds));
}
}

return builder.build();
}
}

return ExecutionInterceptor.super.modifyHttpRequest(context, executionAttributes);
}
}
35 changes: 15 additions & 20 deletions aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -432,21 +432,27 @@ public String getCredential() {
@Override
public void initialize(Map<String, String> props) {
this.properties = SerializableMap.copyOf(props);
Map<String, String> propertiesWithCredentials =
ImmutableMap.<String, String>builder()
.putAll(properties)
.putAll(storageCredentialConfig())
.buildKeepingLast();

this.s3FileIOProperties = new S3FileIOProperties(propertiesWithCredentials);
this.s3FileIOProperties = new S3FileIOProperties(properties);
this.createStack =
PropertyUtil.propertyAsBoolean(props, "init-creation-stacktrace", true)
? Thread.currentThread().getStackTrace()
: null;

Map<String, String> propertiesWithStorageCredentials =
ImmutableMap.<String, String>builder()
.putAll(properties)
.putAll(
StorageCredential.toMap(
storageCredentials.stream()
.filter(c -> c.prefix().startsWith("s3"))
.collect(Collectors.toList())))
.build();

// Do not override s3 client if it was provided
if (s3 == null) {
Object clientFactory = S3FileIOAwsClientFactories.initialize(props);
Object clientFactory =
S3FileIOAwsClientFactories.initialize(propertiesWithStorageCredentials);
if (clientFactory instanceof S3FileIOAwsClientFactory) {
this.s3 = ((S3FileIOAwsClientFactory) clientFactory)::s3;
}
Expand All @@ -463,7 +469,8 @@ public void initialize(Map<String, String> props) {

// Do not override s3Async client if it was provided
if (s3Async == null) {
Object clientFactory = S3FileIOAwsClientFactories.initialize(props);
Object clientFactory =
S3FileIOAwsClientFactories.initialize(propertiesWithStorageCredentials);
if (clientFactory instanceof S3FileIOAwsClientFactory) {
this.s3Async = ((S3FileIOAwsClientFactory) clientFactory)::s3Async;
}
Expand Down Expand Up @@ -575,16 +582,4 @@ public void setCredentials(List<StorageCredential> credentials) {
public List<StorageCredential> credentials() {
return ImmutableList.copyOf(storageCredentials);
}

private Map<String, String> storageCredentialConfig() {
List<StorageCredential> s3Credentials =
storageCredentials.stream()
.filter(c -> c.prefix().startsWith("s3"))
.collect(Collectors.toList());

Preconditions.checkState(
s3Credentials.size() <= 1, "Invalid S3 Credentials: only one S3 credential should exist");

return s3Credentials.isEmpty() ? Map.of() : s3Credentials.get(0).config();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.net.URI;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
Expand All @@ -32,12 +33,14 @@
import org.apache.iceberg.aws.s3.signer.S3V4RestSignerClient;
import org.apache.iceberg.common.DynMethods;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.io.StorageCredential;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.SerializableMap;
import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
import software.amazon.awssdk.core.exception.SdkServiceException;
Expand Down Expand Up @@ -531,6 +534,7 @@ public class S3FileIOProperties implements Serializable {
private int s3RetryNumRetries;
private long s3RetryMinWaitMs;
private long s3RetryMaxWaitMs;
private List<StorageCredential> storageCredentials;

private boolean s3DirectoryBucketListPrefixAsDirectory;
private final Map<String, String> allProperties;
Expand Down Expand Up @@ -576,6 +580,7 @@ public S3FileIOProperties() {
this.isS3CRTEnabled = S3_CRT_ENABLED_DEFAULT;
this.s3CrtMaxConcurrency = S3_CRT_MAX_CONCURRENCY_DEFAULT;
this.allProperties = Maps.newHashMap();
this.storageCredentials = List.of();

ValidationException.check(
keyIdAccessKeyBothConfigured(),
Expand Down Expand Up @@ -698,6 +703,8 @@ public S3FileIOProperties(Map<String, String> properties) {
PropertyUtil.propertyAsInt(
properties, S3_CRT_MAX_CONCURRENCY, S3_CRT_MAX_CONCURRENCY_DEFAULT);

this.storageCredentials = StorageCredential.fromMap(properties);

ValidationException.check(
keyIdAccessKeyBothConfigured(),
"S3 client access key ID and secret access key must be set at the same time");
Expand Down Expand Up @@ -1011,6 +1018,19 @@ public <T extends S3ClientBuilder> void applySignerConfiguration(T builder) {
}
}

public <T extends AwsClientBuilder> void applyStorageCredentialsInterceptor(T builder) {
if (!storageCredentials.isEmpty()) {
ClientOverrideConfiguration.Builder configBuilder =
null != builder.overrideConfiguration()
? builder.overrideConfiguration().toBuilder()
: ClientOverrideConfiguration.builder();
builder.overrideConfiguration(
configBuilder
.addExecutionInterceptor(new S3ExecutionInterceptor(storageCredentials))
.build());
}
}

/**
* Override the endpoint for an S3 client.
*
Expand Down
50 changes: 50 additions & 0 deletions core/src/main/java/org/apache/iceberg/io/StorageCredential.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,14 @@
package org.apache.iceberg.io;

import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.PropertyUtil;
import org.immutables.value.Value;

@Value.Immutable
Expand All @@ -39,4 +45,48 @@ default void validate() {
static StorageCredential create(String prefix, Map<String, String> config) {
return ImmutableStorageCredential.builder().prefix(prefix).config(config).build();
}

static Map<String, String> toMap(List<StorageCredential> credentials) {
Preconditions.checkArgument(null != credentials, "Invalid storage credentials: null");
Map<String, String> config = Maps.newHashMap();
for (StorageCredential credential : credentials) {
String key = "storage-credential." + credential.prefix();
config.put(key, credential.prefix());
credential
.config()
.forEach(
(k, v) -> {
String innerKey = key + "." + k;
config.put(innerKey, v);
});
}

return config;
}

static List<StorageCredential> fromMap(Map<String, String> map) {
Preconditions.checkArgument(null != map, "Invalid storage credentials config: null");
Map<String, String> config = PropertyUtil.propertiesWithPrefix(map, "storage-credential.");
List<StorageCredential> credentials = Lists.newArrayList();

Set<String> prefixes =
config.entrySet().stream()
.filter(entry -> entry.getKey().equals(entry.getValue()))
.map(Map.Entry::getKey)
.collect(Collectors.toSet());

prefixes.forEach(
prefix -> {
Map<String, String> configForCredential =
PropertyUtil.propertiesWithPrefix(config, prefix + ".");
Preconditions.checkArgument(
!configForCredential.isEmpty(),
"Invalid storage credential config with prefix %s: null or empty",
prefix);
StorageCredential credential = create(prefix, configForCredential);
credentials.add(credential);
});

return credentials;
}
}
Loading