[ 
https://issues.apache.org/jira/browse/HADOOP-17833?focusedWorklogId=778785&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-778785
 ]

ASF GitHub Bot logged work on HADOOP-17833:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 06/Jun/22 22:59
            Start Date: 06/Jun/22 22:59
    Worklog Time Spent: 10m 
      Work Description: mukund-thakur commented on code in PR #3289:
URL: https://github.com/apache/hadoop/pull/3289#discussion_r890553440


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -1599,69 +1604,88 @@ public FSDataOutputStream create(Path f, FsPermission 
permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
     final Path path = qualify(f);
+
     // the span will be picked up inside the output stream
     return trackDurationAndSpan(INVOCATION_CREATE, path, () ->

Review Comment:
   Isn't Audit span suppossed to be already present.
   



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -3612,7 +3697,7 @@ public boolean createEmptyDir(Path path, StoreContext 
storeContext)
           new MkdirOperation(
               storeContext,
               path,
-              createMkdirOperationCallbacks()));
+              createMkdirOperationCallbacks(), false));

Review Comment:
   Why not isMagicPath() call here? Are we sure that createEmptyDir() will 
never be called during Magic Comitter operations?



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -3970,63 +4062,64 @@ InitiateMultipartUploadResult initiateMultipartUpload(
 
   /**
    * Perform post-write actions.
-   * <p></p>
+   * <p>
    * This operation MUST be called after any PUT/multipart PUT completes
    * successfully.
-   * <p></p>
-   * The actions include:
-   * <ol>
-   *   <li>
-   *     Calling
-   *     {@link #deleteUnnecessaryFakeDirectories(Path)}
-   *     if directory markers are not being retained.
-   *   </li>
-   *   <li>
-   *     Updating any metadata store with details on the newly created
-   *     object.
-   *     </li>
-   * </ol>
+   * <p>
+   * The actions include calling
+   * {@link #deleteUnnecessaryFakeDirectories(Path)}
+   * if directory markers are not being retained.
    * @param key key written to
    * @param length  total length of file written
    * @param eTag eTag of the written object
    * @param versionId S3 object versionId of the written object
+   * @param putOptions put object options
    */
   @InterfaceAudience.Private
-  @Retries.RetryTranslated("Except if failOnMetadataWriteError=false, in which"
-      + " case RetryExceptionsSwallowed")
-  void finishedWrite(String key, long length, String eTag, String versionId) {
+  @Retries.RetryExceptionsSwallowed
+  void finishedWrite(
+      String key,
+      long length,
+      String eTag,
+      String versionId,
+      PutObjectOptions putOptions) {
     LOG.debug("Finished write to {}, len {}. etag {}, version {}",
         key, length, eTag, versionId);
-    Path p = keyToQualifiedPath(key);
     Preconditions.checkArgument(length >= 0, "content length is negative");
-    // kick off an async delete
-    CompletableFuture<?> deletion;
-    if (!keepDirectoryMarkers(p)) {
-      deletion = submit(
-          unboundedThreadPool, getActiveAuditSpan(),
-          () -> {
-            deleteUnnecessaryFakeDirectories(
-                p.getParent()
-            );
-            return null;
-          });
-    } else {
-      deletion = null;
+    if (!putOptions.isKeepMarkers()) {

Review Comment:
   Why are we changing async to sync call here?



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java:
##########
@@ -38,8 +39,8 @@
  * in this case:
  * <ol>
  *   <li>{@link #isMagicCommitPath(Path)} will always return false.</li>

Review Comment:
   nit: new method isUnderMagicPath needs to be added here as well.



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java:
##########
@@ -257,17 +257,25 @@ public PutObjectRequest createPutObjectRequest(String 
destKey,
    * Create a {@link PutObjectRequest} request to upload a file.
    * @param dest key to PUT to.
    * @param sourceFile source file
+   * @param headers optional map of custom headers.
    * @return the request
    */
   @Retries.OnceRaw
-  public PutObjectRequest createPutObjectRequest(String dest,
-      File sourceFile) {
+  public PutObjectRequest createPutObjectRequest(
+      String dest,
+      File sourceFile,
+      @Nullable final Map<String, String> headers) {

Review Comment:
   Why nullable? Empty map should be fine ?





Issue Time Tracking
-------------------

    Worklog Id:     (was: 778785)
    Time Spent: 9h 10m  (was: 9h)

> Improve Magic Committer Performance
> -----------------------------------
>
>                 Key: HADOOP-17833
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17833
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs/s3
>    Affects Versions: 3.3.1
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Minor
>              Labels: pull-request-available
>          Time Spent: 9h 10m
>  Remaining Estimate: 0h
>
> Magic committer tasks can be slow because every file created with 
> overwrite=false triggers a HEAD (verify there's no file) and a LIST (that 
> there's no dir). And because of delayed manifestations, it may not behave as 
> expected.
> ParquetOutputFormat is one example of a library which does this.
> we could fix parquet to use overwrite=true, but (a) there may be surprises in 
> other uses (b) it'd still leave the list and (c) do nothing for other formats 
> call
> Proposed: createFile() under a magic path to skip all probes for file/dir at 
> end of path
> Only a single task attempt Will be writing to that directory and it should 
> know what it is doing. If there is conflicting file names and parts across 
> tasks that won't even get picked up at this point. Oh and none of the 
> committers ever check for this: you'll get the last file manifested (s3a) or 
> renamed (file)
> If we skip the checks we will save 2 HTTP requests/file.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to