public class SparkContext extends java.lang.Object implements Logging
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.
param: config a Spark Config object describing the application configuration. Any settings in this config overrides the default configs as well as system properties.
Modifier and Type | Class and Description |
---|---|
static class |
SparkContext.DoubleAccumulatorParam$ |
static class |
SparkContext.FloatAccumulatorParam$ |
static class |
SparkContext.IntAccumulatorParam$ |
static class |
SparkContext.LongAccumulatorParam$ |
Constructor and Description |
---|
SparkContext()
Create a SparkContext that loads settings from system properties (for instance, when
launching with ./bin/spark-submit).
|
SparkContext(SparkConf config) |
SparkContext(SparkConf config,
scala.collection.Map<java.lang.String,scala.collection.Set<SplitInfo>> preferredNodeLocationData)
:: DeveloperApi ::
Alternative constructor for setting preferred locations where Spark will create executors.
|
SparkContext(java.lang.String master,
java.lang.String appName,
SparkConf conf)
Alternative constructor that allows setting common Spark properties directly
|
SparkContext(java.lang.String master,
java.lang.String appName,
java.lang.String sparkHome,
scala.collection.Seq<java.lang.String> jars,
scala.collection.Map<java.lang.String,java.lang.String> environment,
scala.collection.Map<java.lang.String,scala.collection.Set<SplitInfo>> preferredNodeLocationData)
Alternative constructor that allows setting common Spark properties directly
|
Modifier and Type | Method and Description |
---|---|
<R,T> Accumulable<R,T> |
accumulable(R initialValue,
AccumulableParam<R,T> param)
Create an
Accumulable shared variable, to which tasks can add values
with += . |
<R,T> Accumulable<R,T> |
accumulable(R initialValue,
java.lang.String name,
AccumulableParam<R,T> param)
Create an
Accumulable shared variable, with a name for display in the
Spark UI. |
<R,T> Accumulable<R,T> |
accumulableCollection(R initialValue,
scala.Function1<R,scala.collection.generic.Growable<T>> evidence$9,
scala.reflect.ClassTag<R> evidence$10)
Create an accumulator from a "mutable collection" type.
|
<T> Accumulator<T> |
accumulator(T initialValue,
AccumulatorParam<T> param)
Create an
Accumulator variable of a given type, which tasks can "add"
values to using the += method. |
<T> Accumulator<T> |
accumulator(T initialValue,
java.lang.String name,
AccumulatorParam<T> param)
Create an
Accumulator variable of a given type, with a name for display
in the Spark UI. |
scala.collection.mutable.HashMap<java.lang.String,java.lang.Object> |
addedFiles() |
scala.collection.mutable.HashMap<java.lang.String,java.lang.Object> |
addedJars() |
void |
addFile(java.lang.String path)
Add a file to be downloaded with this Spark job on every node.
|
void |
addFile(java.lang.String path,
boolean recursive)
Add a file to be downloaded with this Spark job on every node.
|
void |
addJar(java.lang.String path)
Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
|
void |
addSparkListener(SparkListener listener)
:: DeveloperApi ::
Register a listener to receive up-calls from events that happen during execution.
|
scala.Option<java.lang.String> |
applicationAttemptId() |
java.lang.String |
applicationId()
A unique identifier for the Spark application.
|
java.lang.String |
appName() |
RDD<scala.Tuple2<java.lang.String,PortableDataStream>> |
binaryFiles(java.lang.String path,
int minPartitions)
:: Experimental ::
|
RDD<byte[]> |
binaryRecords(java.lang.String path,
int recordLength,
org.apache.hadoop.conf.Configuration conf)
:: Experimental ::
|
static org.apache.spark.WritableConverter<java.lang.Object> |
booleanWritableConverter() |
static org.apache.hadoop.io.BooleanWritable |
boolToBoolWritable(boolean b) |
<T> Broadcast<T> |
broadcast(T value,
scala.reflect.ClassTag<T> evidence$11)
Broadcast a read-only variable to the cluster, returning a
Broadcast object for reading it in distributed functions. |
static org.apache.hadoop.io.BytesWritable |
bytesToBytesWritable(byte[] aob) |
static org.apache.spark.WritableConverter<byte[]> |
bytesWritableConverter() |
void |
cancelAllJobs()
Cancel all jobs that have been scheduled or are running.
|
void |
cancelJobGroup(java.lang.String groupId)
Cancel active jobs for the specified group.
|
scala.Option<java.lang.String> |
checkpointDir() |
protected <T> RDD<T> |
checkpointFile(java.lang.String path,
scala.reflect.ClassTag<T> evidence$5) |
void |
clearCallSite()
Clear the thread-local property for overriding the call sites
of actions and RDDs.
|
void |
clearFiles()
Clear the job's list of files added by
addFile so that they do not get downloaded to
any new nodes. |
void |
clearJars()
Clear the job's list of JARs added by
addJar so that they do not get downloaded to
any new nodes. |
void |
clearJobGroup()
Clear the current thread's job group ID and its description.
|
int |
defaultMinPartitions()
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.
|
int |
defaultMinSplits()
Default min number of partitions for Hadoop RDDs when not given by user
|
int |
defaultParallelism()
Default level of parallelism to use when not given by user (e.g.
|
static DoubleRDDFunctions |
doubleRDDToDoubleRDDFunctions(RDD<java.lang.Object> rdd) |
static org.apache.hadoop.io.DoubleWritable |
doubleToDoubleWritable(double d) |
static org.apache.spark.WritableConverter<java.lang.Object> |
doubleWritableConverter() |
static java.lang.String |
DRIVER_IDENTIFIER()
Executor id for the driver.
|
<T> |
emptyRDD(scala.reflect.ClassTag<T> evidence$8)
Get an RDD that has no partitions or elements.
|
scala.collection.mutable.HashMap<java.lang.String,java.lang.String> |
executorEnvs() |
java.lang.String |
externalBlockStoreFolderName() |
scala.collection.Seq<java.lang.String> |
files() |
static org.apache.hadoop.io.FloatWritable |
floatToFloatWritable(float f) |
static org.apache.spark.WritableConverter<java.lang.Object> |
floatWritableConverter() |
scala.collection.Seq<org.apache.spark.scheduler.Schedulable> |
getAllPools()
:: DeveloperApi ::
Return pools for fair scheduler
|
scala.Option<java.lang.String> |
getCheckpointDir() |
SparkConf |
getConf()
Return a copy of this SparkContext's configuration.
|
scala.collection.Map<java.lang.String,scala.Tuple2<java.lang.Object,java.lang.Object>> |
getExecutorMemoryStatus()
Return a map from the slave to the max memory available for caching and the remaining
memory available for caching.
|
StorageStatus[] |
getExecutorStorageStatus()
:: DeveloperApi ::
Return information about blocks stored in all of the slaves
|
java.lang.String |
getLocalProperty(java.lang.String key)
Get a local property set in this thread, or null if it is missing.
|
static SparkContext |
getOrCreate()
This function may be used to get or instantiate a SparkContext and register it as a
singleton object.
|
static SparkContext |
getOrCreate(SparkConf config)
This function may be used to get or instantiate a SparkContext and register it as a
singleton object.
|
scala.collection.Map<java.lang.Object,RDD<?>> |
getPersistentRDDs()
Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.
|
scala.Option<org.apache.spark.scheduler.Schedulable> |
getPoolForName(java.lang.String pool)
:: DeveloperApi ::
Return the pool associated with the given name, if one exists
|
RDDInfo[] |
getRDDStorageInfo()
:: DeveloperApi ::
Return information about what RDDs are cached, if they are in mem or on disk, how much space
they take, etc.
|
scala.Enumeration.Value |
getSchedulingMode()
Return current scheduling mode
|
org.apache.hadoop.conf.Configuration |
hadoopConfiguration()
A default Hadoop Configuration for the Hadoop code (e.g.
|
<K,V> RDD<scala.Tuple2<K,V>> |
hadoopFile(java.lang.String path,
java.lang.Class<? extends org.apache.hadoop.mapred.InputFormat<K,V>> inputFormatClass,
java.lang.Class<K> keyClass,
java.lang.Class<V> valueClass,
int minPartitions)
Get an RDD for a Hadoop file with an arbitrary InputFormat
|
<K,V,F extends org.apache.hadoop.mapred.InputFormat<K,V>> |
hadoopFile(java.lang.String path,
scala.reflect.ClassTag<K> km,
scala.reflect.ClassTag<V> vm,
scala.reflect.ClassTag<F> fm)
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.
|
<K,V,F extends org.apache.hadoop.mapred.InputFormat<K,V>> |
hadoopFile(java.lang.String path,
int minPartitions,
scala.reflect.ClassTag<K> km,
scala.reflect.ClassTag<V> vm,
scala.reflect.ClassTag<F> fm)
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.
|
<K,V> RDD<scala.Tuple2<K,V>> |
hadoopRDD(org.apache.hadoop.mapred.JobConf conf,
java.lang.Class<? extends org.apache.hadoop.mapred.InputFormat<K,V>> inputFormatClass,
java.lang.Class<K> keyClass,
java.lang.Class<V> valueClass,
int minPartitions)
Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other
necessary info (e.g.
|
void |
initLocalProperties() |
static org.apache.hadoop.io.IntWritable |
intToIntWritable(int i) |
static org.apache.spark.WritableConverter<java.lang.Object> |
intWritableConverter() |
boolean |
isLocal() |
static scala.Option<java.lang.String> |
jarOfClass(java.lang.Class<?> cls)
Find the JAR from which a given class was loaded, to make it easy for users to pass
their JARs to SparkContext.
|
static scala.Option<java.lang.String> |
jarOfObject(java.lang.Object obj)
Find the JAR that contains the class of a particular object, to make it easy for users
to pass their JARs to SparkContext.
|
scala.collection.Seq<java.lang.String> |
jars() |
boolean |
killExecutor(java.lang.String executorId)
:: DeveloperApi ::
Request that the cluster manager kill the specified executor.
|
boolean |
killExecutors(scala.collection.Seq<java.lang.String> executorIds)
:: DeveloperApi ::
Request that the cluster manager kill the specified executors.
|
static java.lang.String |
LEGACY_DRIVER_IDENTIFIER()
Legacy version of DRIVER_IDENTIFIER, retained for backwards-compatibility.
|
org.apache.spark.scheduler.LiveListenerBus |
listenerBus() |
protected java.lang.InheritableThreadLocal<java.util.Properties> |
localProperties() |
static org.apache.hadoop.io.LongWritable |
longToLongWritable(long l) |
static org.apache.spark.WritableConverter<java.lang.Object> |
longWritableConverter() |
<T> RDD<T> |
makeRDD(scala.collection.Seq<T> seq,
int numSlices,
scala.reflect.ClassTag<T> evidence$2)
Distribute a local Scala collection to form an RDD.
|
<T> RDD<T> |
makeRDD(scala.collection.Seq<scala.Tuple2<T,scala.collection.Seq<java.lang.String>>> seq,
scala.reflect.ClassTag<T> evidence$3)
Distribute a local Scala collection to form an RDD, with one or more
location preferences (hostnames of Spark nodes) for each object.
|
java.lang.String |
master() |
org.apache.spark.metrics.MetricsSystem |
metricsSystem() |
<K,V,F extends org.apache.hadoop.mapreduce.InputFormat<K,V>> |
newAPIHadoopFile(java.lang.String path,
java.lang.Class<F> fClass,
java.lang.Class<K> kClass,
java.lang.Class<V> vClass,
org.apache.hadoop.conf.Configuration conf)
Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
and extra configuration options to pass to the input format.
|
<K,V,F extends org.apache.hadoop.mapreduce.InputFormat<K,V>> |
newAPIHadoopFile(java.lang.String path,
scala.reflect.ClassTag<K> km,
scala.reflect.ClassTag<V> vm,
scala.reflect.ClassTag<F> fm)
Get an RDD for a Hadoop file with an arbitrary new API InputFormat.
|
<K,V,F extends org.apache.hadoop.mapreduce.InputFormat<K,V>> |
newAPIHadoopRDD(org.apache.hadoop.conf.Configuration conf,
java.lang.Class<F> fClass,
java.lang.Class<K> kClass,
java.lang.Class<V> vClass)
Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
and extra configuration options to pass to the input format.
|
static <T> DoubleRDDFunctions |
numericRDDToDoubleRDDFunctions(RDD<T> rdd,
scala.math.Numeric<T> num) |
<T> RDD<T> |
objectFile(java.lang.String path,
int minPartitions,
scala.reflect.ClassTag<T> evidence$4)
Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and
BytesWritable values that contain a serialized partition.
|
<T> RDD<T> |
parallelize(scala.collection.Seq<T> seq,
int numSlices,
scala.reflect.ClassTag<T> evidence$1)
Distribute a local Scala collection to form an RDD.
|
|
persistentRdds() |
scala.collection.Map<java.lang.String,scala.collection.Set<SplitInfo>> |
preferredNodeLocationData() |
RDD<java.lang.Object> |
range(long start,
long end,
long step,
int numSlices)
Creates a new RDD[Long] containing elements from
start to end (exclusive), increased by
step every element. |
static java.lang.String |
RDD_SCOPE_KEY() |
static java.lang.String |
RDD_SCOPE_NO_OVERRIDE_KEY() |
static <T> AsyncRDDActions<T> |
rddToAsyncRDDActions(RDD<T> rdd,
scala.reflect.ClassTag<T> evidence$22) |
static <K,V> OrderedRDDFunctions<K,V,scala.Tuple2<K,V>> |
rddToOrderedRDDFunctions(RDD<scala.Tuple2<K,V>> rdd,
scala.math.Ordering<K> evidence$27,
scala.reflect.ClassTag<K> evidence$28,
scala.reflect.ClassTag<V> evidence$29) |
static <K,V> PairRDDFunctions<K,V> |
rddToPairRDDFunctions(RDD<scala.Tuple2<K,V>> rdd,
scala.reflect.ClassTag<K> kt,
scala.reflect.ClassTag<V> vt,
scala.math.Ordering<K> ord) |
static <K,V> SequenceFileRDDFunctions<K,V> |
rddToSequenceFileRDDFunctions(RDD<scala.Tuple2<K,V>> rdd,
scala.Function1<K,org.apache.hadoop.io.Writable> evidence$23,
scala.reflect.ClassTag<K> evidence$24,
scala.Function1<V,org.apache.hadoop.io.Writable> evidence$25,
scala.reflect.ClassTag<V> evidence$26) |
boolean |
requestExecutors(int numAdditionalExecutors)
:: DeveloperApi ::
Request an additional number of executors from the cluster manager.
|
<T,U,R> PartialResult<R> |
runApproximateJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,
:: DeveloperApi ::
Run a job that can return approximate results.
|
<T,U> java.lang.Object |
runJob(RDD<T> rdd,
scala.Function1<scala.collection.Iterator<T>,U> func,
scala.reflect.ClassTag<U> evidence$19)
Run a job on all partitions in an RDD and return the results in an array.
|
<T,U> void |
runJob(RDD<T> rdd,
scala.Function1<scala.collection.Iterator<T>,U> processPartition,
scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler,
scala.reflect.ClassTag<U> evidence$21)
Run a job on all partitions in an RDD and pass the results to a handler function.
|
<T,U> java.lang.Object |
runJob(RDD<T> rdd,
scala.Function1<scala.collection.Iterator<T>,U> func,
scala.collection.Seq<java.lang.Object> partitions,
boolean allowLocal,
scala.reflect.ClassTag<U> evidence$17)
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 . |
<T,U> java.lang.Object |
runJob(RDD<T> rdd,
scala.Function1<scala.collection.Iterator<T>,U> func,
scala.collection.Seq<java.lang.Object> partitions,
scala.reflect.ClassTag<U> evidence$14)
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 . |
<T,U> java.lang.Object |
runJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,
scala.reflect.ClassTag<U> evidence$18)
Run a job on all partitions in an RDD and return the results in an array.
|
<T,U> void |
runJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> processPartition,
scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler,
scala.reflect.ClassTag<U> evidence$20)
Run a job on all partitions in an RDD and pass the results to a handler function.
|
<T,U> java.lang.Object |
runJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,
scala.collection.Seq<java.lang.Object> partitions,
boolean allowLocal,
scala.reflect.ClassTag<U> evidence$16)
Run a function on a given set of partitions in an RDD and return the results as an array.
|
<T,U> void |
runJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,
scala.collection.Seq<java.lang.Object> partitions,
boolean allowLocal,
scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler,
scala.reflect.ClassTag<U> evidence$15)
Run a function on a given set of partitions in an RDD and pass the results to the given
handler function.
|
<T,U> java.lang.Object |
runJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,
scala.collection.Seq<java.lang.Object> partitions,
scala.reflect.ClassTag<U> evidence$13)
Run a function on a given set of partitions in an RDD and return the results as an array.
|
<T,U> void |
runJob(RDD<T> rdd,
scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,
scala.collection.Seq<java.lang.Object> partitions,
scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler,
scala.reflect.ClassTag<U> evidence$12)
Run a function on a given set of partitions in an RDD and pass the results to the given
handler function.
|
<K,V> RDD<scala.Tuple2<K,V>> |
sequenceFile(java.lang.String path,
java.lang.Class<K> keyClass,
java.lang.Class<V> valueClass)
Get an RDD for a Hadoop SequenceFile with given key and value types.
|
<K,V> RDD<scala.Tuple2<K,V>> |
sequenceFile(java.lang.String path,
java.lang.Class<K> keyClass,
java.lang.Class<V> valueClass,
int minPartitions)
Get an RDD for a Hadoop SequenceFile with given key and value types.
|
<K,V> RDD<scala.Tuple2<K,V>> |
sequenceFile(java.lang.String path,
int minPartitions,
scala.reflect.ClassTag<K> km,
scala.reflect.ClassTag<V> vm,
scala.Function0<org.apache.spark.WritableConverter<K>> kcf,
scala.Function0<org.apache.spark.WritableConverter<V>> vcf)
Version of sequenceFile() for types implicitly convertible to Writables through a
WritableConverter.
|
void |
setCallSite(java.lang.String shortCallSite)
Set the thread-local property for overriding the call sites
of actions and RDDs.
|
void |
setCheckpointDir(java.lang.String directory)
Set the directory under which RDDs are going to be checkpointed.
|
void |
setJobDescription(java.lang.String value)
Set a human readable description of the current job.
|
void |
setJobGroup(java.lang.String groupId,
java.lang.String description,
boolean interruptOnCancel)
Assigns a group ID to all the jobs started by this thread until the group ID is set to a
different value or cleared.
|
void |
setLocalProperty(java.lang.String key,
java.lang.String value)
Set a local property that affects jobs submitted from this thread, such as the
Spark fair scheduler pool.
|
void |
setLogLevel(java.lang.String logLevel)
Control our logLevel.
|
static java.lang.String |
SPARK_JOB_DESCRIPTION() |
static java.lang.String |
SPARK_JOB_GROUP_ID() |
static java.lang.String |
SPARK_JOB_INTERRUPT_ON_CANCEL() |
java.lang.String |
sparkUser() |
long |
startTime() |
SparkStatusTracker |
statusTracker() |
void |
stop() |
java.util.concurrent.atomic.AtomicBoolean |
stopped() |
static org.apache.hadoop.io.Text |
stringToText(java.lang.String s) |
static org.apache.spark.WritableConverter<java.lang.String> |
stringWritableConverter() |
<T,U,R> SimpleFutureAction<R> |
submitJob(RDD<T> rdd,
scala.Function1<scala.collection.Iterator<T>,U> processPartition,
scala.collection.Seq<java.lang.Object> partitions,
scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler,
scala.Function0<R> resultFunc)
:: Experimental ::
Submit a job for execution and return a FutureJob holding the result.
|
java.lang.String |
tachyonFolderName() |
RDD<java.lang.String> |
textFile(java.lang.String path,
int minPartitions)
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.
|
<T> RDD<T> |
union(RDD<T> first,
scala.collection.Seq<RDD<T>> rest,
scala.reflect.ClassTag<T> evidence$7)
Build the union of a list of RDDs passed as variable-length arguments.
|
<T> RDD<T> |
union(scala.collection.Seq<RDD<T>> rdds,
scala.reflect.ClassTag<T> evidence$6)
Build the union of a list of RDDs.
|
java.lang.String |
version()
The version of Spark on which this application is running.
|
RDD<scala.Tuple2<java.lang.String,java.lang.String>> |
wholeTextFiles(java.lang.String path,
int minPartitions)
Read a directory of text files from HDFS, a local file system (available on all nodes), or any
Hadoop-supported file system URI.
|
static <T extends org.apache.hadoop.io.Writable> |
writableWritableConverter() |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
initializeIfNecessary, initializeLogging, isTraceEnabled, log_, log, logDebug, logDebug, logError, logError, logInfo, logInfo, logName, logTrace, logTrace, logWarning, logWarning
public SparkContext(SparkConf config)
public SparkContext()
public SparkContext(SparkConf config, scala.collection.Map<java.lang.String,scala.collection.Set<SplitInfo>> preferredNodeLocationData)
config
- a SparkConf
object specifying other Spark parameterspreferredNodeLocationData
- used in YARN mode to select nodes to launch containers on.
Can be generated using org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations
from a list of input files or InputFormats for the application.public SparkContext(java.lang.String master, java.lang.String appName, SparkConf conf)
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 UIconf
- a SparkConf
object specifying other Spark parameterspublic SparkContext(java.lang.String master, java.lang.String appName, java.lang.String sparkHome, scala.collection.Seq<java.lang.String> jars, scala.collection.Map<java.lang.String,java.lang.String> environment, scala.collection.Map<java.lang.String,scala.collection.Set<SplitInfo>> preferredNodeLocationData)
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.preferredNodeLocationData
- used in YARN mode to select nodes to launch containers on.
Can be generated using org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations
from a list of input files or InputFormats for the application.public static SparkContext getOrCreate(SparkConf config)
Note: This function cannot be used to create multiple SparkContext instances even if multiple contexts are allowed.
config
- (undocumented)public static SparkContext getOrCreate()
This method allows not passing a SparkConf (useful if just retrieving).
Note: This function cannot be used to create multiple SparkContext instances even if multiple contexts are allowed.
public static java.lang.String SPARK_JOB_DESCRIPTION()
public static java.lang.String SPARK_JOB_GROUP_ID()
public static java.lang.String SPARK_JOB_INTERRUPT_ON_CANCEL()
public static java.lang.String RDD_SCOPE_KEY()
public static java.lang.String RDD_SCOPE_NO_OVERRIDE_KEY()
public static java.lang.String DRIVER_IDENTIFIER()
, but this was
changed to driver
because the angle brackets caused escaping issues in URLs and XML (see
SPARK-6716 for more details).public static java.lang.String LEGACY_DRIVER_IDENTIFIER()
public static <K,V> PairRDDFunctions<K,V> rddToPairRDDFunctions(RDD<scala.Tuple2<K,V>> rdd, scala.reflect.ClassTag<K> kt, scala.reflect.ClassTag<V> vt, scala.math.Ordering<K> ord)
public static <T> AsyncRDDActions<T> rddToAsyncRDDActions(RDD<T> rdd, scala.reflect.ClassTag<T> evidence$22)
public static <K,V> SequenceFileRDDFunctions<K,V> rddToSequenceFileRDDFunctions(RDD<scala.Tuple2<K,V>> rdd, scala.Function1<K,org.apache.hadoop.io.Writable> evidence$23, scala.reflect.ClassTag<K> evidence$24, scala.Function1<V,org.apache.hadoop.io.Writable> evidence$25, scala.reflect.ClassTag<V> evidence$26)
public static <K,V> OrderedRDDFunctions<K,V,scala.Tuple2<K,V>> rddToOrderedRDDFunctions(RDD<scala.Tuple2<K,V>> rdd, scala.math.Ordering<K> evidence$27, scala.reflect.ClassTag<K> evidence$28, scala.reflect.ClassTag<V> evidence$29)
public static DoubleRDDFunctions doubleRDDToDoubleRDDFunctions(RDD<java.lang.Object> rdd)
public static <T> DoubleRDDFunctions numericRDDToDoubleRDDFunctions(RDD<T> rdd, scala.math.Numeric<T> num)
public static org.apache.hadoop.io.IntWritable intToIntWritable(int i)
public static org.apache.hadoop.io.LongWritable longToLongWritable(long l)
public static org.apache.hadoop.io.FloatWritable floatToFloatWritable(float f)
public static org.apache.hadoop.io.DoubleWritable doubleToDoubleWritable(double d)
public static org.apache.hadoop.io.BooleanWritable boolToBoolWritable(boolean b)
public static org.apache.hadoop.io.BytesWritable bytesToBytesWritable(byte[] aob)
public static org.apache.hadoop.io.Text stringToText(java.lang.String s)
public static org.apache.spark.WritableConverter<java.lang.Object> intWritableConverter()
public static org.apache.spark.WritableConverter<java.lang.Object> longWritableConverter()
public static org.apache.spark.WritableConverter<java.lang.Object> doubleWritableConverter()
public static org.apache.spark.WritableConverter<java.lang.Object> floatWritableConverter()
public static org.apache.spark.WritableConverter<java.lang.Object> booleanWritableConverter()
public static org.apache.spark.WritableConverter<byte[]> bytesWritableConverter()
public static org.apache.spark.WritableConverter<java.lang.String> stringWritableConverter()
public static <T extends org.apache.hadoop.io.Writable> org.apache.spark.WritableConverter<T> writableWritableConverter()
public static scala.Option<java.lang.String> jarOfClass(java.lang.Class<?> cls)
cls
- (undocumented)public static scala.Option<java.lang.String> jarOfObject(java.lang.Object obj)
obj
- (undocumented)public scala.collection.Map<java.lang.String,scala.collection.Set<SplitInfo>> preferredNodeLocationData()
public long startTime()
public java.util.concurrent.atomic.AtomicBoolean stopped()
public SparkConf getConf()
public scala.collection.Seq<java.lang.String> jars()
public scala.collection.Seq<java.lang.String> files()
public java.lang.String master()
public java.lang.String appName()
public java.lang.String externalBlockStoreFolderName()
public java.lang.String tachyonFolderName()
public boolean isLocal()
public org.apache.spark.scheduler.LiveListenerBus listenerBus()
public scala.collection.mutable.HashMap<java.lang.String,java.lang.Object> addedFiles()
public scala.collection.mutable.HashMap<java.lang.String,java.lang.Object> addedJars()
publicpersistentRdds()
public SparkStatusTracker statusTracker()
public org.apache.hadoop.conf.Configuration hadoopConfiguration()
'''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.
public scala.collection.mutable.HashMap<java.lang.String,java.lang.String> executorEnvs()
public java.lang.String sparkUser()
public java.lang.String applicationId()
public scala.Option<java.lang.String> applicationAttemptId()
public org.apache.spark.metrics.MetricsSystem metricsSystem()
public scala.Option<java.lang.String> checkpointDir()
protected java.lang.InheritableThreadLocal<java.util.Properties> localProperties()
public void setLogLevel(java.lang.String logLevel)
logLevel
- The desired log level as a string.
Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARNpublic void initLocalProperties()
public void setLocalProperty(java.lang.String key, java.lang.String value)
key
- (undocumented)value
- (undocumented)public java.lang.String getLocalProperty(java.lang.String key)
org.apache.spark.SparkContext.setLocalProperty
.key
- (undocumented)public void setJobDescription(java.lang.String value)
public void setJobGroup(java.lang.String groupId, java.lang.String description, boolean interruptOnCancel)
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.
groupId
- (undocumented)description
- (undocumented)interruptOnCancel
- (undocumented)public void clearJobGroup()
public <T> RDD<T> parallelize(scala.collection.Seq<T> seq, int numSlices, scala.reflect.ClassTag<T> evidence$1)
seq
- (undocumented)numSlices
- (undocumented)evidence$1
- (undocumented)public RDD<java.lang.Object> range(long start, long end, long step, int numSlices)
start
to end
(exclusive), increased by
step
every element.
start
- the start value.end
- the end value.step
- the incremental stepnumSlices
- the partition number of the new RDD.public <T> RDD<T> makeRDD(scala.collection.Seq<T> seq, int numSlices, scala.reflect.ClassTag<T> evidence$2)
This method is identical to parallelize
.
seq
- (undocumented)numSlices
- (undocumented)evidence$2
- (undocumented)public <T> RDD<T> makeRDD(scala.collection.Seq<scala.Tuple2<T,scala.collection.Seq<java.lang.String>>> seq, scala.reflect.ClassTag<T> evidence$3)
seq
- (undocumented)evidence$3
- (undocumented)public RDD<java.lang.String> textFile(java.lang.String path, int minPartitions)
path
- (undocumented)minPartitions
- (undocumented)public RDD<scala.Tuple2<java.lang.String,java.lang.String>> wholeTextFiles(java.lang.String path, int minPartitions)
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.public RDD<scala.Tuple2<java.lang.String,PortableDataStream>> binaryFiles(java.lang.String path, int minPartitions)
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.public RDD<byte[]> binaryRecords(java.lang.String path, int recordLength, org.apache.hadoop.conf.Configuration conf)
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.
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 recordsconf
- Configuration for setting up the dataset.
public <K,V> RDD<scala.Tuple2<K,V>> hadoopRDD(org.apache.hadoop.mapred.JobConf conf, java.lang.Class<? extends org.apache.hadoop.mapred.InputFormat<K,V>> inputFormatClass, java.lang.Class<K> keyClass, java.lang.Class<V> valueClass, int minPartitions)
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
- Class of the InputFormatkeyClass
- Class of the keysvalueClass
- Class of the valuesminPartitions
- 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.
public <K,V> RDD<scala.Tuple2<K,V>> hadoopFile(java.lang.String path, java.lang.Class<? extends org.apache.hadoop.mapred.InputFormat<K,V>> inputFormatClass, java.lang.Class<K> keyClass, java.lang.Class<V> valueClass, int 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.
path
- (undocumented)inputFormatClass
- (undocumented)keyClass
- (undocumented)valueClass
- (undocumented)minPartitions
- (undocumented)public <K,V,F extends org.apache.hadoop.mapred.InputFormat<K,V>> RDD<scala.Tuple2<K,V>> hadoopFile(java.lang.String path, int minPartitions, scala.reflect.ClassTag<K> km, scala.reflect.ClassTag<V> vm, scala.reflect.ClassTag<F> fm)
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.
path
- (undocumented)minPartitions
- (undocumented)km
- (undocumented)vm
- (undocumented)fm
- (undocumented)public <K,V,F extends org.apache.hadoop.mapred.InputFormat<K,V>> RDD<scala.Tuple2<K,V>> hadoopFile(java.lang.String path, scala.reflect.ClassTag<K> km, scala.reflect.ClassTag<V> vm, scala.reflect.ClassTag<F> fm)
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.
path
- (undocumented)km
- (undocumented)vm
- (undocumented)fm
- (undocumented)public <K,V,F extends org.apache.hadoop.mapreduce.InputFormat<K,V>> RDD<scala.Tuple2<K,V>> newAPIHadoopFile(java.lang.String path, scala.reflect.ClassTag<K> km, scala.reflect.ClassTag<V> vm, scala.reflect.ClassTag<F> fm)
public <K,V,F extends org.apache.hadoop.mapreduce.InputFormat<K,V>> RDD<scala.Tuple2<K,V>> newAPIHadoopFile(java.lang.String path, java.lang.Class<F> fClass, java.lang.Class<K> kClass, java.lang.Class<V> vClass, org.apache.hadoop.conf.Configuration conf)
'''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.
path
- (undocumented)fClass
- (undocumented)kClass
- (undocumented)vClass
- (undocumented)conf
- (undocumented)public <K,V,F extends org.apache.hadoop.mapreduce.InputFormat<K,V>> RDD<scala.Tuple2<K,V>> newAPIHadoopRDD(org.apache.hadoop.conf.Configuration conf, java.lang.Class<F> fClass, java.lang.Class<K> kClass, java.lang.Class<V> vClass)
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
- Class of the InputFormatkClass
- Class of the keysvClass
- 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.
public <K,V> RDD<scala.Tuple2<K,V>> sequenceFile(java.lang.String path, java.lang.Class<K> keyClass, java.lang.Class<V> valueClass, int 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.
path
- (undocumented)keyClass
- (undocumented)valueClass
- (undocumented)minPartitions
- (undocumented)public <K,V> RDD<scala.Tuple2<K,V>> sequenceFile(java.lang.String path, java.lang.Class<K> keyClass, java.lang.Class<V> valueClass)
'''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.
path
- (undocumented)keyClass
- (undocumented)valueClass
- (undocumented)public <K,V> RDD<scala.Tuple2<K,V>> sequenceFile(java.lang.String path, int minPartitions, scala.reflect.ClassTag<K> km, scala.reflect.ClassTag<V> vm, scala.Function0<org.apache.spark.WritableConverter<K>> kcf, scala.Function0<org.apache.spark.WritableConverter<V>> vcf)
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.
path
- (undocumented)minPartitions
- (undocumented)km
- (undocumented)vm
- (undocumented)kcf
- (undocumented)vcf
- (undocumented)public <T> RDD<T> objectFile(java.lang.String path, int minPartitions, scala.reflect.ClassTag<T> evidence$4)
path
- (undocumented)minPartitions
- (undocumented)evidence$4
- (undocumented)protected <T> RDD<T> checkpointFile(java.lang.String path, scala.reflect.ClassTag<T> evidence$5)
public <T> RDD<T> union(scala.collection.Seq<RDD<T>> rdds, scala.reflect.ClassTag<T> evidence$6)
public <T> RDD<T> union(RDD<T> first, scala.collection.Seq<RDD<T>> rest, scala.reflect.ClassTag<T> evidence$7)
public <T>emptyRDD(scala.reflect.ClassTag<T> evidence$8)
public <T> Accumulator<T> accumulator(T initialValue, AccumulatorParam<T> param)
Accumulator
variable of a given type, which tasks can "add"
values to using the +=
method. Only the driver can access the accumulator's value
.initialValue
- (undocumented)param
- (undocumented)public <T> Accumulator<T> accumulator(T initialValue, java.lang.String name, AccumulatorParam<T> param)
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
.initialValue
- (undocumented)name
- (undocumented)param
- (undocumented)public <R,T> Accumulable<R,T> accumulable(R initialValue, AccumulableParam<R,T> param)
Accumulable
shared variable, to which tasks can add values
with +=
. Only the driver can access the accumuable's value
.initialValue
- (undocumented)param
- (undocumented)public <R,T> Accumulable<R,T> accumulable(R initialValue, java.lang.String name, AccumulableParam<R,T> param)
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
.initialValue
- (undocumented)name
- (undocumented)param
- (undocumented)public <R,T> Accumulable<R,T> accumulableCollection(R initialValue, scala.Function1<R,scala.collection.generic.Growable<T>> evidence$9, scala.reflect.ClassTag<R> evidence$10)
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.
initialValue
- (undocumented)evidence$9
- (undocumented)evidence$10
- (undocumented)public <T> Broadcast<T> broadcast(T value, scala.reflect.ClassTag<T> evidence$11)
Broadcast
object for reading it in distributed functions.
The variable will be sent to each cluster only once.value
- (undocumented)evidence$11
- (undocumented)public void addFile(java.lang.String path)
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.path
- (undocumented)public void addFile(java.lang.String path, boolean recursive)
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.
path
- (undocumented)recursive
- (undocumented)public void addSparkListener(SparkListener listener)
listener
- (undocumented)public boolean requestExecutors(int numAdditionalExecutors)
numAdditionalExecutors
- (undocumented)public boolean killExecutors(scala.collection.Seq<java.lang.String> executorIds)
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}}.
executorIds
- (undocumented)public boolean killExecutor(java.lang.String executorId)
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}}.
executorId
- (undocumented)public java.lang.String version()
public scala.collection.Map<java.lang.String,scala.Tuple2<java.lang.Object,java.lang.Object>> getExecutorMemoryStatus()
public RDDInfo[] getRDDStorageInfo()
public scala.collection.Map<java.lang.Object,RDD<?>> getPersistentRDDs()
public StorageStatus[] getExecutorStorageStatus()
public scala.collection.Seq<org.apache.spark.scheduler.Schedulable> getAllPools()
public scala.Option<org.apache.spark.scheduler.Schedulable> getPoolForName(java.lang.String pool)
pool
- (undocumented)public scala.Enumeration.Value getSchedulingMode()
public void clearFiles()
addFile
so that they do not get downloaded to
any new nodes.public void addJar(java.lang.String path)
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.path
- (undocumented)public void clearJars()
addJar
so that they do not get downloaded to
any new nodes.public void stop()
public void setCallSite(java.lang.String shortCallSite)
shortCallSite
- (undocumented)public void clearCallSite()
public <T,U> void runJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func, scala.collection.Seq<java.lang.Object> partitions, scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler, scala.reflect.ClassTag<U> evidence$12)
rdd
- (undocumented)func
- (undocumented)partitions
- (undocumented)resultHandler
- (undocumented)evidence$12
- (undocumented)public <T,U> java.lang.Object runJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func, scala.collection.Seq<java.lang.Object> partitions, scala.reflect.ClassTag<U> evidence$13)
rdd
- (undocumented)func
- (undocumented)partitions
- (undocumented)evidence$13
- (undocumented)public <T,U> java.lang.Object runJob(RDD<T> rdd, scala.Function1<scala.collection.Iterator<T>,U> func, scala.collection.Seq<java.lang.Object> partitions, scala.reflect.ClassTag<U> evidence$14)
Iterator[T] => U
instead of (TaskContext, Iterator[T]) => U
.rdd
- (undocumented)func
- (undocumented)partitions
- (undocumented)evidence$14
- (undocumented)public <T,U> void runJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func, scala.collection.Seq<java.lang.Object> partitions, boolean allowLocal, scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler, scala.reflect.ClassTag<U> evidence$15)
The allowLocal flag is deprecated as of Spark 1.5.0+.
rdd
- (undocumented)func
- (undocumented)partitions
- (undocumented)allowLocal
- (undocumented)resultHandler
- (undocumented)evidence$15
- (undocumented)public <T,U> java.lang.Object runJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func, scala.collection.Seq<java.lang.Object> partitions, boolean allowLocal, scala.reflect.ClassTag<U> evidence$16)
The allowLocal flag is deprecated as of Spark 1.5.0+.
rdd
- (undocumented)func
- (undocumented)partitions
- (undocumented)allowLocal
- (undocumented)evidence$16
- (undocumented)public <T,U> java.lang.Object runJob(RDD<T> rdd, scala.Function1<scala.collection.Iterator<T>,U> func, scala.collection.Seq<java.lang.Object> partitions, boolean allowLocal, scala.reflect.ClassTag<U> evidence$17)
Iterator[T] => U
instead of (TaskContext, Iterator[T]) => U
.
The allowLocal argument is deprecated as of Spark 1.5.0+.
rdd
- (undocumented)func
- (undocumented)partitions
- (undocumented)allowLocal
- (undocumented)evidence$17
- (undocumented)public <T,U> java.lang.Object runJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func, scala.reflect.ClassTag<U> evidence$18)
rdd
- (undocumented)func
- (undocumented)evidence$18
- (undocumented)public <T,U> java.lang.Object runJob(RDD<T> rdd, scala.Function1<scala.collection.Iterator<T>,U> func, scala.reflect.ClassTag<U> evidence$19)
rdd
- (undocumented)func
- (undocumented)evidence$19
- (undocumented)public <T,U> void runJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> processPartition, scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler, scala.reflect.ClassTag<U> evidence$20)
rdd
- (undocumented)processPartition
- (undocumented)resultHandler
- (undocumented)evidence$20
- (undocumented)public <T,U> void runJob(RDD<T> rdd, scala.Function1<scala.collection.Iterator<T>,U> processPartition, scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler, scala.reflect.ClassTag<U> evidence$21)
rdd
- (undocumented)processPartition
- (undocumented)resultHandler
- (undocumented)evidence$21
- (undocumented)public <T,U,R> PartialResult<R> runApproximateJob(RDD<T> rdd, scala.Function2<TaskContext,scala.collection.Iterator<T>,U> func,evaluator, long timeout)
rdd
- (undocumented)func
- (undocumented)evaluator
- (undocumented)timeout
- (undocumented)public <T,U,R> SimpleFutureAction<R> submitJob(RDD<T> rdd, scala.Function1<scala.collection.Iterator<T>,U> processPartition, scala.collection.Seq<java.lang.Object> partitions, scala.Function2<java.lang.Object,U,scala.runtime.BoxedUnit> resultHandler, scala.Function0<R> resultFunc)
rdd
- (undocumented)processPartition
- (undocumented)partitions
- (undocumented)resultHandler
- (undocumented)resultFunc
- (undocumented)public void cancelJobGroup(java.lang.String groupId)
org.apache.spark.SparkContext.setJobGroup
for more information.groupId
- (undocumented)public void cancelAllJobs()
public void setCheckpointDir(java.lang.String directory)
directory
- (undocumented)public scala.Option<java.lang.String> getCheckpointDir()
public int defaultParallelism()
public int defaultMinSplits()
public int defaultMinPartitions()