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

ASF GitHub Bot logged work on MAPREDUCE-7366:
---------------------------------------------

                Author: ASF GitHub Bot
            Created on: 25/Oct/21 16:15
            Start Date: 25/Oct/21 16:15
    Worklog Time Spent: 10m 
      Work Description: steveloughran commented on pull request #3582:
URL: https://github.com/apache/hadoop/pull/3582#issuecomment-951085584


   I recognize the problem. However I don't want this patch. Sorry.
   
   1. We are scared of making any changes to FileOutputCommitter as it is a 
critical part of the workflow of so many applications.
   1. You are using the v2 commit algorithm; its strategy of moving files 
during task commit means that it's task commit phase lacks the atomicity which 
spark and MapReduce expect. That is: v2 is utterly broken and you should stop 
using it. Please. (look for WONTFIX JIRAs related to proposals PRs to 
completely close it, )
   1. v1 contains some big assumptions about being able to rename source dirs 
to dest dirs if the dest path doesn't exist. It's job commit can't be executed 
in parallel for that reason.
   
   The WiP manifest committer of MAPREDUCE-7366 #2971 does aim to offer faster 
job commit than v1 all very large real world directly structures as well as 
task commits which are fast and atomic on azure & gcs storage.
   
   As no data is moved into the dest path until job commit all job execution 
would be safe to execute concurrently.
   It would be interesting to consider whether that job commit could work with 
>1 job in parallel. I think it's possible if every job used exactly the same 
dir tree structure (i.e. partitioning) and every generated file was guaranteed 
to have a unique name.
   
   Why not look at that PR #2971 and see if you could add a patch there to 
support a configurable pending dir?
   
   The patch would need to include at least one test in the protocol test suite 
where this happened and you would actually need to simulate the stage of 
parallel job execution of "consistent" jobs to verify that if the 
pre-conditions are met then the output would contain the results of both jobs.
   
   We also need a section in the documentation explain the option, when it is 
useful and the risks associated with it.
   
   Yes. That's tests and docs. Tests to make sure things work and don't break 
in the future; docs so that people can use it without looking for every line of 
the code. That is extra homework but at least right now that PR is still open 
to changes, while we leave FileOutputCommitter safely untouched
   
   (also, any testing of that manifest committer would be really welcome)
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 669635)
    Time Spent: 0.5h  (was: 20m)

> FileOutputCommitter Enable Concurent Writes 
> --------------------------------------------
>
>                 Key: MAPREDUCE-7366
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7366
>             Project: Hadoop Map/Reduce
>          Issue Type: Improvement
>            Reporter: ismail
>            Priority: Major
>              Labels: committers, easyfix, easytask, pull-request-available
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> is it possible to make `{{PENDING_DIR_NAME}}` configurable? 
> That will enable concurrent writes to same location. current if two spark 
> processes write same destination one of them is failing.
> current
> {code:java}
>  public static final String PENDING_DIR_NAME = "_temporary";{code}
> new:
> {code:java}
> PENDING_DIR_NAME = conf.get("mapreduce.fileoutputcommitter.pending.dir", 
> "_temporary");{code}
> here is custom commiter doing it: 
> https://gist.github.com/ismailsimsek/33c55d8e1fcfc79160483c38a978edbd



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org

Reply via email to