Class/Object

org.apache.spark

SparkContext

Related Docs: object SparkContext | package spark

Permalink

class SparkContext extends Logging

Main entry point for Spark functionality. A SparkContext represents the connection to a Spark cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.

Only one SparkContext may be active per JVM. You must stop() the active SparkContext before creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.

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

Instance Constructors

  1. new SparkContext(master: String, appName: String, sparkHome: String = null, jars: Seq[String] = Nil, environment: Map[String, String] = Map())

    Permalink

    Alternative constructor that allows setting common Spark properties directly

    Alternative constructor that allows setting common Spark properties directly

    master

    Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).

    appName

    A name for your application, to display on the cluster web UI.

    sparkHome

    Location where Spark is installed on cluster nodes.

    jars

    Collection of JARs to send to the cluster. These can be paths on the local file system or HDFS, HTTP, HTTPS, or FTP URLs.

    environment

    Environment variables to set on worker nodes.

  2. new SparkContext(master: String, appName: String, conf: SparkConf)

    Permalink

    Alternative constructor that allows setting common Spark properties directly

    Alternative constructor that allows setting common Spark properties directly

    master

    Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).

    appName

    A name for your application, to display on the cluster web UI

    conf

    a org.apache.spark.SparkConf object specifying other Spark parameters

  3. new SparkContext()

    Permalink

    Create a SparkContext that loads settings from system properties (for instance, when launching with ./bin/spark-submit).

  4. new SparkContext(config: SparkConf)

    Permalink

    config

    a Spark Config object describing the application configuration. Any settings in this config overrides the default configs as well as system properties.

