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