You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/11/15 17:24:23 UTC

[GitHub] [iceberg] nastra opened a new pull request, #6169: AWS,Core: Add S3 REST Signer client + REST Spec

nastra opened a new pull request, #6169:
URL: https://github.com/apache/iceberg/pull/6169

   This introduces an S3 REST signer client and defines a REST spec (`s3-signer-open-api.yml`) for a server implementation. Below is a high-level overview of the introduced changes:
   
   * the main logic and functionality resides in the `S3V4RestSignerClient` class
     * it uses the same **credential/token** exchange flow as we have in `RESTSessionCatalog` and also uses the same token refresh mechanism. In order to achieve that, a few refactorings have been done in `RESTSessionCatalog` / `OAuth2Util`.
     * the default endpoint the signer connects to is `v1/aws/s3/sign` but can be customized.
     * The server decides which headers to sign and can indicate to the `S3V4RestSignerClient` whether a response with signed headers can be cached by sending a `Cache-Control: private` header
   * `AwsProperties` introduce `s3.signer.class` that allows to dynamically load an S3 Signer implementation and apply it when creating an S3 client. This can be any Signer class that implements `software.amazon.awssdk.core.signer.Signer`.
   * `S3SignRequest` and `S3SignResponse` classes define how the request and response looks like
   * an `S3ObjectMapper` class has been introduced that is similar to `RESTObjectMapper` but only contains what's necessary for the S3 REST signer, which are the request/response classes with OAuth-related classes and error handling.
   * Testing is done by using `MinioContainer` (`TestContainers` + `MinIO`) in `TestS3RestSigner`
     * The `S3SignerServlet` defines the minimum amount of work that a server-side implementation might have. It is by no means complete and only serves the purpose of testing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030817942


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();

