the job's or stage's id
the job's output path, or null if committer acts as a noop
If true, Spark will overwrite partition directories at runtime dynamically, i.e., we first write files under a staging directory with partition path, e.g. /path/to/staging/a=1/b=1/xxx.parquet. When committing the job, we first clean up the corresponding partition directories at destination path, e.g. /path/to/destination/a=1/b=1, and move files from staging directory to the corresponding partition directories under destination path.
Aborts a job after the writes fail.
Aborts a job after the writes fail. Must be called on the driver.
Calling this function is a best-effort attempt, because it is possible that the driver just crashes (or killed) before it can call abort.
Aborts a task after the writes have failed.
Aborts a task after the writes have failed. Must be called on the executors when running tasks.
Calling this function is a best-effort attempt, because it is possible that the executor just crashes (or killed) before it can call abort.
Commits a job after the writes succeed.
Commits a job after the writes succeed. Must be called on the driver.
Commits a task after the writes succeed.
Commits a task after the writes succeed. Must be called on the executors when running tasks.
Specifies that a file should be deleted with the commit of this job.
Specifies that a file should be deleted with the commit of this job. The default implementation deletes the file immediately.
Notifies the commit protocol to add a new file, and gets back the full path that should be used.
Notifies the commit protocol to add a new file, and gets back the full path that should be used. Must be called on the executors when running tasks.
Note that the returned temp file may have an arbitrary path. The commit protocol only promises that the file will be at the location specified by the arguments after job commit.
A full file path consists of the following parts:
The "dir" parameter specifies 2, and "ext" parameter specifies both 4 and 5, and the rest are left to the commit protocol implementation to decide.
Important: it is the caller's responsibility to add uniquely identifying content to "ext" if a task is going to write out multiple files to the same dir. The file commit protocol only guarantees that files written by different tasks will not conflict.
Similar to newTaskTempFile(), but allows files to committed to an absolute output location.
Similar to newTaskTempFile(), but allows files to committed to an absolute output location. Depending on the implementation, there may be weaker guarantees around adding files this way.
Important: it is the caller's responsibility to add uniquely identifying content to "ext" if a task is going to write out multiple files to the same dir. The file commit protocol only guarantees that files written by different tasks will not conflict.
Called on the driver after a task commits.
Called on the driver after a task commits. This can be used to access task commit messages before the job has finished. These same task commit messages will be passed to commitJob() if the entire job succeeds.
Setups up a job.
Setups up a job. Must be called on the driver before any other methods can be invoked.
Sets up a task within a job.
Sets up a task within a job. Must be called before any other task related methods can be invoked.
An FileCommitProtocol implementation backed by an underlying Hadoop OutputCommitter (from the newer mapreduce API, not the old mapred API).
Unlike Hadoop's OutputCommitter, this implementation is serializable.