Alternative constructor that allows setting common Spark properties directly
Alternative constructor that allows setting common Spark properties directly
Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
A name for your application, to display on the cluster web UI
a org.apache.spark.SparkConf object specifying other Spark parameters
Create a SparkContext that loads settings from system properties (for instance, when launching with .
Create a SparkContext that loads settings from system properties (for instance, when launching with ./bin/spark-submit).
a Spark Config object describing the application configuration. Any settings in this config overrides the default configs as well as system properties.
Alternative constructor that allows setting common Spark properties directly
Alternative constructor that allows setting common Spark properties directly
Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
A name for your application, to display on the cluster web UI.
Location where Spark is installed on cluster nodes.
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 variables to set on worker nodes.
not used. Left for backward compatibility.
(Since version 1.6.0) Passing in preferred locations has no effect at all, see SPARK-10921
:: DeveloperApi :: Alternative constructor for setting preferred locations where Spark will create executors.
:: DeveloperApi :: Alternative constructor for setting preferred locations where Spark will create executors.
a org.apache.spark.SparkConf object specifying other Spark parameters
not used. Left for backward compatibility.
(Since version 1.5.0) Passing in preferred locations has no effect at all, see SPARK-8949
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 accumuable using the +=
operator. Only the driver can
access the accumuable's value
.
accumulator result type
type that can be added to the accumulator
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 accumuable's value
.
accumulator result type
type that can be added to the accumulator
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.
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
.
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
.
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.
The path
passed 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.
A directory can be given if the recursive option is set to true. Currently directories are only supported for Hadoop-supported filesystems.
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.
The path
passed 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.
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.
The path
passed 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.
:: 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.
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' )
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)
Directory to the input data files, the path can be comma separated paths as the list of inputs.
A suggestion value of the minimal splitting number for input data.
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.
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.
Note: We ensure that the byte array for each record in the resulting RDD has the provided record length.
Directory to the input data files, the path can be comma separated paths as the list of inputs.
The length at which to split the records
Configuration for setting up the dataset.
An RDD of data with values, represented as byte arrays
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.
Cancel all jobs that have been scheduled or are running.
Cancel active jobs for the specified group.
Cancel active jobs for the specified group. See org.apache.spark.SparkContext.setJobGroup for more information.
Clear the thread-local property for overriding the call sites of actions and RDDs.
Clear the current thread's job group ID and its description.
Default min number of partitions for Hadoop RDDs when not given by user Notice that we use math.
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
Default level of parallelism to use when not given by user (e.
Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD).
Get an RDD that has no partitions or elements.
:: DeveloperApi :: Return pools for fair scheduler
:: DeveloperApi :: Return pools for fair scheduler
Return a copy of this SparkContext's configuration.
Return a copy of this SparkContext's configuration. The configuration cannot be changed at runtime.
Return a map from the slave to the max memory available for caching and the remaining memory available for caching.
:: DeveloperApi :: Return information about blocks stored in all of the slaves
:: DeveloperApi :: Return information about blocks stored in all of the slaves
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.
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 that this does not necessarily mean the caching or computation was successful.
:: DeveloperApi :: Return the pool associated with the given name, if one exists
:: DeveloperApi :: Return the pool associated with the given name, if one exists
:: 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.
Return current scheduling mode
A default Hadoop Configuration for the Hadoop code (e.
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.
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)
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.
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)
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.
Get an RDD for a Hadoop file with an arbitrary InputFormat
Get an RDD for a Hadoop file with an arbitrary InputFormat
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.
Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other necessary info (e.
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
).
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.
Class of the InputFormat
Class of the keys
Class of the values
Minimum number of Hadoop Splits to generate.
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.
true if context is stopped or in the midst of stopping.
:: DeveloperApi :: Request that the cluster manager kill the specified executor.
:: DeveloperApi :: Request that the cluster manager kill the specified executor.
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}}.
whether the request is received.
:: DeveloperApi :: Request that the cluster manager kill the specified executors.
:: DeveloperApi :: Request that the cluster manager kill the specified executors.
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}}.
whether the request is received.
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.
Distribute a local Scala collection to form an RDD.
Distribute a local Scala collection to form an RDD.
This method is identical to parallelize
.
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.
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.
Get an RDD for a Hadoop file with an arbitrary new API InputFormat.
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.
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.
Class of the InputFormat
Class of the keys
Class of the values
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.
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.
Distribute a local Scala collection to form an RDD.
Distribute a local Scala collection to form an RDD.
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.
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.
the start value.
the end value.
the incremental step
the partition number of the new RDD.
if we need to cache this RDD, we should make sure each partition does not exceed limit.
:: DeveloperApi :: Request an additional number of executors from the cluster manager.
:: DeveloperApi :: Request an additional number of executors from the cluster manager.
whether the request is received.
:: DeveloperApi :: Run a job that can return approximate results.
:: DeveloperApi :: Run a job that can return approximate results.
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.
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.
Run a job on a given set of partitions of an RDD, but take a function of type
Iterator[T] => U
instead of (TaskContext, Iterator[T]) => U
.
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 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.
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.
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.
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.
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.
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.
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.
Set the thread-local property for overriding the call sites of actions and RDDs.
Set the directory under which RDDs are going to be checkpointed.
Set the directory under which RDDs are going to be checkpointed. The directory must be a HDFS path if running on a cluster.
Set a human readable description of the current job.
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")
If interruptOnCancel is set to true for the job group, 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.
Set a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool.
Control our logLevel.
Control our logLevel. This overrides any user-defined log settings.
The desired log level as a string. Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN
Submit a job for execution and return a FutureJob holding the result.
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.
Build the union of a list of RDDs passed as variable-length arguments.
Build the union of a list of RDDs.
The version of Spark on which this application is running.
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)
Directory to the input data files, the path can be comma separated paths as the list of inputs.
A suggestion value of the minimal splitting number for input data.
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.
Clear the job's list of files added by addFile
so that they do not get downloaded to
any new nodes.
Clear the job's list of files added by addFile
so that they do not get downloaded to
any new nodes.
(Since version 1.0.0) adding files no longer creates local copies that need to be deleted
Clear the job's list of JARs added by addJar
so that they do not get downloaded to
any new nodes.
Clear the job's list of JARs added by addJar
so that they do not get downloaded to
any new nodes.
(Since version 1.0.0) adding jars no longer creates local copies that need to be deleted
Default min number of partitions for Hadoop RDDs when not given by user
Default min number of partitions for Hadoop RDDs when not given by user
(Since version 1.0.0) use defaultMinPartitions
(Since version 1.0.0) Properties no longer need to be explicitly initialized.
Run a job on a given set of partitions of an RDD, but take a function of type
Iterator[T] => U
instead of (TaskContext, Iterator[T]) => U
.
Run a job on a given set of partitions of an RDD, but take a function of type
Iterator[T] => U
instead of (TaskContext, Iterator[T]) => U
.
The allowLocal argument is deprecated as of Spark 1.5.0+.
(Since version 1.5.0) use the version of runJob without the allowLocal parameter
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 allowLocal flag is deprecated as of Spark 1.5.0+.
(Since version 1.5.0) use the version of runJob without the allowLocal parameter
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.
The allowLocal flag is deprecated as of Spark 1.5.0+.
(Since version 1.5.0) use the version of runJob without the allowLocal parameter
(Since version 1.4.0) Use externalBlockStoreFolderName instead.
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.