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

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

steveloughran commented on code in PR #5481:
URL: https://github.com/apache/hadoop/pull/5481#discussion_r1159979341


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:
##########
@@ -767,6 +781,18 @@ public RequestFactoryBuilder withRequestPreparer(
       this.requestPreparer = value;
       return this;
     }
+
+    /**
+     * Multipart enabled

Review Comment:
   always add a trailing "." on javadocs. it'll save review iterations



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUploadSinglePut.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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 java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBytes;
+import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
+
+/**
+ * Test a file upload using a single PUT operation. Multipart uploads will
+ * be disabled in the test.
+ */
+public class ITestS3AHugeFileUploadSinglePut extends S3AScaleTestBase{
+  final private Logger LOG = LoggerFactory.getLogger(
+      ITestS3AHugeFileUploadSinglePut.class.getName());
+
+  private long fileSize = Integer.MAX_VALUE * 2L;
+  @Override
+  protected Configuration createScaleConfiguration() {
+    Configuration configuration = super.createScaleConfiguration();
+    configuration.setBoolean(Constants.MULTIPART_UPLOADS_ENABLED, false);
+    configuration.setLong(MULTIPART_SIZE, 53687091200L);

Review Comment:
   is this some special value? if so: make a constant, explain what it is.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java:
##########
@@ -173,7 +174,11 @@ private void createFactoryObjects(RequestFactory factory) {
     a(factory.newListObjectsV1Request(path, "/", 1));
     a(factory.newListNextBatchOfObjectsRequest(new ObjectListing()));
     a(factory.newListObjectsV2Request(path, "/", 1));
-    a(factory.newMultipartUploadRequest(path, null));
+    try {

Review Comment:
   just change have the method throw IOE and cut the try/catch



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java:
##########
@@ -99,6 +99,11 @@ public interface StreamCapabilities {
    */
   String IOSTATISTICS_CONTEXT = "fs.capability.iocontext.supported";
 
+  /**

Review Comment:
   as this is s3a only, put it in org.apache.hadoop.fs.s3a.Constants with the 
MULTIPART_UPLOADS_ENABLED definition and using the same prefix 
`fs.s3a.capability.` as `STORE_CAPABILITY_DIRECTORY_MARKER*` probes



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:
##########
@@ -460,7 +466,10 @@ public AbortMultipartUploadRequest 
newAbortMultipartUploadRequest(
   @Override
   public InitiateMultipartUploadRequest newMultipartUploadRequest(
       final String destKey,
-      @Nullable final PutObjectOptions options) {
+      @Nullable final PutObjectOptions options) throws IOException {
+    if(!isMultipartEnabled){

Review Comment:
   nit: spacing



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocolFailure.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.commit.magic;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.commit.CommitConstants;
+import org.apache.hadoop.fs.s3a.commit.PathCommitException;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_UPLOADS_ENABLED;
+import static 
org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_NAME;
+import static 
org.apache.hadoop.fs.s3a.commit.CommitConstants.S3A_COMMITTER_FACTORY_KEY;
+
+public class ITestMagicCommitProtocolFailure extends AbstractS3ATestBase {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(MULTIPART_UPLOADS_ENABLED, false);
+    conf.set(S3A_COMMITTER_FACTORY_KEY, CommitConstants.S3A_COMMITTER_FACTORY);
+    conf.set(FS_S3A_COMMITTER_NAME, CommitConstants.COMMITTER_NAME_MAGIC);
+    return conf;
+  }
+
+  @Test
+  public void testCreateCommitter() {
+    TaskAttemptContext tContext = new 
TaskAttemptContextImpl(getConfiguration(),
+        new TaskAttemptID());
+    Path commitPath = getFileSystem().makeQualified(
+        new Path(getContract().getTestPath(), "/testpath"));
+    LOG.debug("{}", commitPath);
+    assertThrows(PathCommitException.class,

Review Comment:
   prefer LambdaTestUtils.intercept. why so?
   * the move to jupiter API is a PITA for backporting etc
   * intercept() will include the toString() value of whatever was returned 
(here: the committer) in the exception raised. hence: automatic diagnostics
   





> S3A to support upload of files greater than 2 GB using DiskBlocks
> -----------------------------------------------------------------
>
>                 Key: HADOOP-18637
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18637
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs/s3
>            Reporter: Harshit Gupta
>            Assignee: Harshit Gupta
>            Priority: Major
>              Labels: pull-request-available
>
> Use S3A Diskblocks to support the upload of files greater than 2 GB using 
> DiskBlocks. Currently, the max upload size of a single block is ~2GB. 
> cc: [~mthakur] [[email protected]] [~mehakmeet] 



--
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