[
https://issues.apache.org/jira/browse/HADOOP-19047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17816480#comment-17816480
]
ASF GitHub Bot commented on HADOOP-19047:
-----------------------------------------
shameersss1 commented on code in PR #6468:
URL: https://github.com/apache/hadoop/pull/6468#discussion_r1485744738
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/InMemoryMagicCommitTracker.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import software.amazon.awssdk.services.s3.model.CompletedPart;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.WriteOperationHelper;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.util.Preconditions;
+
+import static
org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTrackerUtils.extractTaskAttemptIdFromPath;
+
+/**
+ * InMemoryMagicCommitTracker stores the commit data in memory.
+ * The commit data and related data stores are flushed out from
+ * the memory when the task is committed or aborted.
+ */
+public class InMemoryMagicCommitTracker extends MagicCommitTracker {
+
+ // stores taskAttemptId to commit data mapping
Review Comment:
ack
> Support InMemory Tracking Of S3A Magic Commits
> ----------------------------------------------
>
> Key: HADOOP-19047
> URL: https://issues.apache.org/jira/browse/HADOOP-19047
> Project: Hadoop Common
> Issue Type: Improvement
> Components: fs/s3
> Reporter: Syed Shameerur Rahman
> Assignee: Syed Shameerur Rahman
> Priority: Major
> Labels: pull-request-available
>
> The following are the operations which happens within a Task when it uses S3A
> Magic Committer.
> *During closing of stream*
> 1. A 0-byte file with a same name of the original file is uploaded to S3
> using PUT operation. Refer
> [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L152]
> for more information. This is done so that the downstream application like
> Spark could get the size of the file which is being written.
> 2. MultiPartUpload(MPU) metadata is uploaded to S3. Refer
> [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java#L176]
> for more information.
> *During TaskCommit*
> 1. All the MPU metadata which the task wrote to S3 (There will be 'x' number
> of metadata file in S3 if a single task writes to 'x' files) are read and
> rewritten to S3 as a single metadata file. Refer
> [here|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java#L201]
> for more information
> Since these operations happens with the Task JVM, We could optimize as well
> as save cost by storing these information in memory when Task memory usage is
> not a constraint. Hence the proposal here is to introduce a new MagicCommit
> Tracker called "InMemoryMagicCommitTracker" which will store the
> 1. Metadata of MPU in memory till the Task is committed
> 2. Store the size of the file which can be used by the downstream application
> to get the file size before it is committed/visible to the output path.
> This optimization will save 2 PUT S3 calls, 1 LIST S3 call, and 1 GET S3 call
> given a Task writes only 1 file.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]