Izeren commented on code in PR #27187:
URL: https://github.com/apache/flink/pull/27187#discussion_r2783185972


##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3AccessHelper.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.flink.fs.s3native.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3native.S3EncryptionConfig;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.core.sync.RequestBody;
+import software.amazon.awssdk.core.sync.ResponseTransformer;
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
+import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
+import 
software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
+import software.amazon.awssdk.services.s3.model.CompletedMultipartUpload;
+import software.amazon.awssdk.services.s3.model.CompletedPart;
+import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
+import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse;
+import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
+import software.amazon.awssdk.services.s3.model.GetObjectRequest;
+import software.amazon.awssdk.services.s3.model.GetObjectResponse;
+import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
+import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
+import software.amazon.awssdk.services.s3.model.NoSuchUploadException;
+import software.amazon.awssdk.services.s3.model.PutObjectRequest;
+import software.amazon.awssdk.services.s3.model.PutObjectResponse;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.UploadPartRequest;
+import software.amazon.awssdk.services.s3.model.UploadPartResponse;
+import software.amazon.awssdk.transfer.s3.S3TransferManager;
+import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
+import software.amazon.awssdk.transfer.s3.model.FileUpload;
+import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class for S3 operations including multipart uploads, object storage, 
and retrieval.
+ *
+ * <p><b>Retry Handling:</b> This class relies on the AWS SDK's built-in retry 
mechanism which is
+ * configured in {@link org.apache.flink.fs.s3native.S3ClientProvider}. The 
SDK automatically
+ * handles retries for transient errors including:
+ *
+ * <ul>
+ *   <li>5xx server errors (including S3 throttling which manifests as 503 
Service Unavailable)
+ *   <li>Network timeouts and connection errors
+ *   <li>Request timeouts
+ * </ul>
+ *
+ * <p>Terminal (non-retriable) errors include:
+ *
+ * <ul>
+ *   <li>4xx client errors (400 Bad Request, 403 Forbidden, 404 Not Found)
+ *   <li>Authentication/authorization failures
+ *   <li>Invalid bucket or key names
+ * </ul>
+ *
+ * <p><b>Encryption Support:</b> Currently supports SSE-S3 and SSE-KMS 
encryption modes. The
+ * encryption logic is applied through the {@link S3EncryptionConfig} class. 
Future enhancements may
+ * include:
+ *
+ * <ul>
+ *   <li>SSE-C (customer-provided keys) via a KeyProvider interface
+ *   <li>Client-side encryption via an EncryptionHandler interface
+ *   <li>Encryption context for SSE-KMS (see HADOOP-19197)
+ * </ul>
+ *
+ * <p><b>S3 URI Handling:</b> The {@link #extractKey(Path)} and {@link 
#extractBucketName(Path)}
+ * methods expect URIs in the standard {@code s3://bucket/key} format. Other 
formats like path-style
+ * ({@code https://s3.amazonaws.com/bucket/key}) or virtual-hosted-style 
({@code
+ * https://bucket.s3.amazonaws.com/key}) are not currently supported.
+ */
+@Internal
+public class NativeS3AccessHelper {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(NativeS3AccessHelper.class);
+
+    private final S3Client s3Client;
+    private final S3AsyncClient s3AsyncClient;
+    private final S3TransferManager transferManager;
+    private final String bucketName;
+    private final boolean useAsyncOperations;
+    private final S3EncryptionConfig encryptionConfig;
+
+    public NativeS3AccessHelper(S3Client s3Client, String bucketName) {
+        this(s3Client, null, null, bucketName, false, null);
+    }
+
+    public NativeS3AccessHelper(
+            S3Client s3Client,
+            S3AsyncClient s3AsyncClient,
+            S3TransferManager transferManager,
+            String bucketName,
+            boolean useAsyncOperations) {
+        this(s3Client, s3AsyncClient, transferManager, bucketName, 
useAsyncOperations, null);
+    }
+
+    public NativeS3AccessHelper(
+            S3Client s3Client,
+            S3AsyncClient s3AsyncClient,
+            S3TransferManager transferManager,
+            String bucketName,
+            boolean useAsyncOperations,
+            S3EncryptionConfig encryptionConfig) {
+        this.s3Client = s3Client;
+        this.s3AsyncClient = s3AsyncClient;
+        this.transferManager = transferManager;
+        this.bucketName = bucketName;
+        this.useAsyncOperations = useAsyncOperations && transferManager != 
null;
+        this.encryptionConfig =
+                encryptionConfig != null ? encryptionConfig : 
S3EncryptionConfig.none();
+    }
+
+    public String startMultiPartUpload(String key) throws IOException {
+        try {
+            CreateMultipartUploadRequest.Builder requestBuilder =
+                    
CreateMultipartUploadRequest.builder().bucket(bucketName).key(key);
+            applyEncryption(requestBuilder);
+
+            CreateMultipartUploadResponse response =
+                    s3Client.createMultipartUpload(requestBuilder.build());
+            return response.uploadId();
+        } catch (S3Exception e) {
+            throw new IOException("Failed to start multipart upload for key: " 
+ key, e);
+        }
+    }
+
+    private void applyEncryption(CreateMultipartUploadRequest.Builder 
requestBuilder) {
+        if (!encryptionConfig.isEnabled()) {
+            return;
+        }
+        
requestBuilder.serverSideEncryption(encryptionConfig.getServerSideEncryption());
+        if (encryptionConfig.getEncryptionType() == 
S3EncryptionConfig.EncryptionType.SSE_KMS) {
+            if (encryptionConfig.getKmsKeyId() != null) {
+                requestBuilder.ssekmsKeyId(encryptionConfig.getKmsKeyId());
+            }
+            if (encryptionConfig.hasEncryptionContext()) {
+                requestBuilder.ssekmsEncryptionContext(
+                        
serializeEncryptionContext(encryptionConfig.getEncryptionContext()));
+            }
+        }
+    }
+
+    private void applyEncryption(PutObjectRequest.Builder requestBuilder) {
+        if (!encryptionConfig.isEnabled()) {
+            return;
+        }
+        
requestBuilder.serverSideEncryption(encryptionConfig.getServerSideEncryption());
+        if (encryptionConfig.getEncryptionType() == 
S3EncryptionConfig.EncryptionType.SSE_KMS) {
+            if (encryptionConfig.getKmsKeyId() != null) {
+                requestBuilder.ssekmsKeyId(encryptionConfig.getKmsKeyId());
+            }
+            if (encryptionConfig.hasEncryptionContext()) {
+                requestBuilder.ssekmsEncryptionContext(
+                        
serializeEncryptionContext(encryptionConfig.getEncryptionContext()));
+            }
+        }
+    }
+
+    /**
+     * Serializes the encryption context map to a Base64-encoded JSON string 
as required by S3 API.
+     */
+    private String serializeEncryptionContext(java.util.Map<String, String> 
context) {
+        StringBuilder json = new StringBuilder("{");
+        boolean first = true;
+        for (java.util.Map.Entry<String, String> entry : context.entrySet()) {
+            if (!first) {
+                json.append(",");
+            }
+            json.append("\"")
+                    .append(escapeJson(entry.getKey()))
+                    .append("\":\"")
+                    .append(escapeJson(entry.getValue()))
+                    .append("\"");
+            first = false;
+        }
+        json.append("}");
+        return 
java.util.Base64.getEncoder().encodeToString(json.toString().getBytes());
+    }
+
+    private String escapeJson(String value) {
+        return value.replace("\\", "\\\\").replace("\"", "\\\"");
+    }
+
+    public UploadPartResult uploadPart(
+            String key, String uploadId, int partNumber, File inputFile, long 
length)
+            throws IOException {
+        try {
+            UploadPartRequest request =
+                    UploadPartRequest.builder()
+                            .bucket(bucketName)
+                            .key(key)
+                            .uploadId(uploadId)
+                            .partNumber(partNumber)
+                            .build();
+
+            UploadPartResponse response =
+                    s3Client.uploadPart(request, 
RequestBody.fromFile(inputFile));
+
+            return new UploadPartResult(partNumber, response.eTag());
+        } catch (S3Exception e) {
+            throw new IOException(
+                    String.format(
+                            "Failed to upload part %d for key: %s, uploadId: 
%s",
+                            partNumber, key, uploadId),
+                    e);
+        }
+    }
+
+    public PutObjectResult putObject(String key, File inputFile) throws 
IOException {
+        if (useAsyncOperations && transferManager != null) {
+            return putObjectAsync(key, inputFile);
+        }
+
+        try {
+            PutObjectRequest.Builder requestBuilder =
+                    PutObjectRequest.builder().bucket(bucketName).key(key);
+            applyEncryption(requestBuilder);
+
+            PutObjectResponse response =
+                    s3Client.putObject(requestBuilder.build(), 
RequestBody.fromFile(inputFile));
+            return new PutObjectResult(response.eTag());
+        } catch (S3Exception e) {
+            throw new IOException("Failed to put object for key: " + key, e);
+        }
+    }
+
+    /**
+     * Uploads an object asynchronously using the S3TransferManager.
+     *
+     * <p>Note: This method uses async internally for better throughput 
(leveraging the transfer

Review Comment:
   This is an implementation detail, it shouldn't be a part of the method name. 
`putObjectAsync` implies that method is non blocking and caller would have to 
call `.join` on `result` which is not the case. Could we change the name of the 
method to avoid confusion?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to