Packages

abstract class FileCommitProtocol extends Logging

An interface to define how a single Spark job commits its outputs. Three notes:

1. Implementations must be serializable, as the committer instance instantiated on the driver will be used for tasks on executors. 2. Implementations should have a constructor with 2 or 3 arguments: (jobId: String, path: String) or (jobId: String, path: String, dynamicPartitionOverwrite: Boolean) 3. A committer should not be reused across multiple Spark jobs.

The proper call sequence is:

1. Driver calls setupJob. 2. As part of each task's execution, executor calls setupTask and then commitTask (or abortTask if task failed). 3. When all necessary tasks completed successfully, the driver calls commitJob. If the job failed to execute (e.g. too many failed tasks), the job should call abortJob.

Source
FileCommitProtocol.scala
Linear Supertypes
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. FileCommitProtocol
  2. Logging
  3. AnyRef
  4. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

Instance Constructors

  1. new FileCommitProtocol()

Abstract Value Members

  1. abstract def abortJob(jobContext: JobContext): Unit

    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.

  2. abstract def abortTask(taskContext: TaskAttemptContext): Unit

    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.

  3. abstract def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit

    Commits a job after the writes succeed.

    Commits a job after the writes succeed. Must be called on the driver.

  4. abstract def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage

    Commits a task after the writes succeed.

    Commits a task after the writes succeed. Must be called on the executors when running tasks.

  5. abstract def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String

    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:

    1. the base path 2. some sub-directory within the base path, used to specify partitioning 3. file prefix, usually some unique job id with the task id 4. bucket id 5. source specific file extension, e.g. ".snappy.parquet"

    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.

  6. abstract def newTaskTempFileAbsPath(taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String

    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.

  7. abstract def setupJob(jobContext: JobContext): Unit

    Setups up a job.

    Setups up a job. Must be called on the driver before any other methods can be invoked.

  8. abstract def setupTask(taskContext: TaskAttemptContext): Unit

    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.

Concrete Value Members

  1. final def !=(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  4. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  5. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()
  6. def deleteWithJob(fs: FileSystem, path: Path, recursive: Boolean): Boolean

    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.

  7. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  8. def equals(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  9. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  10. final def getClass(): Class[_]
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  11. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  12. def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit
    Definition Classes
    Logging
  13. def initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean = false): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  14. def initializeLogIfNecessary(isInterpreter: Boolean): Unit
    Attributes
    protected
    Definition Classes
    Logging
  15. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  16. def isTraceEnabled(): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  17. def log: Logger
    Attributes
    protected
    Definition Classes
    Logging
  18. def logDebug(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  19. def logDebug(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  20. def logError(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  21. def logError(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  22. def logInfo(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  23. def logInfo(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  24. def logName: String
    Attributes
    protected
    Definition Classes
    Logging
  25. def logTrace(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  26. def logTrace(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  27. def logWarning(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  28. def logWarning(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  29. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  30. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  31. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  32. def onTaskCommit(taskCommit: TaskCommitMessage): Unit

    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.

  33. final def synchronized[T0](arg0: ⇒ T0): T0
    Definition Classes
    AnyRef
  34. def toString(): String
    Definition Classes
    AnyRef → Any
  35. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  36. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  37. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()

Inherited from Logging

Inherited from AnyRef

Inherited from Any

Ungrouped