[ 
https://issues.apache.org/jira/browse/HADOOP-18695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17716057#comment-17716057
 ] 

ASF GitHub Bot commented on HADOOP-18695:
-----------------------------------------

mukund-thakur commented on code in PR #5548:
URL: https://github.com/apache/hadoop/pull/5548#discussion_r1175910129


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java:
##########
@@ -142,4 +144,20 @@ public static <T extends HandlerContextAware> void 
attachSpanToRequest(
     request.addHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT, span);
   }
 
+  /**
+   * Translate an audit exception.
+   * @param path path of operation.
+   * @param exception exception
+   * @return the IOE to raise.
+   */
+  public static IOException translateAuditException(String path,
+      AuditFailureException exception) {
+    if (exception instanceof AuditOperationRejectedException) {
+      // special handling of this subclass
+      return new UnsupportedRequestException(path,
+          exception.getMessage(), exception);
+    }
+    return (AccessDeniedException)new AccessDeniedException(path, null,

Review Comment:
   Why AccessDenied?



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java:
##########
@@ -75,6 +76,7 @@ public S3AMultipartUploaderStatisticsImpl(
             MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(),
             MULTIPART_UPLOAD_COMPLETED.getSymbol(),
             MULTIPART_UPLOAD_STARTED.getSymbol())
+        .withDurationTracking(OBJECT_PUT_REQUESTS.getSymbol())

Review Comment:
   Actual put request will be 0 for multipart right?



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java:
##########
@@ -257,11 +309,28 @@ public void test_010_CreateHugeFile() throws IOException {
     verifyStatisticGaugeValue(iostats, putRequestsActive.getSymbol(), 0);
     verifyStatisticGaugeValue(iostats,
         STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0);
+
     progress.verifyNoFailures(
         "Put file " + fileToCreate + " of size " + filesize);
-    if (streamStatistics != null) {
-      assertEquals("actively allocated blocks in " + streamStatistics,
-          0, streamStatistics.getBlocksActivelyAllocated());
+    assertEquals("actively allocated blocks in " + streamStatistics,
+        0, streamStatistics.getBlocksActivelyAllocated());
+
+    if (!expectImmediateFileVisibility()) {

Review Comment:
   Just an idea. Why don't we put this whole block of code as part of 
ITestS3AHugeMagicCommits? 



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesNoMultipart.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.s3a.api.UnsupportedRequestException;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_UPLOADS_ENABLED;
+import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT;
+import static 
org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Use a single PUT for the whole upload/rename/delete workflow; include 
verification
+ * that the transfer manager will fail fast unless the multipart threshold is 
huge.
+ */
+public class ITestS3AHugeFilesNoMultipart extends AbstractSTestS3AHugeFiles {
+
+  /**
+   * Size to ensure MPUs don't happen in transfer manager.
+   */
+  public static final String S_1T = "1T";
+
+  public static final String SINGLE_PUT_REQUEST_TIMEOUT = "1h";
+
+  /**
+   * Always use disk storage.
+   * @return disk block store always.
+   */
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_DISK;
+  }
+
+  @Override
+  protected boolean expectMultipartUpload() {
+    return false;
+  }
+
+  /**
+   * Create a configuration without multipart upload,
+   * and a long request timeout to allow for a very slow
+   * PUT in close.
+   * @return the configuration to create the test FS with.
+   */
+  @Override
+  protected Configuration createScaleConfiguration() {
+    Configuration conf = super.createScaleConfiguration();
+    removeBaseAndBucketOverrides(conf,
+        IO_CHUNK_BUFFER_SIZE,
+        MIN_MULTIPART_THRESHOLD,
+        MULTIPART_UPLOADS_ENABLED,
+        MULTIPART_SIZE,
+        REQUEST_TIMEOUT);
+    conf.setInt(IO_CHUNK_BUFFER_SIZE, 655360);
+    conf.set(MIN_MULTIPART_THRESHOLD, S_1T);
+    conf.set(MULTIPART_SIZE, S_1T);
+    conf.setBoolean(MULTIPART_UPLOADS_ENABLED, false);
+    conf.set(REQUEST_TIMEOUT, SINGLE_PUT_REQUEST_TIMEOUT);
+    return conf;
+  }
+
+  /**
+   * After the file is created, attempt a rename with an FS
+   * instance with a small multipart threshold;
+   * this MUST be rejected.
+   */
+  @Override
+  public void test_030_postCreationAssertions() throws Throwable {

Review Comment:
   Should we add a test for successful rename as well?



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java:
##########
@@ -82,7 +82,7 @@ public static IdKey createPartUpload(S3AFileSystem fs, String 
key, int len,
       String uploadId = writeHelper.initiateMultiPartUpload(key, 
PutObjectOptions.keepingDirs());
       UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId,
           partNo, len, in, null, 0L);
-      PartETag partEtag = writeHelper.uploadPart(req).getPartETag();
+      PartETag partEtag = writeHelper.uploadPart(req, 
fs.getDurationTrackerFactory()).getPartETag();

Review Comment:
   why not just pass null here like we are passing in production code and let 
the wrapper check for null?
   my comment won't be valid if we are setting some custom 
DurationTrackerFactory for test fs.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java:
##########
@@ -257,11 +309,28 @@ public void test_010_CreateHugeFile() throws IOException {
     verifyStatisticGaugeValue(iostats, putRequestsActive.getSymbol(), 0);
     verifyStatisticGaugeValue(iostats,
         STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0);
+
     progress.verifyNoFailures(
         "Put file " + fileToCreate + " of size " + filesize);
-    if (streamStatistics != null) {
-      assertEquals("actively allocated blocks in " + streamStatistics,
-          0, streamStatistics.getBlocksActivelyAllocated());
+    assertEquals("actively allocated blocks in " + streamStatistics,
+        0, streamStatistics.getBlocksActivelyAllocated());
+
+    if (!expectImmediateFileVisibility()) {

Review Comment:
   Once the file has been created and all the basic validations done..add for 
extra validations depending on the subclassed tests.





> S3A: reject multipart copy requests when disabled
> -------------------------------------------------
>
>                 Key: HADOOP-18695
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18695
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs/s3
>    Affects Versions: 3.4.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Minor
>              Labels: pull-request-available
>
> follow-on to HADOOP-18637 and support for huge file uploads with stores which 
> don't support MPU.
> * prevent use of API against any s3 store when disabled, using logging 
> auditor to reject it
> * tests to verify rename of huge files still works (by setting large part 
> size)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to