Value Members

  1. final def !=(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int

    Permalink
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  4. def addFile(path: String, recursive: Boolean): Unit

    Permalink

    Add a file to be downloaded with this Spark job on every node.

    Add a file to be downloaded with this Spark job on every node.

    If a file is added during execution, it will not be available until the next TaskSet starts.

    path

    can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, use SparkFiles.get(fileName) to find its download location.

    recursive

    if true, a directory can be given in path. Currently directories are only supported for Hadoop-supported filesystems.

  5. def addFile(path: String): Unit

    Permalink

    Add a file to be downloaded with this Spark job on every node.

    Add a file to be downloaded with this Spark job on every node.

    If a file is added during execution, it will not be available until the next TaskSet starts.

    path

    can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, use SparkFiles.get(fileName) to find its download location.

  6. def addJar(path: String): Unit

    Permalink

    Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.

    Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.

    If a jar is added during execution, it will not be available until the next TaskSet starts.

    path

    can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node.

  7. def addSparkListener(listener: SparkListenerInterface): Unit

    Permalink

    :: DeveloperApi :: Register a listener to receive up-calls from events that happen during execution.

    :: DeveloperApi :: Register a listener to receive up-calls from events that happen during execution.

    Annotations
    @DeveloperApi()
  8. def appName: String

    Permalink
  9. def applicationAttemptId: Option[String]

    Permalink
  10. def applicationId: String

    Permalink

    A unique identifier for the Spark application.

    A unique identifier for the Spark application. Its format depends on the scheduler implementation. (i.e. in case of local spark app something like 'local-1433865536131' in case of YARN something like 'application_1433865536131_34483' in case of MESOS something like 'driver-20170926223339-0001' )

  11. final def asInstanceOf[T0]: T0

    Permalink
    Definition Classes
    Any
  12. def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): RDD[(String, PortableDataStream)]

    Permalink

    Get an RDD for a Hadoop-readable dataset as PortableDataStream for each file (useful for binary data)

    Get an RDD for a Hadoop-readable dataset as PortableDataStream for each file (useful for binary data)

    For example, if you have the following files:

    hdfs://a-hdfs-path/part-00000
    hdfs://a-hdfs-path/part-00001
    ...
    hdfs://a-hdfs-path/part-nnnnn

    Do val rdd = sparkContext.binaryFiles("hdfs://a-hdfs-path"),

    then rdd contains

    (a-hdfs-path/part-00000, its content)
    (a-hdfs-path/part-00001, its content)
    ...
    (a-hdfs-path/part-nnnnn, its content)
    path

    Directory to the input data files, the path can be comma separated paths as the list of inputs.

    minPartitions

    A suggestion value of the minimal splitting number for input data.

    returns

    RDD representing tuples of file path and corresponding file content

    Note

    Partitioning is determined by data locality. This may result in too few partitions by default.

    ,

    On some filesystems, .../path/* can be a more efficient way to read all files in a directory rather than .../path/ or .../path

    ,

    Small files are preferred; very large files may cause bad performance.

  13. def binaryRecords(path: String, recordLength: Int, conf: Configuration = hadoopConfiguration): RDD[Array[Byte]]

    Permalink

    Load data from a flat binary file, assuming the length of each record is constant.

    Load data from a flat binary file, assuming the length of each record is constant.

    path

    Directory to the input data files, the path can be comma separated paths as the list of inputs.

    recordLength

    The length at which to split the records

    conf

    Configuration for setting up the dataset.

    returns

    An RDD of data with values, represented as byte arrays

    Note

    We ensure that the byte array for each record in the resulting RDD has the provided record length.

  14. def broadcast[T](value: T)(implicit arg0: ClassTag[T]): Broadcast[T]

    Permalink

    Broadcast a read-only variable to the cluster, returning a org.apache.spark.broadcast.Broadcast object for reading it in distributed functions.

    Broadcast a read-only variable to the cluster, returning a org.apache.spark.broadcast.Broadcast object for reading it in distributed functions. The variable will be sent to each cluster only once.

    value

    value to broadcast to the Spark nodes

    returns

    Broadcast object, a read-only variable cached on each machine

  15. def cancelAllJobs(): Unit

    Permalink

    Cancel all jobs that have been scheduled or are running.

  16. def cancelJob(jobId: Int): Unit

    Permalink

    Cancel a given job if it's scheduled or running.

    Cancel a given job if it's scheduled or running.

    jobId

    the job ID to cancel

    Note

    Throws InterruptedException if the cancel message cannot be sent

  17. def cancelJob(jobId: Int, reason: String): Unit

    Permalink

    Cancel a given job if it's scheduled or running.

    Cancel a given job if it's scheduled or running.

    jobId

    the job ID to cancel

    reason

    optional reason for cancellation

    Note

    Throws InterruptedException if the cancel message cannot be sent

  18. def cancelJobGroup(groupId: String): Unit

    Permalink

    Cancel active jobs for the specified group.

    Cancel active jobs for the specified group. See org.apache.spark.SparkContext.setJobGroup for more information.

  19. def cancelStage(stageId: Int): Unit

    Permalink

    Cancel a given stage and all jobs associated with it.

    Cancel a given stage and all jobs associated with it.

    stageId

    the stage ID to cancel

    Note

    Throws InterruptedException if the cancel message cannot be sent

  20. def cancelStage(stageId: Int, reason: String): Unit

    Permalink

    Cancel a given stage and all jobs associated with it.

    Cancel a given stage and all jobs associated with it.

    stageId

    the stage ID to cancel

    reason

    reason for cancellation

    Note

    Throws InterruptedException if the cancel message cannot be sent

  21. def checkpointFile[T](path: String)(implicit arg0: ClassTag[T]): RDD[T]

    Permalink
    Attributes
    protected[org.apache.spark]
  22. def clearCallSite(): Unit

    Permalink

    Clear the thread-local property for overriding the call sites of actions and RDDs.

  23. def clearJobGroup(): Unit

    Permalink

    Clear the current thread's job group ID and its description.

  24. def clone(): AnyRef

    Permalink
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  25. def collectionAccumulator[T](name: String): CollectionAccumulator[T]

    Permalink

    Create and register a CollectionAccumulator, which starts with empty list and accumulates inputs by adding them into the list.

  26. def collectionAccumulator[T]: CollectionAccumulator[T]

    Permalink

    Create and register a CollectionAccumulator, which starts with empty list and accumulates inputs by adding them into the list.

  27. def defaultMinPartitions: Int

    Permalink

    Default min number of partitions for Hadoop RDDs when not given by user Notice that we use math.min so the "defaultMinPartitions" cannot be higher than 2.

    Default min number of partitions for Hadoop RDDs when not given by user Notice that we use math.min so the "defaultMinPartitions" cannot be higher than 2. The reasons for this are discussed in https://github.com/mesos/spark/pull/718

  28. def defaultParallelism: Int

    Permalink

    Default level of parallelism to use when not given by user (e.g.

    Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD).

  29. def deployMode: String

    Permalink
  30. def doubleAccumulator(name: String): DoubleAccumulator

    Permalink

    Create and register a double accumulator, which starts with 0 and accumulates inputs by add.

  31. def doubleAccumulator: DoubleAccumulator

    Permalink

    Create and register a double accumulator, which starts with 0 and accumulates inputs by add.

  32. def emptyRDD[T](implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Get an RDD that has no partitions or elements.

  33. final def eq(arg0: AnyRef): Boolean

    Permalink
    Definition Classes
    AnyRef
  34. def equals(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  35. def files: Seq[String]

    Permalink
  36. def finalize(): Unit

    Permalink
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  37. def getAllPools: Seq[Schedulable]

    Permalink

    :: DeveloperApi :: Return pools for fair scheduler

    :: DeveloperApi :: Return pools for fair scheduler

    Annotations
    @DeveloperApi()
  38. def getCheckpointDir: Option[String]

    Permalink
  39. final def getClass(): Class[_]

    Permalink
    Definition Classes
    AnyRef → Any
  40. def getConf: SparkConf

    Permalink

    Return a copy of this SparkContext's configuration.

    Return a copy of this SparkContext's configuration. The configuration cannot be changed at runtime.

  41. def getExecutorMemoryStatus: Map[String, (Long, Long)]

    Permalink

    Return a map from the slave to the max memory available for caching and the remaining memory available for caching.

  42. def getLocalProperty(key: String): String

    Permalink

    Get a local property set in this thread, or null if it is missing.

    Get a local property set in this thread, or null if it is missing. See org.apache.spark.SparkContext.setLocalProperty.

  43. def getPersistentRDDs: Map[Int, RDD[_]]

    Permalink

    Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.

    Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.

    Note

    This does not necessarily mean the caching or computation was successful.

  44. def getPoolForName(pool: String): Option[Schedulable]

    Permalink

    :: DeveloperApi :: Return the pool associated with the given name, if one exists

    :: DeveloperApi :: Return the pool associated with the given name, if one exists

    Annotations
    @DeveloperApi()
  45. def getRDDStorageInfo: Array[RDDInfo]

    Permalink

    :: DeveloperApi :: Return information about what RDDs are cached, if they are in mem or on disk, how much space they take, etc.

    :: DeveloperApi :: Return information about what RDDs are cached, if they are in mem or on disk, how much space they take, etc.

    Annotations
    @DeveloperApi()
  46. def getSchedulingMode: SchedulingMode

    Permalink

    Return current scheduling mode

  47. def hadoopConfiguration: Configuration

    Permalink

    A default Hadoop Configuration for the Hadoop code (e.g.

    A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse.

    Note

    As it will be reused in all Hadoop RDDs, it's better not to modify it unless you plan to set some global configurations for all Hadoop RDDs.

  48. def hadoopFile[K, V, F <: InputFormat[K, V]](path: String)(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)]

    Permalink

    Smarter version of hadoopFile() that uses class tags to figure out the classes of keys, values and the InputFormat so that users don't need to pass them directly.

    Smarter version of hadoopFile() that uses class tags to figure out the classes of keys, values and the InputFormat so that users don't need to pass them directly. Instead, callers can just write, for example,

    val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path)
    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  49. def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minPartitions: Int)(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)]

    Permalink

    Smarter version of hadoopFile() that uses class tags to figure out the classes of keys, values and the InputFormat so that users don't need to pass them directly.

    Smarter version of hadoopFile() that uses class tags to figure out the classes of keys, values and the InputFormat so that users don't need to pass them directly. Instead, callers can just write, for example,

    val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minPartitions)
    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    minPartitions

    suggested minimum number of partitions for the resulting RDD

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  50. def hadoopFile[K, V](path: String, inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], minPartitions: Int = defaultMinPartitions): RDD[(K, V)]

    Permalink

    Get an RDD for a Hadoop file with an arbitrary InputFormat

    Get an RDD for a Hadoop file with an arbitrary InputFormat

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    inputFormatClass

    storage format of the data to be read

    keyClass

    Class of the key associated with the inputFormatClass parameter

    valueClass

    Class of the value associated with the inputFormatClass parameter

    minPartitions

    suggested minimum number of partitions for the resulting RDD

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  51. def hadoopRDD[K, V](conf: JobConf, inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], minPartitions: Int = defaultMinPartitions): RDD[(K, V)]

    Permalink

    Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other necessary info (e.g.

    Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), using the older MapReduce API (org.apache.hadoop.mapred).

    conf

    JobConf for setting up the dataset. Note: This will be put into a Broadcast. Therefore if you plan to reuse this conf to create multiple RDDs, you need to make sure you won't modify the conf. A safe approach is always creating a new conf for a new RDD.

    inputFormatClass

    storage format of the data to be read

    keyClass

    Class of the key associated with the inputFormatClass parameter

    valueClass

    Class of the value associated with the inputFormatClass parameter

    minPartitions

    Minimum number of Hadoop Splits to generate.

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  52. def hashCode(): Int

    Permalink
    Definition Classes
    AnyRef → Any
  53. def initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean = false): Boolean

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  54. def initializeLogIfNecessary(isInterpreter: Boolean): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  55. final def isInstanceOf[T0]: Boolean

    Permalink
    Definition Classes
    Any
  56. def isLocal: Boolean

    Permalink
  57. def isStopped: Boolean

    Permalink

    returns

    true if context is stopped or in the midst of stopping.

  58. def isTraceEnabled(): Boolean

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  59. def jars: Seq[String]

    Permalink
  60. def killExecutor(executorId: String): Boolean

    Permalink

    :: DeveloperApi :: Request that the cluster manager kill the specified executor.

    :: DeveloperApi :: Request that the cluster manager kill the specified executor.

    returns

    whether the request is received.

    Annotations
    @DeveloperApi()
    Note

    This is an indication to the cluster manager that the application wishes to adjust its resource usage downwards. If the application wishes to replace the executor it kills through this method with a new one, it should follow up explicitly with a call to {{SparkContext#requestExecutors}}.

  61. def killExecutors(executorIds: Seq[String]): Boolean

    Permalink

    :: DeveloperApi :: Request that the cluster manager kill the specified executors.

    :: DeveloperApi :: Request that the cluster manager kill the specified executors.

    returns

    whether the request is received.

    Annotations
    @DeveloperApi()
    Note

    This is an indication to the cluster manager that the application wishes to adjust its resource usage downwards. If the application wishes to replace the executors it kills through this method with new ones, it should follow up explicitly with a call to {{SparkContext#requestExecutors}}.

  62. def killTaskAttempt(taskId: Long, interruptThread: Boolean = true, reason: String = ...): Boolean

    Permalink

    Kill and reschedule the given task attempt.

    Kill and reschedule the given task attempt. Task ids can be obtained from the Spark UI or through SparkListener.onTaskStart.

    taskId

    the task ID to kill. This id uniquely identifies the task attempt.

    interruptThread

    whether to interrupt the thread running the task.

    reason

    the reason for killing the task, which should be a short string. If a task is killed multiple times with different reasons, only one reason will be reported.

    returns

    Whether the task was successfully killed.

  63. def listFiles(): Seq[String]

    Permalink

    Returns a list of file paths that are added to resources.

  64. def listJars(): Seq[String]

    Permalink

    Returns a list of jar files that are added to resources.

  65. val localProperties: InheritableThreadLocal[Properties]

    Permalink
    Attributes
    protected[org.apache.spark]
  66. def log: Logger

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  67. def logDebug(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  68. def logDebug(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  69. def logError(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  70. def logError(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  71. def logInfo(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  72. def logInfo(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  73. def logName: String

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  74. def logTrace(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  75. def logTrace(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  76. def logWarning(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  77. def logWarning(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  78. def longAccumulator(name: String): LongAccumulator

    Permalink

    Create and register a long accumulator, which starts with 0 and accumulates inputs by add.

  79. def longAccumulator: LongAccumulator

    Permalink

    Create and register a long accumulator, which starts with 0 and accumulates inputs by add.

  80. def makeRDD[T](seq: Seq[(T, Seq[String])])(implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Distribute a local Scala collection to form an RDD, with one or more location preferences (hostnames of Spark nodes) for each object.

    Distribute a local Scala collection to form an RDD, with one or more location preferences (hostnames of Spark nodes) for each object. Create a new partition for each collection item.

    seq

    list of tuples of data and location preferences (hostnames of Spark nodes)

    returns

    RDD representing data partitioned according to location preferences

  81. def makeRDD[T](seq: Seq[T], numSlices: Int = defaultParallelism)(implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Distribute a local Scala collection to form an RDD.

    Distribute a local Scala collection to form an RDD.

    This method is identical to parallelize.

    seq

    Scala collection to distribute

    numSlices

    number of partitions to divide the collection into

    returns

    RDD representing distributed collection

  82. def master: String

    Permalink
  83. final def ne(arg0: AnyRef): Boolean

    Permalink
    Definition Classes
    AnyRef
  84. def newAPIHadoopFile[K, V, F <: InputFormat[K, V]](path: String, fClass: Class[F], kClass: Class[K], vClass: Class[V], conf: Configuration = hadoopConfiguration): RDD[(K, V)]

    Permalink

    Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.

    Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    fClass

    storage format of the data to be read

    kClass

    Class of the key associated with the fClass parameter

    vClass

    Class of the value associated with the fClass parameter

    conf

    Hadoop configuration

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  85. def newAPIHadoopFile[K, V, F <: InputFormat[K, V]](path: String)(implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)]

    Permalink

    Smarter version of newApiHadoopFile that uses class tags to figure out the classes of keys, values and the org.apache.hadoop.mapreduce.InputFormat (new MapReduce API) so that user don't need to pass them directly.

    Smarter version of newApiHadoopFile that uses class tags to figure out the classes of keys, values and the org.apache.hadoop.mapreduce.InputFormat (new MapReduce API) so that user don't need to pass them directly. Instead, callers can just write, for example: val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path)

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  86. def newAPIHadoopRDD[K, V, F <: InputFormat[K, V]](conf: Configuration = hadoopConfiguration, fClass: Class[F], kClass: Class[K], vClass: Class[V]): RDD[(K, V)]

    Permalink

    Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.

    Get an RDD for a given Hadoop file with an arbitrary new API InputFormat and extra configuration options to pass to the input format.

    conf

    Configuration for setting up the dataset. Note: This will be put into a Broadcast. Therefore if you plan to reuse this conf to create multiple RDDs, you need to make sure you won't modify the conf. A safe approach is always creating a new conf for a new RDD.

    fClass

    storage format of the data to be read

    kClass

    Class of the key associated with the fClass parameter

    vClass

    Class of the value associated with the fClass parameter

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  87. final def notify(): Unit

    Permalink
    Definition Classes
    AnyRef
  88. final def notifyAll(): Unit

    Permalink
    Definition Classes
    AnyRef
  89. def objectFile[T](path: String, minPartitions: Int = defaultMinPartitions)(implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and BytesWritable values that contain a serialized partition.

    Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and BytesWritable values that contain a serialized partition. This is still an experimental storage format and may not be supported exactly as is in future Spark releases. It will also be pretty slow if you use the default serializer (Java serialization), though the nice thing about it is that there's very little effort required to save arbitrary objects.

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    minPartitions

    suggested minimum number of partitions for the resulting RDD

    returns

    RDD representing deserialized data from the file(s)

  90. def parallelize[T](seq: Seq[T], numSlices: Int = defaultParallelism)(implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Distribute a local Scala collection to form an RDD.

    Distribute a local Scala collection to form an RDD.

    seq

    Scala collection to distribute

    numSlices

    number of partitions to divide the collection into

    returns

    RDD representing distributed collection

    Note

    avoid using parallelize(Seq()) to create an empty RDD. Consider emptyRDD for an RDD with no partitions, or parallelize(Seq[T]()) for an RDD of T with empty partitions.

    ,

    Parallelize acts lazily. If seq is a mutable collection and is altered after the call to parallelize and before the first action on the RDD, the resultant RDD will reflect the modified collection. Pass a copy of the argument to avoid this.

  91. def range(start: Long, end: Long, step: Long = 1, numSlices: Int = defaultParallelism): RDD[Long]

    Permalink

    Creates a new RDD[Long] containing elements from start to end(exclusive), increased by step every element.

    Creates a new RDD[Long] containing elements from start to end(exclusive), increased by step every element.

    start

    the start value.

    end

    the end value.

    step

    the incremental step

    numSlices

    number of partitions to divide the collection into

    returns

    RDD representing distributed range

    Note

    if we need to cache this RDD, we should make sure each partition does not exceed limit.

  92. def register(acc: AccumulatorV2[_, _], name: String): Unit

    Permalink

    Register the given accumulator with given name.

    Register the given accumulator with given name.

    Note

    Accumulators must be registered before use, or it will throw exception.

  93. def register(acc: AccumulatorV2[_, _]): Unit

    Permalink

    Register the given accumulator.

    Register the given accumulator.

    Note

    Accumulators must be registered before use, or it will throw exception.

  94. def removeSparkListener(listener: SparkListenerInterface): Unit

    Permalink

    :: DeveloperApi :: Deregister the listener from Spark's listener bus.

    :: DeveloperApi :: Deregister the listener from Spark's listener bus.

    Annotations
    @DeveloperApi()
  95. def requestExecutors(numAdditionalExecutors: Int): Boolean

    Permalink

    :: DeveloperApi :: Request an additional number of executors from the cluster manager.

    :: DeveloperApi :: Request an additional number of executors from the cluster manager.

    returns

    whether the request is received.

    Annotations
    @DeveloperApi()
  96. def requestTotalExecutors(numExecutors: Int, localityAwareTasks: Int, hostToLocalTaskCount: Map[String, Int]): Boolean

    Permalink

    Update the cluster manager on our scheduling needs.

    Update the cluster manager on our scheduling needs. Three bits of information are included to help it make decisions.

    numExecutors

    The total number of executors we'd like to have. The cluster manager shouldn't kill any running executor to reach this number, but, if all existing executors were to die, this is the number of executors we'd want to be allocated.

    localityAwareTasks

    The number of tasks in all active stages that have a locality preferences. This includes running, pending, and completed tasks.

    hostToLocalTaskCount

    A map of hosts to the number of tasks from all active stages that would like to like to run on that host. This includes running, pending, and completed tasks.

    returns

    whether the request is acknowledged by the cluster manager.

    Annotations
    @DeveloperApi()
  97. def runApproximateJob[T, U, R](rdd: RDD[T], func: (TaskContext, Iterator[T]) ⇒ U, evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R]

    Permalink

    :: DeveloperApi :: Run a job that can return approximate results.

    :: DeveloperApi :: Run a job that can return approximate results.

    rdd

    target RDD to run tasks on

    func

    a function to run on each partition of the RDD

    evaluator

    ApproximateEvaluator to receive the partial results

    timeout

    maximum time to wait for the job, in milliseconds

    returns

    partial result (how partial depends on whether the job was finished before or after timeout)

    Annotations
    @DeveloperApi()
  98. def runJob[T, U](rdd: RDD[T], processPartition: (Iterator[T]) ⇒ U, resultHandler: (Int, U) ⇒ Unit)(implicit arg0: ClassTag[U]): Unit

    Permalink

    Run a job on all partitions in an RDD and pass the results to a handler function.

    Run a job on all partitions in an RDD and pass the results to a handler function.

    rdd

    target RDD to run tasks on

    processPartition

    a function to run on each partition of the RDD

    resultHandler

    callback to pass each result to

  99. def runJob[T, U](rdd: RDD[T], processPartition: (TaskContext, Iterator[T]) ⇒ U, resultHandler: (Int, U) ⇒ Unit)(implicit arg0: ClassTag[U]): Unit

    Permalink

    Run a job on all partitions in an RDD and pass the results to a handler function.

    Run a job on all partitions in an RDD and pass the results to a handler function. The function that is run against each partition additionally takes TaskContext argument.

    rdd

    target RDD to run tasks on

    processPartition

    a function to run on each partition of the RDD

    resultHandler

    callback to pass each result to

  100. def runJob[T, U](rdd: RDD[T], func: (Iterator[T]) ⇒ U)(implicit arg0: ClassTag[U]): Array[U]

    Permalink

    Run a job on all partitions in an RDD and return the results in an array.

    Run a job on all partitions in an RDD and return the results in an array.

    rdd

    target RDD to run tasks on

    func

    a function to run on each partition of the RDD

    returns

    in-memory collection with a result of the job (each collection element will contain a result from one partition)

  101. def runJob[T, U](rdd: RDD[T], func: (TaskContext, Iterator[T]) ⇒ U)(implicit arg0: ClassTag[U]): Array[U]

    Permalink

    Run a job on all partitions in an RDD and return the results in an array.

    Run a job on all partitions in an RDD and return the results in an array. The function that is run against each partition additionally takes TaskContext argument.

    rdd

    target RDD to run tasks on

    func

    a function to run on each partition of the RDD

    returns

    in-memory collection with a result of the job (each collection element will contain a result from one partition)

  102. def runJob[T, U](rdd: RDD[T], func: (Iterator[T]) ⇒ U, partitions: Seq[Int])(implicit arg0: ClassTag[U]): Array[U]

    Permalink

    Run a function on a given set of partitions in an RDD and return the results as an array.

    Run a function on a given set of partitions in an RDD and return the results as an array.

    rdd

    target RDD to run tasks on

    func

    a function to run on each partition of the RDD

    partitions

    set of partitions to run on; some jobs may not want to compute on all partitions of the target RDD, e.g. for operations like first()

    returns

    in-memory collection with a result of the job (each collection element will contain a result from one partition)

  103. def runJob[T, U](rdd: RDD[T], func: (TaskContext, Iterator[T]) ⇒ U, partitions: Seq[Int])(implicit arg0: ClassTag[U]): Array[U]

    Permalink

    Run a function on a given set of partitions in an RDD and return the results as an array.

    Run a function on a given set of partitions in an RDD and return the results as an array. The function that is run against each partition additionally takes TaskContext argument.

    rdd

    target RDD to run tasks on

    func

    a function to run on each partition of the RDD

    partitions

    set of partitions to run on; some jobs may not want to compute on all partitions of the target RDD, e.g. for operations like first()

    returns

    in-memory collection with a result of the job (each collection element will contain a result from one partition)

  104. def runJob[T, U](rdd: RDD[T], func: (TaskContext, Iterator[T]) ⇒ U, partitions: Seq[Int], resultHandler: (Int, U) ⇒ Unit)(implicit arg0: ClassTag[U]): Unit

    Permalink

    Run a function on a given set of partitions in an RDD and pass the results to the given handler function.

    Run a function on a given set of partitions in an RDD and pass the results to the given handler function. This is the main entry point for all actions in Spark.

    rdd

    target RDD to run tasks on

    func

    a function to run on each partition of the RDD

    partitions

    set of partitions to run on; some jobs may not want to compute on all partitions of the target RDD, e.g. for operations like first()

    resultHandler

    callback to pass each result to

  105. def sequenceFile[K, V](path: String, minPartitions: Int = defaultMinPartitions)(implicit km: ClassTag[K], vm: ClassTag[V], kcf: () ⇒ WritableConverter[K], vcf: () ⇒ WritableConverter[V]): RDD[(K, V)]

    Permalink

    Version of sequenceFile() for types implicitly convertible to Writables through a WritableConverter.

    Version of sequenceFile() for types implicitly convertible to Writables through a WritableConverter. For example, to access a SequenceFile where the keys are Text and the values are IntWritable, you could simply write

    sparkContext.sequenceFile[String, Int](path, ...)

    WritableConverters are provided in a somewhat strange way (by an implicit function) to support both subclasses of Writable and types for which we define a converter (e.g. Int to IntWritable). The most natural thing would've been to have implicit objects for the converters, but then we couldn't have an object for every subclass of Writable (you can't have a parameterized singleton object). We use functions instead to create a new converter for the appropriate type. In addition, we pass the converter a ClassTag of its type to allow it to figure out the Writable class to use in the subclass case.

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    minPartitions

    suggested minimum number of partitions for the resulting RDD

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  106. def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)]

    Permalink

    Get an RDD for a Hadoop SequenceFile with given key and value types.

    Get an RDD for a Hadoop SequenceFile with given key and value types.

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    keyClass

    Class of the key associated with SequenceFileInputFormat

    valueClass

    Class of the value associated with SequenceFileInputFormat

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  107. def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V], minPartitions: Int): RDD[(K, V)]

    Permalink

    Get an RDD for a Hadoop SequenceFile with given key and value types.

    Get an RDD for a Hadoop SequenceFile with given key and value types.

    path

    directory to the input data files, the path can be comma separated paths as a list of inputs

    keyClass

    Class of the key associated with SequenceFileInputFormat

    valueClass

    Class of the value associated with SequenceFileInputFormat

    minPartitions

    suggested minimum number of partitions for the resulting RDD

    returns

    RDD of tuples of key and corresponding value

    Note

    Because Hadoop's RecordReader class re-uses the same Writable object for each record, directly caching the returned RDD or directly passing it to an aggregation or shuffle operation will create many references to the same object. If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first copy them using a map function.

  108. def setCallSite(shortCallSite: String): Unit

    Permalink

    Set the thread-local property for overriding the call sites of actions and RDDs.

  109. def setCheckpointDir(directory: String): Unit

    Permalink

    Set the directory under which RDDs are going to be checkpointed.

    Set the directory under which RDDs are going to be checkpointed.

    directory

    path to the directory where checkpoint files will be stored (must be HDFS path if running in cluster)

  110. def setJobDescription(value: String): Unit

    Permalink

    Set a human readable description of the current job.

  111. def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean = false): Unit

    Permalink

    Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared.

    Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared.

    Often, a unit of execution in an application consists of multiple Spark actions or jobs. Application programmers can use this method to group all those jobs together and give a group description. Once set, the Spark web UI will associate such jobs with this group.

    The application can also use org.apache.spark.SparkContext.cancelJobGroup to cancel all running jobs in this group. For example,

    // In the main thread:
    sc.setJobGroup("some_job_to_cancel", "some job description")
    sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count()
    
    // In a separate thread:
    sc.cancelJobGroup("some_job_to_cancel")
    interruptOnCancel

    If true, then job cancellation will result in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead.

  112. def setLocalProperty(key: String, value: String): Unit

    Permalink

    Set a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool.

    Set a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool. User-defined properties may also be set here. These properties are propagated through to worker tasks and can be accessed there via org.apache.spark.TaskContext#getLocalProperty.

    These properties are inherited by child threads spawned from this thread. This may have unexpected consequences when working with thread pools. The standard java implementation of thread pools have worker threads spawn other worker threads. As a result, local properties may propagate unpredictably.

  113. def setLogLevel(logLevel: String): Unit

    Permalink

    Control our logLevel.

    Control our logLevel. This overrides any user-defined log settings.

    logLevel

    The desired log level as a string. Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN

  114. val sparkUser: String

    Permalink
  115. val startTime: Long

    Permalink
  116. def statusTracker: SparkStatusTracker

    Permalink
  117. def stop(): Unit

    Permalink

    Shut down the SparkContext.

  118. def submitJob[T, U, R](rdd: RDD[T], processPartition: (Iterator[T]) ⇒ U, partitions: Seq[Int], resultHandler: (Int, U) ⇒ Unit, resultFunc: ⇒ R): SimpleFutureAction[R]

    Permalink

    Submit a job for execution and return a FutureJob holding the result.

    Submit a job for execution and return a FutureJob holding the result.

    rdd

    target RDD to run tasks on

    processPartition

    a function to run on each partition of the RDD

    partitions

    set of partitions to run on; some jobs may not want to compute on all partitions of the target RDD, e.g. for operations like first()

    resultHandler

    callback to pass each result to

    resultFunc

    function to be executed when the result is ready

  119. final def synchronized[T0](arg0: ⇒ T0): T0

    Permalink
    Definition Classes
    AnyRef
  120. def textFile(path: String, minPartitions: Int = defaultMinPartitions): RDD[String]

    Permalink

    Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings.

    Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings.

    path

    path to the text file on a supported file system

    minPartitions

    suggested minimum number of partitions for the resulting RDD

    returns

    RDD of lines of the text file

  121. def toString(): String

    Permalink
    Definition Classes
    AnyRef → Any
  122. def uiWebUrl: Option[String]

    Permalink
  123. def union[T](first: RDD[T], rest: RDD[T]*)(implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Build the union of a list of RDDs passed as variable-length arguments.

  124. def union[T](rdds: Seq[RDD[T]])(implicit arg0: ClassTag[T]): RDD[T]

    Permalink

    Build the union of a list of RDDs.

  125. def version: String

    Permalink

    The version of Spark on which this application is running.

  126. final def wait(): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  127. final def wait(arg0: Long, arg1: Int): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  128. final def wait(arg0: Long): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  129. def wholeTextFiles(path: String, minPartitions: Int = defaultMinPartitions): RDD[(String, String)]

    Permalink

    Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI.

    Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file.

    For example, if you have the following files:

    hdfs://a-hdfs-path/part-00000
    hdfs://a-hdfs-path/part-00001
    ...
    hdfs://a-hdfs-path/part-nnnnn

    Do val rdd = sparkContext.wholeTextFile("hdfs://a-hdfs-path"),

    then rdd contains

    (a-hdfs-path/part-00000, its content)
    (a-hdfs-path/part-00001, its content)
    ...
    (a-hdfs-path/part-nnnnn, its content)
    path

    Directory to the input data files, the path can be comma separated paths as the list of inputs.

    minPartitions

    A suggestion value of the minimal splitting number for input data.

    returns

    RDD representing tuples of file path and the corresponding file content

    Note

    Partitioning is determined by data locality. This may result in too few partitions by default.

    ,

    On some filesystems, .../path/* can be a more efficient way to read all files in a directory rather than .../path/ or .../path

    ,

    Small files are preferred, large file is also allowable, but may cause bad performance.

Deprecated Value Members

  1. def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]): Accumulable[R, T]

    Permalink

    Create an org.apache.spark.Accumulable shared variable, with a name for display in the Spark UI.

    Create an org.apache.spark.Accumulable shared variable, with a name for display in the Spark UI. Tasks can add values to the accumulable using the += operator. Only the driver can access the accumulable's value.

    R

    accumulator result type

    T

    type that can be added to the accumulator

    Annotations
    @deprecated
    Deprecated

    (Since version 2.0.0) use AccumulatorV2

  2. def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]): Accumulable[R, T]

    Permalink

    Create an org.apache.spark.Accumulable shared variable, to which tasks can add values with +=.

    Create an org.apache.spark.Accumulable shared variable, to which tasks can add values with +=. Only the driver can access the accumulable's value.

    R

    accumulator result type

    T

    type that can be added to the accumulator

    Annotations
    @deprecated
    Deprecated

    (Since version 2.0.0) use AccumulatorV2

  3. def accumulableCollection[R, T](initialValue: R)(implicit arg0: (R) ⇒ Growable[T] with TraversableOnce[T] with Serializable, arg1: ClassTag[R]): Accumulable[R, T]

    Permalink

    Create an accumulator from a "mutable collection" type.

    Create an accumulator from a "mutable collection" type.

    Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by standard mutable collections. So you can use this with mutable Map, Set, etc.

    Annotations
    @deprecated
    Deprecated

    (Since version 2.0.0) use AccumulatorV2

  4. def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]): Accumulator[T]

    Permalink

    Create an org.apache.spark.Accumulator variable of a given type, with a name for display in the Spark UI.

    Create an org.apache.spark.Accumulator variable of a given type, with a name for display in the Spark UI. Tasks can "add" values to the accumulator using the += method. Only the driver can access the accumulator's value.

    Annotations
    @deprecated
    Deprecated

    (Since version 2.0.0) use AccumulatorV2

  5. def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T]

    Permalink

    Create an org.apache.spark.Accumulator variable of a given type, which tasks can "add" values to using the += method.

    Create an org.apache.spark.Accumulator variable of a given type, which tasks can "add" values to using the += method. Only the driver can access the accumulator's value.

    Annotations
    @deprecated
    Deprecated

    (Since version 2.0.0) use AccumulatorV2

  6. def getExecutorStorageStatus: Array[StorageStatus]

    Permalink

    :: DeveloperApi :: Return information about blocks stored in all of the slaves

    :: DeveloperApi :: Return information about blocks stored in all of the slaves

    Annotations
    @DeveloperApi() @deprecated
    Deprecated

    (Since version 2.2.0) This method may change or be removed in a future release.

Inherited from Logging

Inherited from AnyRef

Inherited from Any

Ungrouped