Skip to content

Commit 032796a

Browse files
authored
HADOOP-19047: S3A: Support in-memory tracking of Magic Commit data (#6468)
If the option fs.s3a.committer.magic.track.commits.in.memory.enabled is set to true, then rather than save data about in-progress uploads to S3, this information is cached in memory. If the number of files being committed is low, this will save network IO in both the generation of .pending and marker files, and in the scanning of task attempt directory trees during task commit. Contributed by Syed Shameerur Rahman
1 parent 9fe371a commit 032796a

File tree

13 files changed

+626
-110
lines changed

13 files changed

+626
-110
lines changed

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -117,6 +117,7 @@
117117
import org.apache.hadoop.fs.s3a.auth.SignerManager;
118118
import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations;
119119
import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
120+
import org.apache.hadoop.fs.s3a.commit.magic.InMemoryMagicCommitTracker;
120121
import org.apache.hadoop.fs.s3a.impl.AWSCannedACL;
121122
import org.apache.hadoop.fs.s3a.impl.AWSHeaders;
122123
import org.apache.hadoop.fs.s3a.impl.BulkDeleteRetryHandler;
@@ -231,6 +232,8 @@
231232
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding;
232233
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_ABORT_PENDING_UPLOADS;
233234
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS;
235+
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_PENDING_OBJECT_ETAG_NAME;
236+
import static org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTrackerUtils.isTrackMagicCommitsInMemoryEnabled;
234237
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
235238
import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_NO_OVERWRITE;
236239
import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_OVERWRITE;
@@ -3900,6 +3903,21 @@ public void access(final Path f, final FsAction mode)
39003903
@Retries.RetryTranslated
39013904
public FileStatus getFileStatus(final Path f) throws IOException {
39023905
Path path = qualify(f);
3906+
if (isTrackMagicCommitsInMemoryEnabled(getConf()) && isMagicCommitPath(path)) {
3907+
// Some downstream apps might call getFileStatus for a magic path to get the file size.
3908+
// when commit data is stored in memory construct the dummy S3AFileStatus with correct
3909+
// file size fetched from the memory.
3910+
if (InMemoryMagicCommitTracker.getPathToBytesWritten().containsKey(path)) {
3911+
long len = InMemoryMagicCommitTracker.getPathToBytesWritten().get(path);
3912+
return new S3AFileStatus(len,
3913+
0L,
3914+
path,
3915+
getDefaultBlockSize(path),
3916+
username,
3917+
MAGIC_COMMITTER_PENDING_OBJECT_ETAG_NAME,
3918+
null);
3919+
}
3920+
}
39033921
return trackDurationAndSpan(
39043922
INVOCATION_GET_FILE_STATUS, path, () ->
39053923
innerGetFileStatus(path, false, StatusProbeEnum.ALL));

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,10 @@ private CommitConstants() {
5858
*/
5959
public static final String PENDINGSET_SUFFIX = ".pendingset";
6060

61+
/**
62+
* Etag name to be returned on non-committed S3 object: {@value}.
63+
*/
64+
public static final String MAGIC_COMMITTER_PENDING_OBJECT_ETAG_NAME = "pending";
6165

6266
/**
6367
* Prefix to use for config options: {@value}.
@@ -242,6 +246,18 @@ private CommitConstants() {
242246
*/
243247
public static final int DEFAULT_COMMITTER_THREADS = 32;
244248

249+
/**
250+
* Should Magic committer track all the pending commits in memory?
251+
*/
252+
public static final String FS_S3A_COMMITTER_MAGIC_TRACK_COMMITS_IN_MEMORY_ENABLED =
253+
"fs.s3a.committer.magic.track.commits.in.memory.enabled";
254+
255+
/**
256+
* Default value for {@link #FS_S3A_COMMITTER_MAGIC_TRACK_COMMITS_IN_MEMORY_ENABLED}: {@value}.
257+
*/
258+
public static final boolean FS_S3A_COMMITTER_MAGIC_TRACK_COMMITS_IN_MEMORY_ENABLED_DEFAULT =
259+
false;
260+
245261
/**
246262
* Path in the cluster filesystem for temporary data: {@value}.
247263
* This is for HDFS, not the local filesystem.

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -26,11 +26,13 @@
2626
import org.apache.hadoop.fs.Path;
2727
import org.apache.hadoop.fs.s3a.S3AFileSystem;
2828
import org.apache.hadoop.fs.s3a.Statistic;
29-
import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
29+
import org.apache.hadoop.fs.s3a.commit.magic.InMemoryMagicCommitTracker;
30+
import org.apache.hadoop.fs.s3a.commit.magic.S3MagicCommitTracker;
3031
import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation;
3132
import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics;
3233

3334
import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
35+
import static org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTrackerUtils.isTrackMagicCommitsInMemoryEnabled;
3436

3537
/**
3638
* Adds the code needed for S3A to support magic committers.
@@ -105,13 +107,15 @@ public PutTracker createTracker(Path path, String key,
105107
String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
106108
getStoreContext().incrementStatistic(
107109
Statistic.COMMITTER_MAGIC_FILES_CREATED);
108-
tracker = new MagicCommitTracker(path,
109-
getStoreContext().getBucket(),
110-
key,
111-
destKey,
112-
pendingsetPath,
113-
owner.getWriteOperationHelper(),
114-
trackerStatistics);
110+
if (isTrackMagicCommitsInMemoryEnabled(getStoreContext().getConfiguration())) {
111+
tracker = new InMemoryMagicCommitTracker(path, getStoreContext().getBucket(),
112+
key, destKey, pendingsetPath, owner.getWriteOperationHelper(),
113+
trackerStatistics);
114+
} else {
115+
tracker = new S3MagicCommitTracker(path, getStoreContext().getBucket(),
116+
key, destKey, pendingsetPath, owner.getWriteOperationHelper(),
117+
trackerStatistics);
118+
}
115119
LOG.debug("Created {}", tracker);
116120
} else {
117121
LOG.warn("File being created has a \"magic\" path, but the filesystem"
Lines changed: 146 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,146 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.fs.s3a.commit.magic;
20+
21+
import java.io.IOException;
22+
import java.util.ArrayList;
23+
import java.util.Collections;
24+
import java.util.List;
25+
import java.util.Map;
26+
import java.util.concurrent.ConcurrentHashMap;
27+
28+
import software.amazon.awssdk.services.s3.model.CompletedPart;
29+
30+
import org.apache.commons.lang3.StringUtils;
31+
import org.apache.hadoop.fs.Path;
32+
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
33+
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
34+
import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics;
35+
import org.apache.hadoop.fs.statistics.IOStatistics;
36+
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
37+
import org.apache.hadoop.util.Preconditions;
38+
39+
import static org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTrackerUtils.extractTaskAttemptIdFromPath;
40+
41+
/**
42+
* InMemoryMagicCommitTracker stores the commit data in memory.
43+
* The commit data and related data stores are flushed out from
44+
* the memory when the task is committed or aborted.
45+
*/
46+
public class InMemoryMagicCommitTracker extends MagicCommitTracker {
47+
48+
/**
49+
* Map to store taskAttemptId, and it's corresponding list of pending commit data.
50+
* The entries in the Map gets removed when a task commits or aborts.
51+
*/
52+
private final static Map<String, List<SinglePendingCommit>> TASK_ATTEMPT_ID_TO_MPU_METADATA = new ConcurrentHashMap<>();
53+
54+
/**
55+
* Map to store path of the file, and it's corresponding size.
56+
* The entries in the Map gets removed when a task commits or aborts.
57+
*/
58+
private final static Map<Path, Long> PATH_TO_BYTES_WRITTEN = new ConcurrentHashMap<>();
59+
60+
/**
61+
* Map to store taskAttemptId, and list of paths to files written by it.
62+
* The entries in the Map gets removed when a task commits or aborts.
63+
*/
64+
private final static Map<String, List<Path>> TASK_ATTEMPT_ID_TO_PATH = new ConcurrentHashMap<>();
65+
66+
public InMemoryMagicCommitTracker(Path path,
67+
String bucket,
68+
String originalDestKey,
69+
String destKey,
70+
String pendingsetKey,
71+
WriteOperationHelper writer,
72+
PutTrackerStatistics trackerStatistics) {
73+
super(path, bucket, originalDestKey, destKey, pendingsetKey, writer, trackerStatistics);
74+
}
75+
76+
@Override
77+
public boolean aboutToComplete(String uploadId,
78+
List<CompletedPart> parts,
79+
long bytesWritten,
80+
final IOStatistics iostatistics)
81+
throws IOException {
82+
Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId),
83+
"empty/null upload ID: " + uploadId);
84+
Preconditions.checkArgument(parts != null, "No uploaded parts list");
85+
Preconditions.checkArgument(!parts.isEmpty(), "No uploaded parts to save");
86+
87+
// build the commit summary
88+
SinglePendingCommit commitData = new SinglePendingCommit();
89+
commitData.touch(System.currentTimeMillis());
90+
commitData.setDestinationKey(getDestKey());
91+
commitData.setBucket(getBucket());
92+
commitData.setUri(getPath().toUri().toString());
93+
commitData.setUploadId(uploadId);
94+
commitData.setText("");
95+
commitData.setLength(bytesWritten);
96+
commitData.bindCommitData(parts);
97+
commitData.setIOStatistics(new IOStatisticsSnapshot(iostatistics));
98+
99+
// extract the taskAttemptId from the path
100+
String taskAttemptId = extractTaskAttemptIdFromPath(getPath());
101+
102+
// store the commit data with taskAttemptId as the key
103+
TASK_ATTEMPT_ID_TO_MPU_METADATA.computeIfAbsent(taskAttemptId,
104+
k -> Collections.synchronizedList(new ArrayList<>())).add(commitData);
105+
106+
// store the byteswritten(length) for the corresponding file
107+
PATH_TO_BYTES_WRITTEN.put(getPath(), bytesWritten);
108+
109+
// store the mapping between taskAttemptId and path
110+
// This information is used for removing entries from
111+
// the map once the taskAttempt is completed/committed.
112+
TASK_ATTEMPT_ID_TO_PATH.computeIfAbsent(taskAttemptId,
113+
k -> Collections.synchronizedList(new ArrayList<>())).add(getPath());
114+
115+
LOG.info("commit metadata for {} parts in {}. size: {} byte(s) "
116+
+ "for the taskAttemptId: {} is stored in memory",
117+
parts.size(), getPendingPartKey(), bytesWritten, taskAttemptId);
118+
LOG.debug("Closed MPU to {}, saved commit information to {}; data=:\n{}",
119+
getPath(), getPendingPartKey(), commitData);
120+
121+
return false;
122+
}
123+
124+
@Override
125+
public String toString() {
126+
final StringBuilder sb = new StringBuilder(
127+
"InMemoryMagicCommitTracker{");
128+
sb.append(", Number of taskAttempts=").append(TASK_ATTEMPT_ID_TO_MPU_METADATA.size());
129+
sb.append(", Number of files=").append(PATH_TO_BYTES_WRITTEN.size());
130+
sb.append('}');
131+
return sb.toString();
132+
}
133+
134+
135+
public static Map<String, List<SinglePendingCommit>> getTaskAttemptIdToMpuMetadata() {
136+
return TASK_ATTEMPT_ID_TO_MPU_METADATA;
137+
}
138+
139+
public static Map<Path, Long> getPathToBytesWritten() {
140+
return PATH_TO_BYTES_WRITTEN;
141+
}
142+
143+
public static Map<String, List<Path>> getTaskAttemptIdToPath() {
144+
return TASK_ATTEMPT_ID_TO_PATH;
145+
}
146+
}

0 commit comments

Comments
 (0)