Review Comment:
   This duplicates the handling below. Can you refactor to avoid the duplication? Something like this:
   
   ```java
     SignedComponent signed;
     if (cachedSignedComponent != null) {
       signed = cachedSignedComponent;
     } else {
       S3SignResponse s3SignResponse = httpClient().post(...)
       signed = ImmutableSignedComponent.builder()...build()
     }
   
     SdkHttpFullRequest.Builder request ...
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031603431


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();

Review Comment:
   good point, I've updated this part to make it easier to read



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093810144


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("create")
+              .impl(s3SignerImpl, Map.class)
+              .buildStaticChecked()
+              .invoke(allProperties);
+    } catch (NoSuchMethodException e) {

Review Comment:
   I've removed the nesting and added logging as you suggested.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093370721


##########
aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.signer;
+
+import static java.lang.String.format;
+import static org.apache.iceberg.rest.RESTCatalogAdapter.castRequest;
+import static org.apache.iceberg.rest.RESTCatalogAdapter.castResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.time.Clock;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTUtil;
+import org.apache.iceberg.rest.ResourcePaths;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.AwsS3V4Signer;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+import software.amazon.awssdk.http.SdkHttpMethod;
+import software.amazon.awssdk.regions.Region;
+
+/**
+ * The {@link S3V4RestSignerClient} performs OAuth and S3 sign requests against a REST server. The
+ * {@link S3SignerServlet} provides a simple servlet implementation to emulate the server-side
+ * behavior of signing S3 requests and handling OAuth.
+ */
+public class S3SignerServlet extends HttpServlet {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3SignerServlet.class);
+
+  static final Clock SIGNING_CLOCK = Clock.fixed(Instant.now(), ZoneId.of("UTC"));
+  static final Set<String> UNSIGNED_HEADERS =
+      Sets.newHashSet(
+          Arrays.asList("range", "x-amz-date", "amz-sdk-invocation-id", "amz-sdk-retry"));
+  private static final String POST = "POST";
+
+  private static final Set<SdkHttpMethod> CACHEABLE_METHODS =
+      Stream.of(SdkHttpMethod.GET, SdkHttpMethod.HEAD).collect(Collectors.toSet());
+
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+  private final ObjectMapper mapper;
+
+  public S3SignerServlet(ObjectMapper mapper) {
+    this.mapper = mapper;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  private OAuthTokenResponse handleOAuth(Map<String, String> requestMap) {
+    String grantType = requestMap.get("grant_type");
+    switch (grantType) {
+      case "client_credentials":
+        return castResponse(
+            OAuthTokenResponse.class,
+            OAuthTokenResponse.builder()
+                .withToken("client-credentials-token:sub=" + requestMap.get("client_id"))
+                .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token")
+                .withTokenType("Bearer")
+                .build());
+
+      case "urn:ietf:params:oauth:grant-type:token-exchange":
+        String actor = requestMap.get("actor_token");
+        String token =
+            String.format(
+                "token-exchange-token:sub=%s%s",
+                requestMap.get("subject_token"), actor != null ? ",act=" + actor : "");
+        return castResponse(
+            OAuthTokenResponse.class,
+            OAuthTokenResponse.builder()
+                .withToken(token)
+                .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token")
+                .withTokenType("Bearer")
+                .build());
+
+      default:
+        throw new UnsupportedOperationException("Unsupported grant_type: " + grantType);
+    }
+  }
+
+  private S3SignResponse signRequest(S3SignRequest request) {
+    AwsS3V4SignerParams signingParams =
+        AwsS3V4SignerParams.builder()
+            .awsCredentials(TestS3RestSigner.CREDENTIALS_PROVIDER.resolveCredentials())
+            .enablePayloadSigning(false)
+            .signingClockOverride(SIGNING_CLOCK)
+            .enableChunkedEncoding(false)
+            .signingRegion(Region.of(request.region()))
+            .doubleUrlEncode(false)
+            .timeOffset(0)
+            .signingName("s3")
+            .build();
+
+    Map<String, List<String>> unsignedHeaders =
+        request.headers().entrySet().stream()
+            .filter(e -> UNSIGNED_HEADERS.contains(e.getKey().toLowerCase()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    Map<String, List<String>> signedHeaders =
+        request.headers().entrySet().stream()
+            .filter(e -> !UNSIGNED_HEADERS.contains(e.getKey().toLowerCase()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    SdkHttpFullRequest sign =
+        AwsS3V4Signer.create()
+            .sign(
+                SdkHttpFullRequest.builder()
+                    .uri(request.uri())
+                    .method(SdkHttpMethod.fromValue(request.method()))
+                    .headers(signedHeaders)
+                    .build(),
+                signingParams);
+
+    Map<String, List<String>> headers = Maps.newHashMap(sign.headers());
+    headers.putAll(unsignedHeaders);
+
+    return ImmutableS3SignResponse.builder().uri(request.uri()).headers(headers).build();
+  }
+
+  protected void execute(HttpServletRequest request, HttpServletResponse response) {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    String path = request.getRequestURI().substring(1);
+    Object requestBody;
+    try {
+      // we only need to handle oauth tokens & s3 sign request routes here as those are the only
+      // requests that are being done by the S3V4RestSignerClient
+      if (POST.equals(request.getMethod())
+          && S3V4RestSignerClient.S3_SIGNER_DEFAULT_ENDPOINT.equals(path)) {
+        S3SignRequest s3SignRequest =
+            castRequest(
+                S3SignRequest.class, mapper.readValue(request.getReader(), S3SignRequest.class));
+        S3SignResponse s3SignResponse = signRequest(s3SignRequest);
+        if (CACHEABLE_METHODS.contains(SdkHttpMethod.fromValue(s3SignRequest.method()))) {
+          // tell the client this can be cached
+          response.setHeader(
+              S3V4RestSignerClient.CACHE_CONTROL, S3V4RestSignerClient.CACHE_CONTROL_PRIVATE);
+        } else {
+          response.setHeader(
+              S3V4RestSignerClient.CACHE_CONTROL, S3V4RestSignerClient.CACHE_CONTROL_NO_CACHE);
+        }
+
+        mapper.writeValue(response.getWriter(), s3SignResponse);
+      } else if (POST.equals(request.getMethod()) && ResourcePaths.tokens().equals(path)) {
+        try (Reader reader = new InputStreamReader(request.getInputStream())) {
+          requestBody = RESTUtil.decodeFormData(CharStreams.toString(reader));
+        }

Review Comment:
   updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031561052


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   I've tried to outline in https://github.com/apache/iceberg/pull/6169#discussion_r1031555625 why I'd like to keep this an Immutable class. Given that an Immutable class doesn't provide a no-arg constructor (at least not out-of-the-box), we can't use our conventional way of dynamically loading this class (no-arg constructor + initialize method) unfortunately



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#issuecomment-1323218731

   I have moved the S3 Signer REST Spec to the `iceberg-aws` module.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030819675


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();
+    }
+
+    S3SignResponse s3SignResponse =
+        httpClient()
+            .post(
+                endpoint(),
+                remoteSigningRequest,
+                S3SignResponse.class,
+                () -> OAuth2Util.authHeaders(authSession().token()),
+                ErrorHandlers.defaultErrorHandler());
+
+    if (canBeCached(s3SignResponse)) {
+      SIGNED_COMPONENT_CACHE.put(
+          cacheKey,
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build());
+    }
+
+    // Update URI and append signed and unsigned headers
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(s3SignResponse.uri());
+    reconstructHeaders(s3SignResponse.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(S3SignResponse s3SignResponse) {
+    List<String> cacheControlHeader = s3SignResponse.headers().get(CACHE_CONTROL);

Review Comment:
   Shouldn't cache control be specified in the response headers, not the headers included in the response?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1035245025


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();

Review Comment:
   This should be `expireAfterWrite` so that repeated calls don't result in signature timeouts.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1069231526


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();
+    }
+
+    S3SignResponse s3SignResponse =
+        httpClient()
+            .post(
+                endpoint(),
+                remoteSigningRequest,
+                S3SignResponse.class,
+                () -> OAuth2Util.authHeaders(authSession().token()),
+                ErrorHandlers.defaultErrorHandler());
+
+    if (canBeCached(s3SignResponse)) {
+      SIGNED_COMPONENT_CACHE.put(
+          cacheKey,
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build());
+    }
+
+    // Update URI and append signed and unsigned headers
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(s3SignResponse.uri());
+    reconstructHeaders(s3SignResponse.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(S3SignResponse s3SignResponse) {
+    List<String> cacheControlHeader = s3SignResponse.headers().get(CACHE_CONTROL);

Review Comment:
   @rdblue I've opened https://github.com/apache/iceberg/pull/6578 to have a way to access the response headers via the `HTTPClient`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] bryanck commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
bryanck commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1044555925


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public String baseSignerUri() {
+    return properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = new AuthSession(OAuth2Util.authHeaders(token()), null, null);
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");

Review Comment:
   If I'm reading this right, it looks like this could end up passing an expired token in the authorization header. For example, if a long running Flink task restarts and the AWS properties contains a token, the task will be restored with a potentially expired token. Then this will be passed in the auth header for the client credentials OAuth call.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030811542


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");

Review Comment:
   I think that this isn't supported. We could extend the signer to support pre-signing.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030907471


##########
core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java:
##########
@@ -381,5 +394,42 @@ public Pair<Integer, TimeUnit> refresh(RESTClient client) {
 
       return null;
     }
+
+    @SuppressWarnings("FutureReturnValueIgnored")

Review Comment:
   I see it was there before. I don't remember seeing this though. Anyway, it's fine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1023007309


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -261,6 +267,8 @@ public class AwsProperties implements Serializable {
 
   public static final boolean S3_CHECKSUM_ENABLED_DEFAULT = false;
 
+  public static final String S3_SIGNER_CLASS = "s3.signer.class";

Review Comment:
   makes sense, updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1091029462


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();

Review Comment:
   Just curious, is it worth making the cache expiration and size configurable?



##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+  private static final String SCOPE = "sign";
+  private static RESTClient httpClient;
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public String baseSignerUri() {
+    return properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  private RESTClient httpClient() {
+    if (null == httpClient) {
+      // TODO: should be closed
+      httpClient =
+          HTTPClient.builder()
+              .uri(baseSignerUri())
+              .withObjectMapper(S3ObjectMapper.mapper())
+              .build();
+    }
+
+    return httpClient;
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    if (null != token()) {
+      return AuthSession.fromAccessToken(
+          httpClient(),
+          TOKEN_REFRESH_EXECUTOR,
+          token(),
+          expiresAtMillis(properties()),
+          new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE));
+    }
+
+    if (credentialProvided()) {
+      AuthSession session = new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE);
+      long startTimeMillis = System.currentTimeMillis();
+      OAuthTokenResponse authResponse =
+          OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), SCOPE);
+      return AuthSession.fromTokenResponse(
+          httpClient(), TOKEN_REFRESH_EXECUTOR, authResponse, startTimeMillis, session);
+    }
+
+    return AuthSession.empty();
+  }
+
+  private boolean credentialProvided() {
+    return null != credential() && !credential().isEmpty();
+  }
+
+  private Long expiresAtMillis(Map<String, String> properties) {
+    if (properties.containsKey(OAuth2Properties.TOKEN_EXPIRES_IN_MS)) {
+      long expiresInMillis =
+          PropertyUtil.propertyAsLong(
+              properties,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+      return System.currentTimeMillis() + expiresInMillis;
+    } else {
+      return null;
+    }
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+    SignedComponent signedComponent;
+
+    if (null != cachedSignedComponent) {
+      signedComponent = cachedSignedComponent;
+    } else {
+      Map<String, String> responseHeaders = Maps.newHashMap();
+      Consumer<Map<String, String>> responseHeadersConsumer = responseHeaders::putAll;
+      S3SignResponse s3SignResponse =
+          httpClient()
+              .post(
+                  endpoint(),
+                  remoteSigningRequest,
+                  S3SignResponse.class,
+                  () -> authSession().headers(),
+                  ErrorHandlers.defaultErrorHandler(),
+                  responseHeadersConsumer);
+
+      signedComponent =
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build();
+
+      if (canBeCached(responseHeaders)) {
+        SIGNED_COMPONENT_CACHE.put(cacheKey, signedComponent);
+      }
+    }
+
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(signedComponent.signedURI());
+    reconstructHeaders(signedComponent.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(Map<String, String> responseHeaders) {
+    return responseHeaders.containsKey(CACHE_CONTROL)
+        && CACHE_CONTROL_PRIVATE.equals(responseHeaders.get(CACHE_CONTROL));

Review Comment:
   Nit can just do
   
   ```
   CACHE_CONTROL_PRIVATE.equals(responseHeaders.get(CACHE_CONTROL))
   ```



##########
aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.signer;
+
+import static java.lang.String.format;
+import static org.apache.iceberg.rest.RESTCatalogAdapter.castRequest;
+import static org.apache.iceberg.rest.RESTCatalogAdapter.castResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.time.Clock;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTUtil;
+import org.apache.iceberg.rest.ResourcePaths;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.AwsS3V4Signer;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+import software.amazon.awssdk.http.SdkHttpMethod;
+import software.amazon.awssdk.regions.Region;
+
+/**
+ * The {@link S3V4RestSignerClient} performs OAuth and S3 sign requests against a REST server. The
+ * {@link S3SignerServlet} provides a simple servlet implementation to emulate the server-side
+ * behavior of signing S3 requests and handling OAuth.
+ */
+public class S3SignerServlet extends HttpServlet {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3SignerServlet.class);
+
+  static final Clock SIGNING_CLOCK = Clock.fixed(Instant.now(), ZoneId.of("UTC"));
+  static final Set<String> UNSIGNED_HEADERS =
+      Sets.newHashSet(
+          Arrays.asList("range", "x-amz-date", "amz-sdk-invocation-id", "amz-sdk-retry"));
+  private static final String POST = "POST";
+
+  private static final Set<SdkHttpMethod> CACHEABLE_METHODS =
+      Stream.of(SdkHttpMethod.GET, SdkHttpMethod.HEAD).collect(Collectors.toSet());
+
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+  private final ObjectMapper mapper;
+
+  public S3SignerServlet(ObjectMapper mapper) {
+    this.mapper = mapper;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response) {
+    execute(request, response);
+  }
+
+  private OAuthTokenResponse handleOAuth(Map<String, String> requestMap) {
+    String grantType = requestMap.get("grant_type");
+    switch (grantType) {
+      case "client_credentials":
+        return castResponse(
+            OAuthTokenResponse.class,
+            OAuthTokenResponse.builder()
+                .withToken("client-credentials-token:sub=" + requestMap.get("client_id"))
+                .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token")
+                .withTokenType("Bearer")
+                .build());
+
+      case "urn:ietf:params:oauth:grant-type:token-exchange":
+        String actor = requestMap.get("actor_token");
+        String token =
+            String.format(
+                "token-exchange-token:sub=%s%s",
+                requestMap.get("subject_token"), actor != null ? ",act=" + actor : "");
+        return castResponse(
+            OAuthTokenResponse.class,
+            OAuthTokenResponse.builder()
+                .withToken(token)
+                .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token")
+                .withTokenType("Bearer")
+                .build());
+
+      default:
+        throw new UnsupportedOperationException("Unsupported grant_type: " + grantType);
+    }
+  }
+
+  private S3SignResponse signRequest(S3SignRequest request) {
+    AwsS3V4SignerParams signingParams =
+        AwsS3V4SignerParams.builder()
+            .awsCredentials(TestS3RestSigner.CREDENTIALS_PROVIDER.resolveCredentials())
+            .enablePayloadSigning(false)
+            .signingClockOverride(SIGNING_CLOCK)
+            .enableChunkedEncoding(false)
+            .signingRegion(Region.of(request.region()))
+            .doubleUrlEncode(false)
+            .timeOffset(0)
+            .signingName("s3")
+            .build();
+
+    Map<String, List<String>> unsignedHeaders =
+        request.headers().entrySet().stream()
+            .filter(e -> UNSIGNED_HEADERS.contains(e.getKey().toLowerCase()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    Map<String, List<String>> signedHeaders =
+        request.headers().entrySet().stream()
+            .filter(e -> !UNSIGNED_HEADERS.contains(e.getKey().toLowerCase()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    SdkHttpFullRequest sign =
+        AwsS3V4Signer.create()
+            .sign(
+                SdkHttpFullRequest.builder()
+                    .uri(request.uri())
+                    .method(SdkHttpMethod.fromValue(request.method()))
+                    .headers(signedHeaders)
+                    .build(),
+                signingParams);
+
+    Map<String, List<String>> headers = Maps.newHashMap(sign.headers());
+    headers.putAll(unsignedHeaders);
+
+    return ImmutableS3SignResponse.builder().uri(request.uri()).headers(headers).build();
+  }
+
+  protected void execute(HttpServletRequest request, HttpServletResponse response) {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    String path = request.getRequestURI().substring(1);
+    Object requestBody;
+    try {
+      // we only need to handle oauth tokens & s3 sign request routes here as those are the only
+      // requests that are being done by the S3V4RestSignerClient
+      if (POST.equals(request.getMethod())
+          && S3V4RestSignerClient.S3_SIGNER_DEFAULT_ENDPOINT.equals(path)) {
+        S3SignRequest s3SignRequest =
+            castRequest(
+                S3SignRequest.class, mapper.readValue(request.getReader(), S3SignRequest.class));
+        S3SignResponse s3SignResponse = signRequest(s3SignRequest);
+        if (CACHEABLE_METHODS.contains(SdkHttpMethod.fromValue(s3SignRequest.method()))) {
+          // tell the client this can be cached
+          response.setHeader(
+              S3V4RestSignerClient.CACHE_CONTROL, S3V4RestSignerClient.CACHE_CONTROL_PRIVATE);
+        } else {
+          response.setHeader(
+              S3V4RestSignerClient.CACHE_CONTROL, S3V4RestSignerClient.CACHE_CONTROL_NO_CACHE);
+        }
+
+        mapper.writeValue(response.getWriter(), s3SignResponse);
+      } else if (POST.equals(request.getMethod()) && ResourcePaths.tokens().equals(path)) {
+        try (Reader reader = new InputStreamReader(request.getInputStream())) {
+          requestBody = RESTUtil.decodeFormData(CharStreams.toString(reader));
+        }

Review Comment:
   Nit: Newline after the try block



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1035685517


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();

Review Comment:
   makes sense, updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] bryanck commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
bryanck commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1054650368


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public String baseSignerUri() {
+    return properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    // we assume that the initially provided token is already expired, so we trigger an
+    // immediate refresh
+    AuthSession session = newSessionWithImmediateRefresh(token());
+
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+    SignedComponent signedComponent;
+
+    if (null != cachedSignedComponent) {
+      signedComponent = cachedSignedComponent;
+    } else {
+      S3SignResponse s3SignResponse =
+          httpClient()
+              .post(
+                  endpoint(),
+                  remoteSigningRequest,
+                  S3SignResponse.class,
+                  () -> authSession().headers(),
+                  ErrorHandlers.defaultErrorHandler());
+
+      signedComponent =
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build();
+
+      if (canBeCached(s3SignResponse)) {
+        SIGNED_COMPONENT_CACHE.put(cacheKey, signedComponent);
+      }
+    }
+
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(signedComponent.signedURI());
+    reconstructHeaders(signedComponent.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(S3SignResponse s3SignResponse) {
+    List<String> cacheControlHeader = s3SignResponse.headers().get(CACHE_CONTROL);
+    if (null != cacheControlHeader) {
+      LOG.debug("Received {} header: {}", CACHE_CONTROL, cacheControlHeader);
+
+      boolean canBeCached = cacheControlHeader.contains(CACHE_CONTROL_PRIVATE);
+      if (!canBeCached && !cacheControlHeader.contains(CACHE_CONTROL_NO_CACHE)) {
+        LOG.warn("Unsupported {} header: {}", CACHE_CONTROL, cacheControlHeader);
+      }
+      return canBeCached;
+    }
+    return false;
+  }
+
+  private AuthSession newSession(String token, AuthSession parent) {
+    AuthSession session =
+        new AuthSession(parent.headers(), token, OAuth2Properties.ACCESS_TOKEN_TYPE);
+    AuthSession.scheduleTokenRefresh(
+        httpClient(),
+        TOKEN_REFRESH_EXECUTOR,
+        session,
+        System.currentTimeMillis(),
+        tokenExpiresInMs(),
+        TimeUnit.MILLISECONDS);
+    return session;
+  }
+
+  private AuthSession newSession(OAuthTokenResponse response, AuthSession parent) {
+    AuthSession session =
+        new AuthSession(parent.headers(), response.token(), response.issuedTokenType());
+    if (response.expiresInSeconds() != null) {
+      AuthSession.scheduleTokenRefresh(
+          httpClient(),
+          TOKEN_REFRESH_EXECUTOR,
+          session,
+          System.currentTimeMillis(),
+          response.expiresInSeconds(),
+          TimeUnit.SECONDS);
+    }
+    return session;
+  }
+
+  private AuthSession newSessionWithImmediateRefresh(String token) {
+    AuthSession session =
+        new AuthSession(ImmutableMap.of(), token, OAuth2Properties.ACCESS_TOKEN_TYPE);
+    if (null != token) {
+      AuthSession.scheduleTokenRefresh(
+          httpClient(),
+          TOKEN_REFRESH_EXECUTOR,
+          session,
+          System.currentTimeMillis(),
+          0,
+          TimeUnit.MILLISECONDS);
+    }
+
+    return session;

Review Comment:
   If the token is expired (e.g. a task is restored in Flink), then it looks like this will create a session with an expired token, and also schedule a refresh of the expired token, which should fail.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030807246


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   I think we should follow the standard conventions for dynamic loading. This should always call the no-arg constructor and then call `initialize` with the properties map. If we don't have an interface with the `initialize` method, then it can be called dynamically as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031640368


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();

Review Comment:
   good catch, fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#issuecomment-1317273978

   > 1 general question regarding this PR, @nastra @rdblue @danielcweeks this is a feature very specific to AWS S3. What is the general guideline in the community for adding this as a part of the OpenAPI spec? In my mind the spec should be something generic for Iceberg, and not related to any specific cloud provider. Although many cloud storage providers have built S3 compatible API, but mostly just do it to some extent to support basic operations like get and put object. I would imaging signing with sigv4 is still a very AWS-specific thing, but maybe I am wrong about that.
   > 
   > Overall, have we thought about making this spec more generic, or is that not possible to achieve without being cloud provider specific?
   
   @jackye1995 I think the purpose behind the open-api doc is just to be clear about what the expectations are from a call/response/error perspective.  While it has some overlap with the other open-api doc, they are separate entities.
   
   This signer is specific to AWS and I think that's fine since the signer implementation is in the `aws` module.  It may make more sense to put the open-api doc in the `aws` module since it really only applies there (@nastra was going to look into that option).
   
   As a general pattern, all cloud providers sign requests in some form and we may see similar implementations using different protocols in the future, but it may also be that this can be generalized in the future.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093367675


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+  private static final String SCOPE = "sign";
+  private static RESTClient httpClient;
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public String baseSignerUri() {
+    return properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  private RESTClient httpClient() {
+    if (null == httpClient) {
+      // TODO: should be closed
+      httpClient =
+          HTTPClient.builder()
+              .uri(baseSignerUri())
+              .withObjectMapper(S3ObjectMapper.mapper())
+              .build();
+    }
+
+    return httpClient;
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    if (null != token()) {
+      return AuthSession.fromAccessToken(
+          httpClient(),
+          TOKEN_REFRESH_EXECUTOR,
+          token(),
+          expiresAtMillis(properties()),
+          new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE));
+    }
+
+    if (credentialProvided()) {
+      AuthSession session = new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE);
+      long startTimeMillis = System.currentTimeMillis();
+      OAuthTokenResponse authResponse =
+          OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), SCOPE);
+      return AuthSession.fromTokenResponse(
+          httpClient(), TOKEN_REFRESH_EXECUTOR, authResponse, startTimeMillis, session);
+    }
+
+    return AuthSession.empty();
+  }
+
+  private boolean credentialProvided() {
+    return null != credential() && !credential().isEmpty();
+  }
+
+  private Long expiresAtMillis(Map<String, String> properties) {
+    if (properties.containsKey(OAuth2Properties.TOKEN_EXPIRES_IN_MS)) {
+      long expiresInMillis =
+          PropertyUtil.propertyAsLong(
+              properties,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+      return System.currentTimeMillis() + expiresInMillis;
+    } else {
+      return null;
+    }
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+    SignedComponent signedComponent;
+
+    if (null != cachedSignedComponent) {
+      signedComponent = cachedSignedComponent;
+    } else {
+      Map<String, String> responseHeaders = Maps.newHashMap();
+      Consumer<Map<String, String>> responseHeadersConsumer = responseHeaders::putAll;
+      S3SignResponse s3SignResponse =
+          httpClient()
+              .post(
+                  endpoint(),
+                  remoteSigningRequest,
+                  S3SignResponse.class,
+                  () -> authSession().headers(),
+                  ErrorHandlers.defaultErrorHandler(),
+                  responseHeadersConsumer);
+
+      signedComponent =
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build();
+
+      if (canBeCached(responseHeaders)) {
+        SIGNED_COMPONENT_CACHE.put(cacheKey, signedComponent);
+      }
+    }
+
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(signedComponent.signedURI());
+    reconstructHeaders(signedComponent.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(Map<String, String> responseHeaders) {
+    return responseHeaders.containsKey(CACHE_CONTROL)
+        && CACHE_CONTROL_PRIVATE.equals(responseHeaders.get(CACHE_CONTROL));
+  }
+
+  private void checkSignerParams(AwsS3V4SignerParams signerParams) {
+    if (signerParams.enablePayloadSigning()) {
+      throw new UnsupportedOperationException("Payload signing not supported");
+    }
+
+    if (signerParams.enableChunkedEncoding()) {
+      throw new UnsupportedOperationException("Chunked encoding not supported");
+    }
+  }
+
+  @Value.Immutable
+  interface Key {
+    String method();
+
+    String region();
+
+    String uri();
+
+    static Key from(S3SignRequest request) {
+      return ImmutableKey.builder()
+          .method(request.method())
+          .region(request.region())
+          .uri(request.uri().toString())
+          .build();
+    }
+  }
+
+  @Value.Immutable
+  interface SignedComponent {
+    Map<String, List<String>> headers();
+
+    URI signedURI();
+  }
+
+  public static S3V4RestSignerClient of(Map<String, String> properties) {

Review Comment:
   just fyi, we discussed with @danielcweeks offline and decided to rename it to `create` to cause less confusion. I don't think the javadoc is required, since it's not "only" for dynamic loading. It's just rather a quicker way to init an instance than going through the Immutable builder



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093365977


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   we discussed this offline with @danielcweeks and decided to name the method `create`. 
   /cc @jackye1995 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#issuecomment-1310416636

   For testing we might actually need https://github.com/apache/iceberg/pull/4855


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030813431


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();

Review Comment:
   If the token is set, it should be used here to fetch the credential.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030817942


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();

Review Comment:
   This duplicates the handling below. Can you refactor to avoid the duplication? Something like this:
   
   ```java
     SignedComponent signed;
     if (cachedSignedComponent != null) {
       signed = cachedSignedComponent;
     } else {
       S3SignResponse s3SignResponse = httpClient().post(...)
     }
   
     SdkHttpFullRequest.Builder request ...
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030906363


