You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "danielcweeks (via GitHub)" <gi...@apache.org> on 2023/01/27 18:13:30 UTC

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

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