[
https://issues.apache.org/jira/browse/HADOOP-18637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17709502#comment-17709502
]
ASF GitHub Bot commented on HADOOP-18637:
-----------------------------------------
mukund-thakur commented on code in PR #5481:
URL: https://github.com/apache/hadoop/pull/5481#discussion_r1160262354
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:
##########
@@ -124,6 +124,11 @@ public class RequestFactoryImpl implements RequestFactory {
*/
private final StorageClass storageClass;
+ /**
+ * Is Multipart Enabled
Review Comment:
. in the end for javadoc.
##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.staging.integration;
+
+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.InternalCommitterConstants;
+import org.apache.hadoop.fs.s3a.commit.PathCommitException;
+import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter;
+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 ITestStagingCommitProtocolFailure 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,
InternalCommitterConstants.COMMITTER_NAME_STAGING);
+ 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:
same intercept.
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:
##########
@@ -124,6 +124,11 @@ public class RequestFactoryImpl implements RequestFactory {
*/
private final StorageClass storageClass;
+ /**
+ * Is Multipart Enabled
Review Comment:
Other config names clearly mentions it is multipart upload only. Why are we
not using isMultipartUploadEnabled here as well?
##########
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:
and it is beautifully integrated with other parts of tests well.
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:
##########
@@ -124,6 +124,11 @@ public class RequestFactoryImpl implements RequestFactory {
*/
private final StorageClass storageClass;
+ /**
+ * Is Multipart Enabled
Review Comment:
m small, is multipart upload enabled. As this is only for uploads not
downloads as well.
##########
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);
Review Comment:
log what are you printing.
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -1831,6 +1832,11 @@ private FSDataOutputStream innerCreateFile(
final PutObjectOptions putOptions =
new PutObjectOptions(keep, null, options.getHeaders());
+ if(!checkDiskBuffer(getConf())){
Review Comment:
This is still pending. I don't really mind leaving it as it is but I think
my suggestion is consistent with other parts of the code and is more readable.
CC @steveloughran
##########
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:
+1
> 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]