##########
core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java:
##########
@@ -381,5 +394,42 @@ public Pair<Integer, TimeUnit> refresh(RESTClient client) {
 
       return null;
     }
+
+    @SuppressWarnings("FutureReturnValueIgnored")

Review Comment:
   What is the issue this is suppressing?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031540310


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {

Review Comment:
   agreed, I've removed usage of `URI` and changed it to a plain string



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031591808


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();
+    }
+
+    S3SignResponse s3SignResponse =
+        httpClient()
+            .post(
+                endpoint(),
+                remoteSigningRequest,
+                S3SignResponse.class,
+                () -> OAuth2Util.authHeaders(authSession().token()),
+                ErrorHandlers.defaultErrorHandler());
+
+    if (canBeCached(s3SignResponse)) {
+      SIGNED_COMPONENT_CACHE.put(
+          cacheKey,
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build());
+    }
+
+    // Update URI and append signed and unsigned headers
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(s3SignResponse.uri());
+    reconstructHeaders(s3SignResponse.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server

Review Comment:
   the cache-control header is only meant for the S3 Signer client, so we shouldn't propagate it further. See also my comment [here](https://github.com/apache/iceberg/pull/6169#discussion_r1031590899) about why the cache-control header is in the headers of the response



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1021760449


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -261,6 +267,8 @@ public class AwsProperties implements Serializable {
 
   public static final boolean S3_CHECKSUM_ENABLED_DEFAULT = false;
 
+  public static final String S3_SIGNER_CLASS = "s3.signer.class";

Review Comment:
   based on the convention we have been using, I think we prefer to call this config signer-impl?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1021818225


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);

Review Comment:
   I see, the cache control is determined using the header and thus determined by the service side, so it delegates the security posture to the service.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1021793958


##########
open-api/s3-signer-open-api.yaml:
##########
@@ -0,0 +1,273 @@
+#
+# 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.
+#
+
+---
+openapi: 3.0.3
+info:
+  title: Apache Iceberg S3 Signer API
+  license:
+    name: Apache 2.0
+    url: https://www.apache.org/licenses/LICENSE-2.0.html
+  version: 0.0.1
+  description:
+    Defines the specification for the S3 Signer API.
+servers:
+  - url: "{scheme}://{host}/{basePath}"
+    description: Server URL when the port can be inferred from the scheme
+    variables:
+      scheme:
+        description: The scheme of the URI, either http or https.
+        default: https
+      host:
+        description: The host address for the specified server
+        default: localhost
+      basePath:
+        description: Optional prefix to be prepended to all routes
+        default: ""
+  - url: "{scheme}://{host}:{port}/{basePath}"
+    description: Generic base server URL, with all parts configurable
+    variables:
+      scheme:
+        description: The scheme of the URI, either http or https.
+        default: https
+      host:
+        description: The host address for the specified server
+        default: localhost
+      port:
+        description: The port used when addressing the host
+        default: "443"
+      basePath:
+        description: Optional prefix to be appended to all routes
+        default: ""
+
+paths:
+
+  /v1/aws/s3/sign:
+
+    post:
+      tags:
+        - S3 Signer API
+      summary: Remotely signs S3 requests
+      operationId: signS3Request
+      requestBody:
+        description: The request containing the headers to be signed
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/S3SignRequest'
+        required: true
+      responses:
+        200:
+          $ref: '#/components/responses/S3SignResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  ##############################
+  # Application Schema Objects #
+  ##############################
+components:
+  schemas:
+
+    ErrorModel:

Review Comment:
   looks like this is exactly the same as https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L857-L881, is there any inheritance model for OpenAPI so we don't need to re-define the error model?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1029607140


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   This seems suspicious to me. We don't use a static `of(Map)` for any other dynamically loaded class.



##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")
+              .impl(s3SignerImpl, Map.class)
+              .buildChecked()
+              .invoke(null, allProperties);

Review Comment:
   Instead of passing `null`, this should use `buildStatic`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030807606


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")

Review Comment:
   Using more standard loading would avoid this problem so we don't need to suppress it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031538544


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -157,22 +154,22 @@ public void initialize(String name, Map<String, String> unresolved) {
     // build the final configuration and set up the catalog's auth
     Map<String, String> mergedProps = config.merge(props);
     Map<String, String> baseHeaders = configHeaders(mergedProps);
-    this.catalogAuth = new AuthSession(baseHeaders, null, null);
-    if (authResponse != null) {
-      this.catalogAuth = newSession(authResponse, startTimeMillis, catalogAuth);
-    } else if (initToken != null) {
-      this.catalogAuth = newSession(initToken, expiresInMs(mergedProps), catalogAuth);
-    }
-
+    this.client = clientBuilder.apply(mergedProps);
     this.sessions = newSessionCache(mergedProps);
     this.refreshAuthByDefault =
         PropertyUtil.propertyAsBoolean(
             mergedProps,
             CatalogProperties.AUTH_DEFAULT_REFRESH_ENABLED,
             CatalogProperties.AUTH_DEFAULT_REFRESH_ENABLED_DEFAULT);
-    this.client = clientBuilder.apply(mergedProps);
     this.paths = ResourcePaths.forCatalogProperties(mergedProps);
 
+    this.catalogAuth = new AuthSession(baseHeaders, null, null);
+    if (authResponse != null) {
+      this.catalogAuth = newSession(authResponse, startTimeMillis, catalogAuth);
+    } else if (initToken != null) {
+      this.catalogAuth = newSession(initToken, expiresInMs(mergedProps), catalogAuth);
+    }

Review Comment:
   this is mainly because `scheduleTokenRefresh` became `static` and moved to `AuthSession` and we're passing the `client` in both of those `newSession()` calls. However, with the old code the client` wasn't initialized yet. So this re-ordering just makes sure that the `client` is initialized before we schedule a token refresh



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#issuecomment-1310373071

   /cc @rdblue @danielcweeks @jackye1995 @amogh-jahagirdar


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1023039258


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);

Review Comment:
   yes we leave the decision to the server implementation of whether the signed headers can/should be cached or not



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#issuecomment-1315625210

   1 general question regarding this PR, @nastra @rdblue @danielcweeks this is a feature very specific to AWS S3. What is the general guideline in the community for adding this as a part of the OpenAPI spec? In my mind the spec should be something generic for Iceberg, and not related to any specific cloud provider. Although many cloud storage providers have built S3 compatible API, but mostly just do it to some extent to support basic operations like get and put object. I would imaging signing with sigv4 is still a very AWS-specific thing, but maybe I am wrong about that.
   
   Overall, have we thought about making this spec more generic, or is that not possible to achieve without being cloud provider specific?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030822090


##########
aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.net.URI;
+import java.time.Duration;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.utility.Base58;
+import software.amazon.awssdk.auth.credentials.AwsCredentials;
+
+public class MinioContainer extends GenericContainer<MinioContainer> {

Review Comment:
   How long do these tests take to complete?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030132891


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")
+              .impl(s3SignerImpl, Map.class)
+              .buildChecked()
+              .invoke(null, allProperties);

Review Comment:
   makes sense, updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1089652555


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+  private static final String SCOPE = "sign";
+  private static RESTClient httpClient;
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public String baseSignerUri() {
+    return properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  private RESTClient httpClient() {
+    if (null == httpClient) {
+      // TODO: should be closed
+      httpClient =
+          HTTPClient.builder()
+              .uri(baseSignerUri())
+              .withObjectMapper(S3ObjectMapper.mapper())
+              .build();
+    }
+
+    return httpClient;
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    if (null != token()) {
+      return AuthSession.fromAccessToken(
+          httpClient(),
+          TOKEN_REFRESH_EXECUTOR,
+          token(),
+          expiresAtMillis(properties()),
+          new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE));
+    }
+
+    if (credentialProvided()) {
+      AuthSession session = new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE);
+      long startTimeMillis = System.currentTimeMillis();
+      OAuthTokenResponse authResponse =
+          OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), SCOPE);
+      return AuthSession.fromTokenResponse(
+          httpClient(), TOKEN_REFRESH_EXECUTOR, authResponse, startTimeMillis, session);
+    }
+
+    return AuthSession.empty();
+  }
+
+  private boolean credentialProvided() {
+    return null != credential() && !credential().isEmpty();
+  }
+
+  private Long expiresAtMillis(Map<String, String> properties) {
+    if (properties.containsKey(OAuth2Properties.TOKEN_EXPIRES_IN_MS)) {
+      long expiresInMillis =
+          PropertyUtil.propertyAsLong(
+              properties,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+      return System.currentTimeMillis() + expiresInMillis;
+    } else {
+      return null;
+    }
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+    SignedComponent signedComponent;
+
+    if (null != cachedSignedComponent) {
+      signedComponent = cachedSignedComponent;
+    } else {
+      Map<String, String> responseHeaders = Maps.newHashMap();
+      Consumer<Map<String, String>> responseHeadersConsumer = responseHeaders::putAll;
+      S3SignResponse s3SignResponse =
+          httpClient()
+              .post(
+                  endpoint(),
+                  remoteSigningRequest,
+                  S3SignResponse.class,
+                  () -> authSession().headers(),
+                  ErrorHandlers.defaultErrorHandler(),
+                  responseHeadersConsumer);
+
+      signedComponent =
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build();
+
+      if (canBeCached(responseHeaders)) {
+        SIGNED_COMPONENT_CACHE.put(cacheKey, signedComponent);
+      }
+    }
+
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(signedComponent.signedURI());
+    reconstructHeaders(signedComponent.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(Map<String, String> responseHeaders) {
+    return responseHeaders.containsKey(CACHE_CONTROL)
+        && CACHE_CONTROL_PRIVATE.equals(responseHeaders.get(CACHE_CONTROL));
+  }
+
+  private void checkSignerParams(AwsS3V4SignerParams signerParams) {
+    if (signerParams.enablePayloadSigning()) {
+      throw new UnsupportedOperationException("Payload signing not supported");
+    }
+
+    if (signerParams.enableChunkedEncoding()) {
+      throw new UnsupportedOperationException("Chunked encoding not supported");
+    }
+  }
+
+  @Value.Immutable
+  interface Key {
+    String method();
+
+    String region();
+
+    String uri();
+
+    static Key from(S3SignRequest request) {
+      return ImmutableKey.builder()
+          .method(request.method())
+          .region(request.region())
+          .uri(request.uri().toString())
+          .build();
+    }
+  }
+
+  @Value.Immutable
+  interface SignedComponent {
+    Map<String, List<String>> headers();
+
+    URI signedURI();
+  }
+
+  public static S3V4RestSignerClient of(Map<String, String> properties) {

Review Comment:
   nit: might want to add a javadoc here describing this class is used for dynamic initialization



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1091008163


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   Actually, based on the docs, I think that switches the instantiation to a constructor method, so it might be possible to get closer to the default behavior if we can construct with parameters.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1091005884


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   Looks like we could use the style annotations to change it to something else ([they recommend](https://immutables.github.io/style.html#Apply_Style) `new`):
   
   ```java
   @Value.Style(builder = "new")
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks merged pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks merged PR #6169:
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1089273386


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");

Review Comment:
   Pre-signed urls is a very different protocol.  We probably don't want to support that until there's a specific need as it would require a number of changes to how requests are made.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093622510


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("create")
+              .impl(s3SignerImpl, Map.class)
+              .buildStaticChecked()
+              .invoke(allProperties);
+    } catch (NoSuchMethodException e) {

Review Comment:
   I think we can avoid this nesting by using this structure:
   
   ```
   try {
         signer =
             DynMethods.builder("create")
                 .impl(s3SignerImpl, Map.class)
                 .buildStaticChecked()
                 .invoke(allProperties);
       } catch (NoSuchMethodException e) {
     LOG.warn("Cannot find static method create(Map<String, String properties) for signer {}", s3SignerImpl);
   }
   
   try {
    signer = DynMethods.builder("create").impl(s3SignerImpl).buildChecked().invoke(null);
   } catch (NoSuchMethodException e) {
     LOG.warn("Cannot find instance method create(Map<String, String properties) for signer {}", s3SignerImpl);
   }
   
   try {
    signer = DynMethods.builder().impl(s3SignerImpl).buildChecked().invokeInstance();
   } catch (NoSuchMethodException e) {
     LOG.warn("Cannot find no-arg constructor for signer {}", s3SignerImpl);
   }
   
   if (signer == null) {
     throw ...
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra closed pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec
URL: https://github.com/apache/iceberg/pull/6169


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1021815626


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);

Review Comment:
   Would it be a security risk to reuse a signature? Also looks like only `signerParams.signingRegion().id()` in signing parameter is used for checking the cache hit, why are the other parameters in https://github.com/aws/aws-sdk-java-v2/blob/6c7d78429ba3eed8ceec0e34434843b1dfa030d7/core/auth/src/main/java/software/amazon/awssdk/auth/signer/internal/AbstractAws4Signer.java#L412 not relevant? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030810542


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {

Review Comment:
   Is the use of URI required? In general it is a best practice to avoid the use of URI because the API handles escaping in an unusual and confusing way.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031642099


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");

Review Comment:
   because we might need a separate REST endpoint to do pre-signing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031536783


##########
aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.net.URI;
+import java.time.Duration;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.HttpWaitStrategy;
+import org.testcontainers.utility.Base58;
+import software.amazon.awssdk.auth.credentials.AwsCredentials;
+
+public class MinioContainer extends GenericContainer<MinioContainer> {

Review Comment:
   all tests combined from `TestS3RestSigner` where `MinioContainer` is being used take ~15 seconds total on my local machine, so those are usually fairly quick



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031590899


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();
+    }
+
+    S3SignResponse s3SignResponse =
+        httpClient()
+            .post(
+                endpoint(),
+                remoteSigningRequest,
+                S3SignResponse.class,
+                () -> OAuth2Util.authHeaders(authSession().token()),
+                ErrorHandlers.defaultErrorHandler());
+
+    if (canBeCached(s3SignResponse)) {
+      SIGNED_COMPONENT_CACHE.put(
+          cacheKey,
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build());
+    }
+
+    // Update URI and append signed and unsigned headers
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(s3SignResponse.uri());
+    reconstructHeaders(s3SignResponse.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(S3SignResponse s3SignResponse) {
+    List<String> cacheControlHeader = s3SignResponse.headers().get(CACHE_CONTROL);

Review Comment:
   probably yes, but I don't think we currently have a way to propagate those response headers via our `HTTPClient`. That's why I had to include the cache-control header into the headers of the sign response (and that's also the reason why we have to remove that header before propagating it further up



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031595516


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");

Review Comment:
   is this something we'd want to do as part of this PR or a separate one?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1023008382


##########
open-api/s3-signer-open-api.yaml:
##########
@@ -0,0 +1,273 @@
+#
+# 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.
+#
+
+---
+openapi: 3.0.3
+info:
+  title: Apache Iceberg S3 Signer API
+  license:
+    name: Apache 2.0
+    url: https://www.apache.org/licenses/LICENSE-2.0.html
+  version: 0.0.1
+  description:
+    Defines the specification for the S3 Signer API.
+servers:
+  - url: "{scheme}://{host}/{basePath}"
+    description: Server URL when the port can be inferred from the scheme
+    variables:
+      scheme:
+        description: The scheme of the URI, either http or https.
+        default: https
+      host:
+        description: The host address for the specified server
+        default: localhost
+      basePath:
+        description: Optional prefix to be prepended to all routes
+        default: ""
+  - url: "{scheme}://{host}:{port}/{basePath}"
+    description: Generic base server URL, with all parts configurable
+    variables:
+      scheme:
+        description: The scheme of the URI, either http or https.
+        default: https
+      host:
+        description: The host address for the specified server
+        default: localhost
+      port:
+        description: The port used when addressing the host
+        default: "443"
+      basePath:
+        description: Optional prefix to be appended to all routes
+        default: ""
+
+paths:
+
+  /v1/aws/s3/sign:
+
+    post:
+      tags:
+        - S3 Signer API
+      summary: Remotely signs S3 requests
+      operationId: signS3Request
+      requestBody:
+        description: The request containing the headers to be signed
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/S3SignRequest'
+        required: true
+      responses:
+        200:
+          $ref: '#/components/responses/S3SignResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  ##############################
+  # Application Schema Objects #
+  ##############################
+components:
+  schemas:
+
+    ErrorModel:

Review Comment:
   good point, turns out we can actually just refer to elements in the other file. I've updated it so that we don't have to repeat anything and also added a CI action that validates this newly added OpenAPI spec



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1031555625


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable

Review Comment:
   It gives us a much cleaner API, free validation checks and also allows expressing lazily loaded things in an easier fashion rather than having to do such things manually.
   
   From the API definion it's immediately obvious that the only required attribute to be set is `Map<String, String> properties()`. The attribute annotated with `@Value.Default` can be optionally set but has a default value if not.
   
   Everything annotated with `@Value.Lazy` are attributes that **cannot** be set. Those attributes are invoked lazily and computed only once in a thread-safe manner.
   
   In the method annotated with `@Value.Check` we perform additional validation, meaning that we can only construct a valid instance of this class once all validations (including the ones in this method pass).
   
   That being said, doing all of those things conventionally would just mean more code to write/maintain.
   I would be really in favor of keeping this an `@Value.Immutable` class, since it makes many things easier. 
   The only downside is that it doesn't fit with our conventional dynamic loading, because it doesn't provide a no-arg constructor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030818746


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())
+        .withObjectMapper(S3ObjectMapper.mapper())
+        .build();
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    AuthSession session = AuthSession.empty();
+    OAuthTokenResponse authResponse = null;
+    if (null != credential() && !credential().isEmpty()) {
+      authResponse = OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), "sign");
+    }
+
+    if (null != authResponse) {
+      session = newSession(authResponse, session);
+    } else if (null != token()) {
+      session = newSession(token(), session);
+    }
+
+    return session;
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  private Long tokenExpiresInMs() {
+    return PropertyUtil.propertyAsLong(
+        properties(),
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+        OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+
+    if (cachedSignedComponent != null) {
+      // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+      // so we need to clear the current path from the request
+      SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+      mutableRequest.encodedPath("");
+      mutableRequest.uri(cachedSignedComponent.signedURI());
+      reconstructHeaders(cachedSignedComponent.headers(), mutableRequest);
+
+      return mutableRequest.build();
+    }
+
+    S3SignResponse s3SignResponse =
+        httpClient()
+            .post(
+                endpoint(),
+                remoteSigningRequest,
+                S3SignResponse.class,
+                () -> OAuth2Util.authHeaders(authSession().token()),
+                ErrorHandlers.defaultErrorHandler());
+
+    if (canBeCached(s3SignResponse)) {
+      SIGNED_COMPONENT_CACHE.put(
+          cacheKey,
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build());
+    }
+
+    // Update URI and append signed and unsigned headers
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(s3SignResponse.uri());
+    reconstructHeaders(s3SignResponse.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server

Review Comment:
   Why does this override the cache control from the service?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030814197


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable

Review Comment:
   What is the value of using `Immutable` for this class?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030907128


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -157,22 +154,22 @@ public void initialize(String name, Map<String, String> unresolved) {
     // build the final configuration and set up the catalog's auth
     Map<String, String> mergedProps = config.merge(props);
     Map<String, String> baseHeaders = configHeaders(mergedProps);
-    this.catalogAuth = new AuthSession(baseHeaders, null, null);
-    if (authResponse != null) {
-      this.catalogAuth = newSession(authResponse, startTimeMillis, catalogAuth);
-    } else if (initToken != null) {
-      this.catalogAuth = newSession(initToken, expiresInMs(mergedProps), catalogAuth);
-    }
-
+    this.client = clientBuilder.apply(mergedProps);
     this.sessions = newSessionCache(mergedProps);
     this.refreshAuthByDefault =
         PropertyUtil.propertyAsBoolean(
             mergedProps,
             CatalogProperties.AUTH_DEFAULT_REFRESH_ENABLED,
             CatalogProperties.AUTH_DEFAULT_REFRESH_ENABLED_DEFAULT);
-    this.client = clientBuilder.apply(mergedProps);
     this.paths = ResourcePaths.forCatalogProperties(mergedProps);
 
+    this.catalogAuth = new AuthSession(baseHeaders, null, null);
+    if (authResponse != null) {
+      this.catalogAuth = newSession(authResponse, startTimeMillis, catalogAuth);
+    } else if (initToken != null) {
+      this.catalogAuth = newSession(initToken, expiresInMs(mergedProps), catalogAuth);
+    }

Review Comment:
   Why did these sections move around?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030141917


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   This is the method from the `S3V4RestSignerClient`:
   ```
   public static S3V4RestSignerClient of(Map<String, String> properties) {
       return ImmutableS3V4RestSignerClient.builder().properties(properties).build();
     }
   ```
   We could also rename this if required



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1030812842


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,328 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterAccess(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public URI baseSignerUri() {
+    return URI.create(
+        properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI)));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  @Value.Lazy
+  RESTClient httpClient() {
+    // TODO: should be closed
+    return HTTPClient.builder()
+        .uri(baseSignerUri().toString())

Review Comment:
   This should not create a URI only to call `toString`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1089652523


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1119,6 +1139,54 @@ public <T extends S3ClientBuilder> void applyS3ServiceConfigurations(T builder)
                 .build());
   }
 
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyS3SignerConfiguration(T builder) {
+    if (null != s3SignerImpl) {
+      builder.overrideConfiguration(
+          c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, loadS3SignerDynamically()));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:NestedTryDepth")
+  private Signer loadS3SignerDynamically() {
+    // load the signer implementation dynamically
+    Object signer;
+    try {
+      signer =
+          DynMethods.builder("of")

Review Comment:
   I think I get why we want to break the pattern of dynamic initialization because of using Immutable, I will leave that part to Ryan to make a call to use that for the signer class or not. But can we at least use a method name that is more intuitive to readers of this code, such as `initialize`, instead of `of`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093371937


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();

Review Comment:
   I think for now we'd probably go as is and follow up and make it configurable if we see that it helps with certain use cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#issuecomment-1416260930

   Thanks @nastra !!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6169: AWS,Core: Add S3 REST Signer client + REST Spec

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6169:
URL: https://github.com/apache/iceberg/pull/6169#discussion_r1093369444


##########
aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.signer;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession;
+import org.apache.iceberg.rest.responses.OAuthTokenResponse;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.immutables.value.Value;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.internal.AbstractAws4Signer;
+import software.amazon.awssdk.auth.signer.internal.Aws4SignerRequestParams;
+import software.amazon.awssdk.auth.signer.params.Aws4PresignerParams;
+import software.amazon.awssdk.auth.signer.params.AwsS3V4SignerParams;
+import software.amazon.awssdk.core.checksums.SdkChecksum;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+
+@Value.Immutable
+public abstract class S3V4RestSignerClient
+    extends AbstractAws4Signer<AwsS3V4SignerParams, Aws4PresignerParams> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(S3V4RestSignerClient.class);
+  public static final String S3_SIGNER_URI = "s3.signer.uri";
+  public static final String S3_SIGNER_ENDPOINT = "s3.signer.endpoint";
+  static final String S3_SIGNER_DEFAULT_ENDPOINT = "v1/aws/s3/sign";
+  static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  static final String CACHE_CONTROL = "Cache-Control";
+  static final String CACHE_CONTROL_PRIVATE = "private";
+  static final String CACHE_CONTROL_NO_CACHE = "no-cache";
+
+  private static final Cache<Key, SignedComponent> SIGNED_COMPONENT_CACHE =
+      Caffeine.newBuilder().expireAfterWrite(30, TimeUnit.SECONDS).maximumSize(100).build();
+
+  private static final ScheduledExecutorService TOKEN_REFRESH_EXECUTOR =
+      ThreadPools.newScheduledPool("s3-signer-token-refresh", 1);
+  private static final String SCOPE = "sign";
+  private static RESTClient httpClient;
+
+  public abstract Map<String, String> properties();
+
+  @Value.Default
+  public Supplier<Map<String, String>> requestPropertiesSupplier() {
+    return Collections::emptyMap;
+  }
+
+  @Value.Lazy
+  public String baseSignerUri() {
+    return properties().getOrDefault(S3_SIGNER_URI, properties().get(CatalogProperties.URI));
+  }
+
+  @Value.Lazy
+  public String endpoint() {
+    return properties().getOrDefault(S3_SIGNER_ENDPOINT, S3_SIGNER_DEFAULT_ENDPOINT);
+  }
+
+  /** A credential to exchange for a token in the OAuth2 client credentials flow. */
+  @Nullable
+  @Value.Lazy
+  public String credential() {
+    return properties().get(OAuth2Properties.CREDENTIAL);
+  }
+
+  /** A Bearer token which will be used for interaction with the server. */
+  @Nullable
+  @Value.Lazy
+  public String token() {
+    return properties().get(OAuth2Properties.TOKEN);
+  }
+
+  private RESTClient httpClient() {
+    if (null == httpClient) {
+      // TODO: should be closed
+      httpClient =
+          HTTPClient.builder()
+              .uri(baseSignerUri())
+              .withObjectMapper(S3ObjectMapper.mapper())
+              .build();
+    }
+
+    return httpClient;
+  }
+
+  @Value.Lazy
+  AuthSession authSession() {
+    if (null != token()) {
+      return AuthSession.fromAccessToken(
+          httpClient(),
+          TOKEN_REFRESH_EXECUTOR,
+          token(),
+          expiresAtMillis(properties()),
+          new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE));
+    }
+
+    if (credentialProvided()) {
+      AuthSession session = new AuthSession(ImmutableMap.of(), token(), null, credential(), SCOPE);
+      long startTimeMillis = System.currentTimeMillis();
+      OAuthTokenResponse authResponse =
+          OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), SCOPE);
+      return AuthSession.fromTokenResponse(
+          httpClient(), TOKEN_REFRESH_EXECUTOR, authResponse, startTimeMillis, session);
+    }
+
+    return AuthSession.empty();
+  }
+
+  private boolean credentialProvided() {
+    return null != credential() && !credential().isEmpty();
+  }
+
+  private Long expiresAtMillis(Map<String, String> properties) {
+    if (properties.containsKey(OAuth2Properties.TOKEN_EXPIRES_IN_MS)) {
+      long expiresInMillis =
+          PropertyUtil.propertyAsLong(
+              properties,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS,
+              OAuth2Properties.TOKEN_EXPIRES_IN_MS_DEFAULT);
+      return System.currentTimeMillis() + expiresInMillis;
+    } else {
+      return null;
+    }
+  }
+
+  @Value.Check
+  protected void check() {
+    Preconditions.checkArgument(
+        properties().containsKey(S3_SIGNER_URI) || properties().containsKey(CatalogProperties.URI),
+        "S3 signer service URI is required");
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected void processRequestPayload(
+      SdkHttpFullRequest.Builder mutableRequest,
+      byte[] signature,
+      byte[] signingKey,
+      Aws4SignerRequestParams signerRequestParams,
+      AwsS3V4SignerParams signerParams,
+      SdkChecksum sdkChecksum) {
+    checkSignerParams(signerParams);
+  }
+
+  @Override
+  protected String calculateContentHashPresign(
+      SdkHttpFullRequest.Builder mutableRequest, Aws4PresignerParams signerParams) {
+    return UNSIGNED_PAYLOAD;
+  }
+
+  @Override
+  public SdkHttpFullRequest presign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    throw new UnsupportedOperationException("Pre-signing not allowed.");
+  }
+
+  @Override
+  public SdkHttpFullRequest sign(
+      SdkHttpFullRequest request, ExecutionAttributes executionAttributes) {
+    AwsS3V4SignerParams signerParams =
+        extractSignerParams(AwsS3V4SignerParams.builder(), executionAttributes).build();
+
+    S3SignRequest remoteSigningRequest =
+        ImmutableS3SignRequest.builder()
+            .method(request.method().name())
+            .region(signerParams.signingRegion().id())
+            .uri(request.getUri())
+            .headers(request.headers())
+            .properties(requestPropertiesSupplier().get())
+            .build();
+
+    Key cacheKey = Key.from(remoteSigningRequest);
+    SignedComponent cachedSignedComponent = SIGNED_COMPONENT_CACHE.getIfPresent(cacheKey);
+    SignedComponent signedComponent;
+
+    if (null != cachedSignedComponent) {
+      signedComponent = cachedSignedComponent;
+    } else {
+      Map<String, String> responseHeaders = Maps.newHashMap();
+      Consumer<Map<String, String>> responseHeadersConsumer = responseHeaders::putAll;
+      S3SignResponse s3SignResponse =
+          httpClient()
+              .post(
+                  endpoint(),
+                  remoteSigningRequest,
+                  S3SignResponse.class,
+                  () -> authSession().headers(),
+                  ErrorHandlers.defaultErrorHandler(),
+                  responseHeadersConsumer);
+
+      signedComponent =
+          ImmutableSignedComponent.builder()
+              .headers(s3SignResponse.headers())
+              .signedURI(s3SignResponse.uri())
+              .build();
+
+      if (canBeCached(responseHeaders)) {
+        SIGNED_COMPONENT_CACHE.put(cacheKey, signedComponent);
+      }
+    }
+
+    // The SdkHttpFullRequest Builder appends the raw path from the input URI in .uri(),
+    // so we need to clear the current path from the request
+    SdkHttpFullRequest.Builder mutableRequest = request.toBuilder();
+    mutableRequest.encodedPath("");
+    mutableRequest.uri(signedComponent.signedURI());
+    reconstructHeaders(signedComponent.headers(), mutableRequest);
+
+    return mutableRequest.build();
+  }
+
+  private void reconstructHeaders(
+      Map<String, List<String>> signedAndUnsignedHeaders,
+      SdkHttpFullRequest.Builder mutableRequest) {
+    Map<String, List<String>> headers = Maps.newHashMap(signedAndUnsignedHeaders);
+    // we need to remove the Cache-Control header that is being sent by the server
+    headers.remove(CACHE_CONTROL);
+
+    // we need to overwrite whatever headers the server signed/unsigned with the ones from the
+    // original request and then put all headers back to the request
+    headers.putAll(mutableRequest.headers());
+    headers.forEach(mutableRequest::putHeader);
+  }
+
+  private boolean canBeCached(Map<String, String> responseHeaders) {
+    return responseHeaders.containsKey(CACHE_CONTROL)
+        && CACHE_CONTROL_PRIVATE.equals(responseHeaders.get(CACHE_CONTROL));

Review Comment:
   good point, thanks. I've adjusted that



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org