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-unsubscr...@iceberg.apache.org

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


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

Reply via email to