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

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

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


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -558,19 +565,21 @@ public String toString() {
   }
 
   /**
-   * Upload the current block as a single PUT request; if the buffer
-   * is empty a 0-byte PUT will be invoked, as it is needed to create an
-   * entry at the far end.
-   * @throws IOException any problem.
-   * @return number of bytes uploaded. If thread was interrupted while
-   * waiting for upload to complete, returns zero with interrupted flag set
-   * on this thread.
+   * Upload the current block as a single PUT request; if the buffer is empty a
+   * 0-byte PUT will be invoked, as it is needed to create an entry at the far

Review Comment:
   no need to reformat the entire javadoc, you don't want your IDE set to do 
this as it only makes cherrypicking harder



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -169,6 +169,9 @@ class S3ABlockOutputStream extends OutputStream implements
   /** Thread level IOStatistics Aggregator. */
   private final IOStatisticsAggregator threadIOStatisticsAggregator;
 
+  /**Is multipart upload allowed? */

Review Comment:
   nit, add a space after the **



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -1126,6 +1135,11 @@ public static final class BlockOutputStreamBuilder {
      */
     private IOStatisticsAggregator ioStatisticsAggregator;
 
+    /**
+     * Is Multipart Uploads enabled for the given upload

Review Comment:
   add a . to keep javadoc happy



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -1126,6 +1135,11 @@ public static final class BlockOutputStreamBuilder {
      */
     private IOStatisticsAggregator ioStatisticsAggregator;
 
+    /**
+     * Is Multipart Uploads enabled for the given upload
+     */
+    private boolean isMultipartAllowed;

Review Comment:
   rename isMultipartEnabled



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -1276,5 +1290,11 @@ public BlockOutputStreamBuilder 
withIOStatisticsAggregator(
       ioStatisticsAggregator = value;
       return this;
     }
+
+    public BlockOutputStreamBuilder withMultipartAllowed(

Review Comment:
   again, rename



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;

Review Comment:
   and add java.* block above the others.



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+
+public class ITestS3AHugeFileUpload extends S3AScaleTestBase{
+  final private Logger LOG = LoggerFactory.getLogger(
+      ITestS3AHugeFileUpload.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);
+    configuration.setInt(KEY_TEST_TIMEOUT, 36000);
+    configuration.setInt(IO_CHUNK_BUFFER_SIZE, 655360);
+    configuration.set("fs.s3a.connection.request.timeout", "1h");

Review Comment:
   use the relevant constant



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java:
##########
@@ -1031,6 +1031,19 @@ public static long 
getMultipartSizeProperty(Configuration conf,
     return partSize;
   }
 
+  public static boolean checkDiskBuffer(Configuration conf){

Review Comment:
   1. javadocs
   2. add a space after the ) and {



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+
+public class ITestS3AHugeFileUpload extends S3AScaleTestBase{
+  final private Logger LOG = LoggerFactory.getLogger(
+      ITestS3AHugeFileUpload.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);
+    configuration.setInt(KEY_TEST_TIMEOUT, 36000);
+    configuration.setInt(IO_CHUNK_BUFFER_SIZE, 655360);
+    configuration.set("fs.s3a.connection.request.timeout", "1h");
+    return configuration;
+  }
+
+  @Test
+  public void uploadFileSinglePut() throws IOException {
+    LOG.info("Creating file with size : {}", fileSize);
+    ContractTestUtils.createAndVerifyFile(getFileSystem(),

Review Comment:
   after the upload, use the iostatistics of the fs to verify that only one PUT 
operation took place, and therefore that the operation worked



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+
+public class ITestS3AHugeFileUpload extends S3AScaleTestBase{
+  final private Logger LOG = LoggerFactory.getLogger(
+      ITestS3AHugeFileUpload.class.getName());
+
+  private long fileSize = Integer.MAX_VALUE * 2L;

Review Comment:
   is this going to be configurable? as i might want to make the size smaller 
so my tests don't time out as much (in fact: I absolutely WILL because I run 
with -Dscale most of the time)
   
   look at AbstractSTestS3AHugeFiles.setup() to see how it picks up the size. 
use the same configuration option to control the size in this test case.



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -1831,6 +1830,11 @@ private FSDataOutputStream innerCreateFile(
     final PutObjectOptions putOptions =
         new PutObjectOptions(keep, null, options.getHeaders());
 
+    if(!checkDiskBuffer(getConf())){
+      throw new IOException("The filesystem conf is not " +

Review Comment:
   that's not a particularly useful error message. better to say which options 
are inconsistent



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##########
@@ -1255,4 +1255,8 @@ private Constants() {
    */
   public static final String PREFETCH_BLOCK_COUNT_KEY = 
"fs.s3a.prefetch.block.count";
   public static final int PREFETCH_BLOCK_DEFAULT_COUNT = 8;
+
+  public static final String MULTIPART_UPLOADS_ENABLED = 
"fs.s3a.multipart.uploads.enabled";

Review Comment:
   add javadocs for these constants with `{@value}` in them, so IDEs will show 
what they mean and the generated docs show the strings
   
   also add a mention in the "How S3A writes data to S3" in 
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;

Review Comment:
   split import blocks, into apache and non apache, put the non-apache one first



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+
+public class ITestS3AHugeFileUpload extends S3AScaleTestBase{
+  final private Logger LOG = LoggerFactory.getLogger(
+      ITestS3AHugeFileUpload.class.getName());
+
+  private long fileSize = Integer.MAX_VALUE * 2L;
+  @Override

Review Comment:
   nit: newline between these two



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileUpload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static 
org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+
+public class ITestS3AHugeFileUpload extends S3AScaleTestBase{

Review Comment:
   * add a javadoc to see what it is doing.
   * give it a name to indicate it is doing a single put, e.g 
`ITestS3AHugeFileUploadSinglePut`
   * add a space before {



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -369,6 +373,9 @@ private synchronized void uploadCurrentBlock(boolean isLast)
    */
   @Retries.RetryTranslated
   private void initMultipartUpload() throws IOException {
+    if (!isMultipartAllowed){
+      return;

Review Comment:
   good q. we should throw it because if something does get as far as calling 
it, something has gone very wrong.
   
   I would just use 
   ```
   Preconditions.checkState(!isMultipartEnabled, "multipart upload is disabled")
   ```
   





> 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