-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-18637:S3A to support upload of files greater than 2 GB using DiskBlocks #5481
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
03a8c8b
768f41b
ddde1e6
542ffc2
773e03a
58a0453
5d3f9d9
a2d25f6
f381b88
ca725f9
ea0007f
1f56e2a
4e922b4
13fc2d5
1476424
f18c0cb
7207fdd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -414,6 +414,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, | |
| */ | ||
| private ArnResource accessPoint; | ||
|
|
||
| /** | ||
| * Is this S3A FS instance has multipart uploads enabled? | ||
|
||
| */ | ||
| private boolean isMultipartEnabled; | ||
|
|
||
| /** | ||
| * A cache of files that should be deleted when the FileSystem is closed | ||
| * or the JVM is exited. | ||
|
|
@@ -533,6 +538,8 @@ public void initialize(URI name, Configuration originalConf) | |
| this.prefetchBlockSize = (int) prefetchBlockSizeLong; | ||
| this.prefetchBlockCount = | ||
| intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); | ||
| this.isMultipartEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, | ||
| MULTIPART_UPLOAD_ENABLED_DEFAULT); | ||
|
|
||
| initThreadPools(conf); | ||
|
|
||
|
|
@@ -1080,6 +1087,7 @@ protected RequestFactory createRequestFactory() { | |
| .withRequestPreparer(getAuditManager()::requestCreated) | ||
| .withContentEncoding(contentEncoding) | ||
| .withStorageClass(storageClass) | ||
| .withMultipartEnabled(isMultipartEnabled) | ||
| .build(); | ||
| } | ||
|
|
||
|
|
@@ -1831,8 +1839,8 @@ private FSDataOutputStream innerCreateFile( | |
| new PutObjectOptions(keep, null, options.getHeaders()); | ||
|
|
||
| if(!checkDiskBuffer(getConf())){ | ||
|
||
| throw new IOException("The filesystem conf is not " + | ||
| "proper for the output stream"); | ||
| throw new IOException("Unable to create OutputStream with the given" | ||
| + " multipart upload and buffer configuration."); | ||
| } | ||
|
|
||
| final S3ABlockOutputStream.BlockOutputStreamBuilder builder = | ||
|
|
@@ -1859,8 +1867,7 @@ private FSDataOutputStream innerCreateFile( | |
| .withPutOptions(putOptions) | ||
| .withIOStatisticsAggregator( | ||
| IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator()) | ||
| .withMultipartEnabled(getConf().getBoolean( | ||
| MULTIPART_UPLOADS_ENABLED, MULTIPART_UPLOAD_ENABLED_DEFAULT)); | ||
| .withMultipartEnabled(isMultipartEnabled); | ||
| return new FSDataOutputStream( | ||
| new S3ABlockOutputStream(builder), | ||
| null); | ||
|
|
@@ -5418,4 +5425,8 @@ public RequestFactory getRequestFactory() { | |
| public boolean isCSEEnabled() { | ||
| return isCSEEnabled; | ||
| } | ||
|
|
||
| public boolean isMultipartEnabled() { | ||
| return isMultipartEnabled; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -124,6 +124,11 @@ public class RequestFactoryImpl implements RequestFactory { | |
| */ | ||
| private final StorageClass storageClass; | ||
|
|
||
| /** | ||
| * Is Multipart Enabled | ||
HarshitGupta11 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| */ | ||
| private final boolean isMultipartEnabled; | ||
|
|
||
| /** | ||
| * Constructor. | ||
| * @param builder builder with all the configuration. | ||
|
|
@@ -137,6 +142,7 @@ protected RequestFactoryImpl( | |
| this.requestPreparer = builder.requestPreparer; | ||
| this.contentEncoding = builder.contentEncoding; | ||
| this.storageClass = builder.storageClass; | ||
| this.isMultipartEnabled = builder.isMultipartEnabled; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -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){ | ||
HarshitGupta11 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| throw new IOException("Multipart uploads are disabled on the given filesystem."); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. make a PathIOException and include destkey. This gives a bit more detail. |
||
| } | ||
| final ObjectMetadata objectMetadata = newObjectMetadata(-1); | ||
| maybeSetMetadata(options, objectMetadata); | ||
| final InitiateMultipartUploadRequest initiateMPURequest = | ||
|
|
@@ -682,6 +691,11 @@ public static final class RequestFactoryBuilder { | |
| */ | ||
| private PrepareRequest requestPreparer; | ||
|
|
||
| /** | ||
| * Is Multipart Enabled on the path. | ||
| */ | ||
| private boolean isMultipartEnabled = true; | ||
|
|
||
| private RequestFactoryBuilder() { | ||
| } | ||
|
|
||
|
|
@@ -767,6 +781,18 @@ public RequestFactoryBuilder withRequestPreparer( | |
| this.requestPreparer = value; | ||
| return this; | ||
| } | ||
|
|
||
| /** | ||
| * Multipart enabled | ||
HarshitGupta11 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| * | ||
| * @param value new value | ||
| * @return the builder | ||
| */ | ||
| public RequestFactoryBuilder withMultipartEnabled( | ||
| final boolean value) { | ||
| this.isMultipartEnabled = value; | ||
| return this; | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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); | ||
HarshitGupta11 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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); | ||
HarshitGupta11 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| assertThrows(PathCommitException.class, | ||
HarshitGupta11 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| () -> new MagicS3GuardCommitter(commitPath, tContext)); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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); | ||
HarshitGupta11 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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, | ||
|
||
| () -> new StagingCommitter(commitPath, tContext)); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.