addFile creates a Hadoop Path from the given path. For a no-schema path, addFile converts it to a canonical form.
addFile prints out the following WARN message to the logs and exits.
File with 'local' scheme is not supported to add to file server, since it is already available on every node.
-
addFile...FIXME
In the end, addFile adds the file to the addedFiles internal registry (with the current timestamp):
For new files, addFile prints out the following INFO message to the logs, fetches the file (to the root directory and without using the cache) and postEnvironmentUpdate.
Added file [path] at [key] with timestamp [timestamp]
+
Firstly, addFile validate the schema of given path. For a no-schema path, addFile converts it to a canonical form. For a local schema path, addFile prints out the following WARN message to the logs and exits.
File with 'local' scheme is not supported to add to file server, since it is already available on every node.
+
And for other schema path, addFile creates a Hadoop Path from the given path.
addFile Will validate the URL if the path is an HTTP, HTTPS or FTP URI.
addFile Will throw SparkException with below message if path is local directories but not in local mode.
addFile does not support local directories when not running local mode.
+
addFile Will throw SparkException with below message if path is directories but not turn on recursive flag.
Added file $hadoopPath is a directory and recursive is not turned on.
+
In the end, addFile adds the file to the addedFiles internal registry (with the current timestamp):
For new files, addFile prints out the following INFO message to the logs, fetches the file (to the root directory and without using the cache) and postEnvironmentUpdate.
Added file [path] at [key] with timestamp [timestamp]
For files that were already added, addFile prints out the following WARN message to the logs:
The path [path] has been added already. Overwriting of added paths is not supported in the current version.
Spark local is one of the available runtime environments in Apache Spark. It is the only available runtime with no need for a proper cluster manager (and hence many call it a pseudo-cluster, however such concept do exist in Spark and is a bit different).
Spark local is used for the following master URLs (as specified using <<../SparkConf.md#, SparkConf.setMaster>> method or <<../configuration-properties.md#spark.master, spark.master>> configuration property):
local (with exactly 1 CPU core)
local[n] (with exactly n CPU cores)
++local[]++* (with the total number of CPU cores that is the number of available CPU cores on the local machine)
local[n, m] (with exactly n CPU cores and m retries when a task fails)
++local[, m]++* (with the total number of CPU cores that is the number of available CPU cores on the local machine)
Internally, Spark local uses <> as the <<../SchedulerBackend.md#, SchedulerBackend>> and executor:ExecutorBackend.md[].
.Architecture of Spark local image::../diagrams/spark-local-architecture.png[align="center"]
In this non-distributed multi-threaded runtime environment, Spark spawns all the main execution components - the spark-driver.md[driver] and an executor:Executor.md[] - in the same single JVM.
The default parallelism is the number of threads as specified in the <>. This is the only mode where a driver is used for execution (as it acts both as the driver and the only executor).
The local mode is very convenient for testing, debugging or demonstration purposes as it requires no earlier setup to launch Spark applications.
Spark local is one of the available runtime environments in Apache Spark. It is the only available runtime with no need for a proper cluster manager (and hence many call it a pseudo-cluster, however such concept do exist in Spark and is a bit different).
Spark local is used for the following master URLs (as specified using <<../SparkConf.md#, SparkConf.setMaster>> method or <<../configuration-properties.md#spark.master, spark.master>> configuration property):
local (with exactly 1 CPU core)
local[n] (with exactly n CPU cores)
local[*] (with the total number of CPU cores that is the number of available CPU cores on the local machine)
local[n, m] (with exactly n CPU cores and m retries when a task fails)
local[*, m] (with the total number of CPU cores that is the number of available CPU cores on the local machine)
Internally, Spark local uses <> as the <<../SchedulerBackend.md#, SchedulerBackend>> and executor:ExecutorBackend.md[].
In this non-distributed multi-threaded runtime environment, Spark spawns all the main execution components - the spark-driver.md[driver] and an executor:Executor.md[] - in the same single JVM.
The default parallelism is the number of threads as specified in the <>. This is the only mode where a driver is used for execution (as it acts both as the driver and the only executor).
The local mode is very convenient for testing, debugging or demonstration purposes as it requires no earlier setup to launch Spark applications.
Welcome to The Internals of Spark Core online book! \ud83e\udd19
I'm Jacek Laskowski, a Freelance Data Engineer specializing in Apache Spark (incl. Spark SQL and Spark Structured Streaming), Delta Lake, Databricks, and Apache Kafka (incl. Kafka Streams) with brief forays into a wider data engineering space (e.g., Trino, Dask and dbt, mostly during Warsaw Data Engineering meetups).
I'm very excited to have you here and hope you will enjoy exploring the internals of Spark Core as much as I have.
Flannery O'Connor
I write to discover what I know.
\"The Internals Of\" series
I'm also writing other online books in the \"The Internals Of\" series. Please visit \"The Internals Of\" Online Books home page.
Expect text and code snippets from a variety of public sources. Attribution follows.
Now, let's take a deep dive into Spark Core \ud83d\udd25
long spill(\n long size,\n MemoryConsumer trigger)\n
spill is part of the MemoryConsumer abstraction.
Only when the given MemoryConsumer is not this BytesToBytesMap and the destructive MapIterator has been used, spill requests the destructive MapIterator to spill (the given size bytes).
spill returns 0 when the trigger is this BytesToBytesMap or there is no destructiveIterator in use. Otherwise, spill returns how much bytes the destructiveIterator managed to release.
ConsoleProgressBar shows the progress of active stages to standard error, i.e. stderr. It uses SparkStatusTracker to poll the status of stages periodically and print out active stages with more than one task. It keeps overwriting itself to hold in one line for at most 3 first concurrent stages at a time.
The progress includes the stage id, the number of completed, active, and total tasks.
TIP: ConsoleProgressBar may be useful when you ssh to workers and want to see the progress of active stages.
<ConsoleProgressBar is created>> when SparkContext is created with spark.ui.showConsoleProgress enabled and the logging level of SparkContext.md[org.apache.spark.SparkContext] logger as WARN or higher (i.e. less messages are printed out and so there is a \"space\" for ConsoleProgressBar)."},{"location":"ConsoleProgressBar/#source-scala","title":"[source, scala]","text":"
To print the progress nicely ConsoleProgressBar uses COLUMNS environment variable to know the width of the terminal. It assumes 80 columns.
The progress bar prints out the status after a stage has ran at least 500 milliseconds every spark-webui-properties.md#spark.ui.consoleProgress.update.interval[spark.ui.consoleProgress.update.interval] milliseconds.
NOTE: The initial delay of 500 milliseconds before ConsoleProgressBar show the progress is not configurable.
See the progress bar in Spark shell with the following:
<1> Make sure spark.ui.showConsoleProgress is true. It is by default. <2> Disable (OFF) the root logger (that includes Spark's logger) <3> Make sure org.apache.spark.SparkContext logger is at least WARN. <4> Run a job with 4 tasks with 500ms initial sleep and 200ms sleep chunks to see the progress bar.
TIP: https://youtu.be/uEmcGo8rwek[Watch the short video] that show ConsoleProgressBar in action.
You may want to use the following example to see the progress bar in full glory - all 3 concurrent stages in console (borrowed from https://github.com/apache/spark/pull/3029#issuecomment-63244719[a comment to [SPARK-4017] show progress bar in console #3029]):
> ./bin/spark-shell\nscala> val a = sc.makeRDD(1 to 1000, 10000).map(x => (x, x)).reduceByKey(_ + _)\nscala> val b = sc.makeRDD(1 to 1000, 10000).map(x => (x, x)).reduceByKey(_ + _)\nscala> a.union(b).count()\n
ConsoleProgressBar requires a SparkContext.md[SparkContext].
When being created, ConsoleProgressBar reads spark-webui-properties.md#spark.ui.consoleProgress.update.interval[spark.ui.consoleProgress.update.interval] configuration property to set up the update interval and COLUMNS environment variable for the terminal width (or assumes 80 columns).
ConsoleProgressBar starts the internal timer refresh progress that does <> and shows progress.
NOTE: ConsoleProgressBar is created when SparkContext is created, spark.ui.showConsoleProgress configuration property is enabled, and the logging level of SparkContext.md[org.apache.spark.SparkContext] logger is WARN or higher (i.e. less messages are printed out and so there is a \"space\" for ConsoleProgressBar).
NOTE: Once created, ConsoleProgressBar is available internally as _progressBar.
FileCommitProtocol is an abstraction of file committers that can setup, commit or abort a Spark job or task (while writing out a pair RDD and partitions).
FileCommitProtocol is used for RDD.saveAsNewAPIHadoopDataset and RDD.saveAsHadoopDataset transformations (that use SparkHadoopWriter utility to write a key-value RDD out).
FileCommitProtocol is created using FileCommitProtocol.instantiate utility.
instantiate tries to find a constructor method that takes three arguments (two of type String and one Boolean) for the given jobId, outputPath and dynamicPartitionOverwrite flag. If found, instantiate prints out the following DEBUG message to the logs:
Using (String, String, Boolean) constructor\n
In case of NoSuchMethodException, instantiate prints out the following DEBUG message to the logs:
Falling back to (String, String) constructor\n
instantiate tries to find a constructor method that takes two arguments (two of type String) for the given jobId and outputPath.
With two String arguments, instantiate requires that the given dynamicPartitionOverwrite flag is disabled (false) or throws an IllegalArgumentException:
requirement failed: Dynamic Partition Overwrite is enabled but the committer [className] does not have the appropriate constructor\n
instantiate is used when:
HadoopMapRedWriteConfigUtil and HadoopMapReduceWriteConfigUtil are requested to create a HadoopMapReduceCommitProtocol committer
(Spark SQL) InsertIntoHadoopFsRelationCommand, InsertIntoHiveDirCommand, and InsertIntoHiveTable logical commands are executed
(Spark Structured Streaming) FileStreamSink is requested to write out a micro-batch data
HadoopWriteConfigUtil[K, V] is an abstraction of writer configurers for SparkHadoopWriter to write a key-value RDD (for RDD.saveAsNewAPIHadoopDataset and RDD.saveAsHadoopDataset operators).
HeartbeatReceiver is a ThreadSafeRpcEndpoint that is registered on the driver as HeartbeatReceiver.
HeartbeatReceiver receives Heartbeat messages from executors for accumulator updates (with task metrics and a Spark application's accumulators) and pass them along to TaskScheduler.
HeartbeatReceiver is registered immediately after a Spark application is started (i.e. when SparkContext is created).
HeartbeatReceiver is a SparkListener to get notified about new executors or executors that are no longer available.
ExecutorMetrics peaks (by stage and stage attempt IDs)
Posted when Executor informs that it is alive and reports task metrics.
When received, HeartbeatReceiver finds the executorId executor (in executorLastSeen internal registry).
When the executor is found, HeartbeatReceiver updates the time the heartbeat was received (in executorLastSeen internal registry).
HeartbeatReceiver uses the Clock to know the current time.
HeartbeatReceiver then submits an asynchronous task to notify TaskScheduler that the heartbeat was received from the executor (using TaskScheduler internal reference). HeartbeatReceiver posts a HeartbeatResponse back to the executor (with the response from TaskScheduler whether the executor has been registered already or not so it may eventually need to re-register).
If however the executor was not found (in executorLastSeen internal registry), i.e. the executor was not registered before, you should see the following DEBUG message in the logs and the response is to notify the executor to re-register.
Received heartbeat from unknown executor [executorId]\n
In a very rare case, when TaskScheduler is not yet assigned to HeartbeatReceiver, you should see the following WARN message in the logs and the response is to notify the executor to re-register.
Dropping [heartbeat] because TaskScheduler is not ready yet\n
== [[InterruptibleIterator]] InterruptibleIterator -- Iterator With Support For Task Cancellation
InterruptibleIterator is a custom Scala https://www.scala-lang.org/api/2.11.x/index.html#scala.collection.Iterator[Iterator] that supports task cancellation, i.e. <>.
Quoting the official Scala https://www.scala-lang.org/api/2.11.x/index.html#scala.collection.Iterator[Iterator] documentation:
Iterators are data structures that allow to iterate over a sequence of elements. They have a hasNext method for checking if there is a next element available, and a next method which returns the next element and discards it from the iterator.
InterruptibleIterator is <> when:
RDD is requested to rdd:RDD.md#getOrCompute[get or compute a RDD partition]
CoGroupedRDD, rdd:HadoopRDD.md#compute[HadoopRDD], rdd:NewHadoopRDD.md#compute[NewHadoopRDD], rdd:ParallelCollectionRDD.md#compute[ParallelCollectionRDD] are requested to compute a partition
BlockStoreShuffleReader is requested to shuffle:BlockStoreShuffleReader.md#read[read combined key-value records for a reduce task]
PairRDDFunctions is requested to rdd:PairRDDFunctions.md#combineByKeyWithClassTag[combineByKeyWithClassTag]
Spark SQL's DataSourceRDD and JDBCRDD are requested to compute a partition
Spark SQL's RangeExec physical operator is requested to doExecute
PySpark's BasePythonRunner is requested to compute
[[creating-instance]] InterruptibleIterator takes the following when created:
[[context]] TaskContext
[[delegate]] Scala Iterator[T]
NOTE: InterruptibleIterator is a Developer API which is a lower-level, unstable API intended for Spark developers that may change or be removed in minor versions of Apache Spark.
NOTE: hasNext is part of ++https://www.scala-lang.org/api/2.11.x/index.html#scala.collection.Iterator@hasNext:Boolean++[Iterator Contract] to test whether this iterator can provide another element.
hasNext requests the <> to kill the task if interrupted (that simply throws a TaskKilledException that in turn breaks the task execution).
NOTE: next is part of ++https://www.scala-lang.org/api/2.11.x/index.html#scala.collection.Iterator@next():A++[Iterator Contract] to produce the next element of this iterator.
next simply requests the <> to next."},{"location":"ListenerBus/","title":"ListenerBus","text":"
ListenerBus is an abstraction of event buses that can notify listeners about scheduling events.
"},{"location":"ListenerBus/#contract","title":"Contract","text":""},{"location":"ListenerBus/#notifying-listener-about-event","title":"Notifying Listener about Event
From the scaladoc (it's a private[spark] class so no way to find it outside the code):
Authority that decides whether tasks can commit output to HDFS. Uses a \"first committer wins\" policy.
OutputCommitCoordinator is instantiated in both the drivers and executors. On executors, it is configured with a reference to the driver's OutputCommitCoordinatorEndpoint, so requests to commit output will be forwarded to the driver's OutputCommitCoordinator.
This class was introduced in SPARK-4879; see that JIRA issue (and the associated pull requests) for an extensive design discussion.
OutputCommitCoordinator is registered as OutputCommitCoordinator (with OutputCommitCoordinatorEndpoint RPC Endpoint) in the RPC Environment on the driver (when SparkEnv utility is used to create \"base\" SparkEnv). Executors have an RpcEndpointRef to the endpoint on the driver.
coordinatorRef is used to post an AskPermissionToCommitOutput (by executors) to the OutputCommitCoordinator (when canCommit).
coordinatorRef is used to stop the OutputCommitCoordinator on the driver (when stop).
SparkContext uses spark.driver.resourcesFile configuration property to discovery driver resources and prints out the following INFO message to the logs:
==============================================================\nResources for [componentName]:\n[resources]\n==============================================================\n
SparkContext prints out the following INFO message to the logs (with the value of spark.app.name configuration property):
Submitted application: [appName]\n
","text":""},{"location":"SparkContext-creating-instance-internals/#spark-on-yarn-and-sparkyarnappid","title":"Spark on YARN and spark.yarn.app.id
For Spark on YARN in cluster deploy mode], SparkContext checks whether spark.yarn.app.id configuration property is defined. SparkException is thrown if it does not exist.
Detected yarn cluster mode, but isn't running on a cluster. Deployment to YARN is not supported directly by SparkContext. Please use spark-submit.\n
With spark.logConf configuration property enabled, SparkContext prints out the following INFO message to the logs:
Spark configuration:\n[conf.toDebugString]\n
Note
SparkConf.toDebugString is used very early in the initialization process and other settings configured afterwards are not included. Use SparkContext.getConf.toDebugString once SparkContext is initialized.
If spark-history-server:EventLoggingListener.md[event logging] is enabled, i.e. EventLoggingListener.md#spark_eventLog_enabled[spark.eventLog.enabled] flag is true, the internal field _eventLogDir is set to the value of EventLoggingListener.md#spark_eventLog_dir[spark.eventLog.dir] setting or the default value /tmp/spark-events.
Also, if spark-history-server:EventLoggingListener.md#spark_eventLog_compress[spark.eventLog.compress] is enabled (it is not by default), the short name of the CompressionCodec is assigned to _eventLogCodec. The config key is spark.io.compression.codec (default: lz4).
SparkContext creates an in-memory store (with an optional AppStatusSource if enabled) and requests the LiveListenerBus to register the AppStatusListener with the status queue.
The AppStatusStore is available using the statusStore property of the SparkContext.
SparkContext determines the amount of memory to allocate to each executor. It is the value of executor:Executor.md#spark.executor.memory[spark.executor.memory] setting, or SparkContext.md#environment-variables[SPARK_EXECUTOR_MEMORY] environment variable (or currently-deprecated SPARK_MEM), or defaults to 1024.
_executorMemory is later available as sc.executorMemory and used for LOCAL_CLUSTER_REGEX, SparkDeploySchedulerBackend, to set executorEnvs(\"SPARK_EXECUTOR_MEMORY\"), MesosSchedulerBackend, CoarseMesosSchedulerBackend.
The value of SPARK_PREPEND_CLASSES environment variable is included in executorEnvs.
","text":""},{"location":"SparkContext-creating-instance-internals/#for-mesos-schedulerbackend-only","title":"For Mesos SchedulerBackend Only
The Mesos scheduler backend's configuration is included in executorEnvs, i.e. SparkContext.md#environment-variables[SPARK_EXECUTOR_MEMORY], _conf.getExecutorEnv, and SPARK_USER.
SparkContext creates a PluginContainer (with itself and the _resources).
","text":""},{"location":"SparkContext-creating-instance-internals/#creating-schedulerbackend-and-taskscheduler","title":"Creating SchedulerBackend and TaskScheduler
SparkContext object is requested to SparkContext.md#createTaskScheduler[create the SchedulerBackend with the TaskScheduler] (for the given master URL) and the result becomes the internal _schedulerBackend and _taskScheduler.
scheduler:DAGScheduler.md#creating-instance[DAGScheduler is created] (as _dagScheduler).
","text":""},{"location":"SparkContext-creating-instance-internals/#setting-spark-applications-and-execution-attempts-ids","title":"Setting Spark Application's and Execution Attempt's IDs
SparkContext sets the internal fields -- _applicationId and _applicationAttemptId -- (using applicationId and applicationAttemptId methods from the scheduler:TaskScheduler.md#contract[TaskScheduler Contract]).
NOTE: SparkContext requests TaskScheduler for the scheduler:TaskScheduler.md#applicationId[unique identifier of a Spark application] (that is currently only implemented by scheduler:TaskSchedulerImpl.md#applicationId[TaskSchedulerImpl] that uses SchedulerBackend to scheduler:SchedulerBackend.md#applicationId[request the identifier]).
NOTE: The unique identifier of a Spark application is used to initialize spark-webui-SparkUI.md#setAppId[SparkUI] and storage:BlockManager.md#initialize[BlockManager].
NOTE: _applicationAttemptId is used when SparkContext is requested for the SparkContext.md#applicationAttemptId[unique identifier of execution attempt of a Spark application] and when EventLoggingListener spark-history-server:EventLoggingListener.md#creating-instance[is created].
","text":""},{"location":"SparkContext-creating-instance-internals/#setting-sparkappid-spark-property-in-sparkconf","title":"Setting spark.app.id Spark Property in SparkConf
SparkContext sets SparkConf.md#spark.app.id[spark.app.id] property to be the <<_applicationId, unique identifier of a Spark application>> and, if enabled, spark-webui-SparkUI.md#setAppId[passes it on to SparkUI].
SparkContext requests the MetricsSystem to start (with the value of thespark.metrics.staticSources.enabled configuration property).
Note
SparkContext starts the MetricsSystem after <> as MetricsSystem uses it to build unique identifiers fo metrics sources.","text":""},{"location":"SparkContext-creating-instance-internals/#attaching-servlet-handlers-to-web-ui","title":"Attaching Servlet Handlers to web UI
SparkContext requests the MetricsSystem for servlet handlers and requests the SparkUI to attach them.
With spark.cleaner.referenceTracking configuration property enabled, SparkContext creates a ContextCleaner (with itself and the _shuffleDriverComponents).
postEnvironmentUpdate is called that posts SparkListener.md#SparkListenerEnvironmentUpdate[SparkListenerEnvironmentUpdate] message on scheduler:LiveListenerBus.md[] with information about Task Scheduler's scheduling mode, added jar and file paths, and other environmental details.
SparkListener.md#SparkListenerApplicationStart[SparkListenerApplicationStart] message is posted to scheduler:LiveListenerBus.md[] (using the internal postApplicationStart method).
TaskScheduler scheduler:TaskScheduler.md#postStartHook[is notified that SparkContext is almost fully initialized].
NOTE: scheduler:TaskScheduler.md#postStartHook[TaskScheduler.postStartHook] does nothing by default, but custom implementations offer more advanced features, i.e. TaskSchedulerImpl scheduler:TaskSchedulerImpl.md#postStartHook[blocks the current thread until SchedulerBackend is ready]. There is also YarnClusterScheduler for Spark on YARN in cluster deploy mode.
NOTE: getClusterManager is used to find a cluster manager for a master URL when SparkContext.md#createTaskScheduler[creating a SchedulerBackend and a TaskScheduler for the driver].
setupAndStartListenerBus is an internal method that reads configuration-properties.md#spark.extraListeners[spark.extraListeners] configuration property from the current SparkConf.md[SparkConf] to create and register SparkListenerInterface listeners.
It expects that the class name represents a SparkListenerInterface listener with one of the following constructors (in this order):
a single-argument constructor that accepts SparkConf.md[SparkConf]
a zero-argument constructor
setupAndStartListenerBus scheduler:LiveListenerBus.md#ListenerBus-addListener[registers every listener class].
You should see the following INFO message in the logs:
INFO Registered listener [className]\n
It scheduler:LiveListenerBus.md#start[starts LiveListenerBus] and records it in the internal _listenerBusStarted.
When no single-SparkConf or zero-argument constructor could be found for a class name in configuration-properties.md#spark.extraListeners[spark.extraListeners] configuration property, a SparkException is thrown with the message:
[className] did not have a zero-argument constructor or a single-argument constructor that accepts SparkConf. Note: if the class is defined inside of another Scala class, then its constructors may accept an implicit parameter that references the enclosing class; in this case, you must define the listener as a top-level class in order to prevent this extra parameter from breaking Spark's ability to find a valid constructor.\n
Any exception while registering a SparkListenerInterface listener stops the SparkContext and a SparkException is thrown and the source exception's message.
Exception when registering SparkListener\n
Tip
Set INFO logging level for org.apache.spark.SparkContext logger to see the extra listeners being registered.
SparkContext is the entry point to all of the components of Apache Spark (execution engine) and so the heart of a Spark application. In fact, you can consider an application a Spark application only when it uses a SparkContext (directly or indirectly).
Important
There should be one active SparkContext per JVM and Spark developers should use SparkContext.getOrCreate utility for sharing it (e.g. across threads).
SparkContext uses an InheritableThreadLocal (Java) of key-value pairs of thread-local properties to pass extra information from a parent thread (on the driver) to child threads.
localProperties is meant to be used by developers using SparkContext.setLocalProperty and SparkContext.getLocalProperty.
Local Properties are available using TaskContext.getLocalProperty.
Local Properties are available to SparkListeners using the following events:
SparkListenerJobStart
SparkListenerStageSubmitted
localProperties are passed down when SparkContext is requested for the following:
Running Job (that in turn makes the local properties available to the DAGScheduler to run a job)
Running Approximate Job
Submitting Job
Submitting MapStage
DAGScheduler passes down local properties when scheduling:
ShuffleMapTasks
ResultTasks
TaskSets
Spark (Core) defines the following local properties.
Name Default Value Setter callSite.longcallSite.shortSparkContext.setCallSitespark.job.description callSite.short SparkContext.setJobDescription (SparkContext.setJobGroup) spark.job.interruptOnCancelSparkContext.setJobGroupspark.jobGroup.idSparkContext.setJobGroupspark.scheduler.pool","text":""},{"location":"SparkContext/#shuffledrivercomponents","title":"ShuffleDriverComponents
SparkContext creates a ShuffleDriverComponents when created.
SparkContext loads the ShuffleDataIO that is in turn requested for the ShuffleDriverComponents. SparkContext requests the ShuffleDriverComponents to initialize.
The ShuffleDriverComponents is used when:
ShuffleDependency is created
SparkContext creates the ContextCleaner (if enabled)
SparkContext requests the ShuffleDriverComponents to clean up when stopping.
addFile creates a Hadoop Path from the given path. For a no-schema path, addFile converts it to a canonical form.
addFile prints out the following WARN message to the logs and exits.
File with 'local' scheme is not supported to add to file server, since it is already available on every node.\n
addFile...FIXME
In the end, addFile adds the file to the addedFiles internal registry (with the current timestamp):
For new files, addFile prints out the following INFO message to the logs, fetches the file (to the root directory and without using the cache) and postEnvironmentUpdate.
Added file [path] at [key] with timestamp [timestamp]\n
For files that were already added, addFile prints out the following WARN message to the logs:
The path [path] has been added already. Overwriting of added paths is not supported in the current version.\n
runJob is essentially executing a func function on all or a subset of partitions of an RDD and returning the result as an array (with elements being the results per partition).
maxNumConcurrentTasks requests the SchedulerBackend for the maximum number of tasks that can be launched concurrently (with the given ResourceProfile).
maxNumConcurrentTasks is used when:
DAGScheduler is requested to checkBarrierStageWithNumSlots
SparkEnv is a handle to Spark Execution Environment with the core services of Apache Spark (that interact with each other to establish a distributed computing platform for a Spark application).
There are two separate SparkEnvs of the driver and executors.
createDriverEnv creates a SparkEnv execution environment for the driver.
createDriverEnv accepts an instance of SparkConf, whether it runs in local mode or not, scheduler:LiveListenerBus.md[], the number of cores to use for execution in local mode or 0 otherwise, and a OutputCommitCoordinator (default: none).
createDriverEnv ensures that spark-driver.md#spark_driver_host[spark.driver.host] and spark-driver.md#spark_driver_port[spark.driver.port] settings are defined.
It then passes the call straight on to the <> (with driver executor id, isDriver enabled, and the input parameters).
createDriverEnv is used when SparkContext is created.
","text":"","tags":["DeveloperApi"]},{"location":"SparkEnv/#creating-sparkenv-for-executor","title":"Creating SparkEnv for Executor
createExecutorEnv creates an executor's (execution) environment that is the Spark execution environment for an executor.
createExecutorEnv simply <> (passing in all the input parameters) and <>.
NOTE: The number of cores numCores is configured using --cores command-line option of CoarseGrainedExecutorBackend and is specific to a cluster manager.
createExecutorEnv is used when CoarseGrainedExecutorBackend utility is requested to run.
create creates the \"base\" SparkEnv (that is common across the driver and executors).
create creates a RpcEnv as sparkDriver on the driver and sparkExecutor on executors.
create creates a Serializer (based on spark.serializer configuration property). create prints out the following DEBUG message to the logs:
Using serializer: [serializer]\n
create creates a SerializerManager.
create creates a JavaSerializer as the closure serializer.
creates creates a BroadcastManager.
creates creates a MapOutputTrackerMaster (on the driver) or a MapOutputTrackerWorker (on executors). creates registers or looks up a MapOutputTrackerMasterEndpoint under the name of MapOutputTracker. creates prints out the following INFO message to the logs (on the driver only):
Registering MapOutputTracker\n
creates creates a ShuffleManager (based on spark.shuffle.manager configuration property).
create creates a UnifiedMemoryManager.
With spark.shuffle.service.enabled configuration property enabled, create creates an ExternalBlockStoreClient.
create creates a BlockManagerMaster.
create creates a NettyBlockTransferService.
create creates a BlockManager.
create creates a MetricsSystem.
create creates a OutputCommitCoordinator and registers or looks up a OutputCommitCoordinatorEndpoint under the name of OutputCommitCoordinator.
create creates a SparkEnv (with all the services \"stitched\" together).
write runs a Spark job to write out partition records (for all partitions of the given key-value RDD) with the given HadoopWriteConfigUtil and a HadoopMapReduceCommitProtocol committer.
The number of writer tasks (parallelism) is the number of the partitions in the given key-value RDD.
executeTask requests the given HadoopWriteConfigUtil to create a TaskAttemptContext.
executeTask requests the given FileCommitProtocol to set up a task with the TaskAttemptContext.
executeTask requests the given HadoopWriteConfigUtil to initWriter (with the TaskAttemptContext and the given sparkPartitionId).
executeTask initHadoopOutputMetrics.
executeTask writes all rows of the RDD partition (from the given Iterator[(K, V)]). executeTask requests the given HadoopWriteConfigUtil to write. In the end, executeTask requests the given HadoopWriteConfigUtil to closeWriter and the given FileCommitProtocol to commit the task.
executeTask updates metrics about writing data to external systems (bytesWritten and recordsWritten) every few records and at the end.
In case of any errors, executeTask requests the given HadoopWriteConfigUtil to closeWriter and the given FileCommitProtocol to abort the task. In the end, executeTask prints out the following ERROR message to the logs:
SparkListenerBus\u00a0is an extension of the ListenerBus abstraction for event buses for SparkListenerInterfaces to be notified about SparkListenerEvents.
"},{"location":"SparkListenerBus/#posting-event-to-sparklistener","title":"Posting Event to SparkListener
SparkListenerInterface is an abstraction of event listeners (that SparkListenerBus notifies about scheduling events).
SparkListenerInterface is a way to intercept scheduling events from the Spark Scheduler that are emitted over the course of execution of a Spark application.
SparkListenerInterface is used heavily to manage communication between internal components in the distributed environment for a Spark application (e.g. web UI, event persistence for History Server, dynamic allocation of executors, keeping track of executors).
SparkListenerInterface can be registered in a Spark application using SparkContext.addSparkListener method or spark.extraListeners configuration property.
Tip
Enable INFO logging level for org.apache.spark.SparkContext logger to see what and when custom Spark listeners are registered.
org.apache.spark.scheduler.StatsReportListener (see https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.scheduler.StatsReportListener[the listener's scaladoc]) is a SparkListener.md[] that logs summary statistics when each stage completes.
StatsReportListener listens to SparkListenerTaskEnd and SparkListenerStageCompleted events and prints them out at INFO logging level.
getDynamicAllocationInitialExecutors gives the maximum value of the following configuration properties (for the initial number of executors):
spark.dynamicAllocation.initialExecutors
spark.dynamicAllocation.minExecutors
spark.executor.instances
getDynamicAllocationInitialExecutors prints out the following INFO message to the logs:
Using initial executors = [initialExecutors],\nmax of spark.dynamicAllocation.initialExecutors, spark.dynamicAllocation.minExecutors and spark.executor.instances\n
With spark.dynamicAllocation.initialExecutors less than spark.dynamicAllocation.minExecutors, getDynamicAllocationInitialExecutors prints out the following WARN message to the logs:
spark.dynamicAllocation.initialExecutors less than spark.dynamicAllocation.minExecutors is invalid,\nignoring its setting, please update your configs.\n
With spark.executor.instances less than spark.dynamicAllocation.minExecutors, getDynamicAllocationInitialExecutors prints out the following WARN message to the logs:
spark.executor.instances less than spark.dynamicAllocation.minExecutors is invalid,\nignoring its setting, please update your configs.\n
getDynamicAllocationInitialExecutors is used when:
ExecutorAllocationManager is created
SchedulerBackendUtils utility is used to getInitialTargetExecutorNumber
","text":""},{"location":"Utils/#local-directories-for-scratch-space","title":"Local Directories for Scratch Space
getConfiguredLocalDirs returns the local directories where Spark can write files to.
getConfiguredLocalDirs uses the given SparkConf to know if External Shuffle Service is enabled or not (based on spark.shuffle.service.enabled configuration property).
When in a YARN container (CONTAINER_ID), getConfiguredLocalDirs uses LOCAL_DIRS environment variable for YARN-approved local directories.
In non-YARN mode (or for the driver in yarn-client mode), getConfiguredLocalDirs checks the following environment variables (in order) and returns the value of the first found:
SPARK_EXECUTOR_DIRS
SPARK_LOCAL_DIRS
MESOS_DIRECTORY (only when External Shuffle Service is not used)
The environment variables are a comma-separated list of local directory paths.
In the end, when no earlier environment variables were found, getConfiguredLocalDirs uses spark.local.dir configuration property (with java.io.tmpdir System property as the default value).
getConfiguredLocalDirs is used when:
DiskBlockManager is requested to createLocalDirs and createLocalDirsForMergedShuffleBlocks
Utils utility is used to get a single random local root directory and create a spark directory in every local root directory
","text":""},{"location":"Utils/#random-local-directory-path","title":"Random Local Directory Path
getLocalDir(\n conf: SparkConf): String\n
getLocalDir takes a random directory path out of the configured local root directories
getLocalDir throws an IOException if no local directory is defined:
Failed to get a temp directory under [[configuredLocalDirs]].\n
getLocalDir is used when:
SparkEnv utility is used to create a base SparkEnv for the driver
Utils utility is used to fetchFile
DriverLogger is created
RocksDBStateStoreProvider (Spark Structured Streaming) is requested for a RocksDB
PythonBroadcast (PySpark) is requested to readObject
AggregateInPandasExec (PySpark) is requested to doExecute
EvalPythonExec (PySpark) is requested to doExecute
WindowInPandasExec (PySpark) is requested to doExecute
PythonForeachWriter (PySpark) is requested for a UnsafeRowBuffer
Client (Spark on YARN) is requested to prepareLocalResources and createConfArchive
Worker (Spark Standalone) is requested to launch an executor
","text":""},{"location":"Utils/#creating-spark-directory-in-every-local-root-directory","title":"Creating spark Directory in Every Local Root Directory
getOrCreateLocalRootDirsImpl creates a spark-[randomUUID] directory under every root directory for local storage (and registers a shutdown hook to delete the directories at shutdown).
getOrCreateLocalRootDirsImpl prints out the following WARN message to the logs when there is a local root directories as a URI (with a scheme):
The configured local directories are not expected to be URIs;\nhowever, got suspicious values [[uris]].\nPlease check your configured local directories.\n
","text":""},{"location":"Utils/#local-uri-scheme","title":"Local URI Scheme
Utils defines a local URI scheme for files that are locally available on worker nodes in the cluster.
getCurrentUserName computes the user name who has started the SparkContext.md[SparkContext] instance.
NOTE: It is later available as SparkContext.md#sparkUser[SparkContext.sparkUser].
Internally, it reads SparkContext.md#SPARK_USER[SPARK_USER] environment variable and, if not set, reverts to Hadoop Security API's UserGroupInformation.getCurrentUser().getShortUserName().
NOTE: It is another place where Spark relies on Hadoop API for its operation.
It starts by checking SPARK_LOCAL_HOSTNAME environment variable for the value. If it is not defined, it uses SPARK_LOCAL_IP to find the name (using InetAddress.getByName). If it is not defined either, it calls InetAddress.getLocalHost for the name.
NOTE: Utils.localHostName is executed while SparkContext.md#creating-instance[SparkContext is created] and also to compute the default value of spark-driver.md#spark_driver_host[spark.driver.host Spark property].
isPushBasedShuffleEnabled takes the value of spark.shuffle.push.enabled configuration property (from the given SparkConf).
If false, isPushBasedShuffleEnabled does nothing and returns false as well.
Otherwise, isPushBasedShuffleEnabled returns whether it is even possible to use push-based shuffle or not based on the following:
External Shuffle Service is used (based on spark.shuffle.service.enabled that should be true)
spark.master is yarn
(only with checkSerializer enabled) spark.serializer is a Serializer that supportsRelocationOfSerializedObjects
spark.io.encryption.enabled is false
In case spark.shuffle.push.enabled configuration property is enabled but the above requirements did not hold, isPushBasedShuffleEnabled prints out the following WARN message to the logs:
Push-based shuffle can only be enabled\nwhen the application is submitted to run in YARN mode,\nwith external shuffle service enabled, IO encryption disabled,\nand relocation of serialized objects supported.\n
isPushBasedShuffleEnabled\u00a0is used when:
ShuffleDependency is requested to canShuffleMergeBeEnabled
MapOutputTrackerMaster is created
MapOutputTrackerWorker is created
DAGScheduler is created
ShuffleBlockPusher utility is used to create a BLOCK_PUSHER_POOL thread pool
BlockManager is requested to initialize and registerWithExternalShuffleServer
BlockManagerMasterEndpoint is created
DiskBlockManager is requested to createLocalDirsForMergedShuffleBlocks
Spark uses a master/worker architecture. There is a spark-driver.md[driver] that talks to a single coordinator called spark-master.md[master] that manages spark-workers.md[workers] in which executor:Executor.md[executors] run.
The driver and the executors run in their own Java processes. You can run them all on the same (horizontal cluster) or separate machines (vertical cluster) or in a mixed machine configuration.
.Spark architecture in detail image::sparkapp-sparkcontext-master-slaves.png[align=\"center\"]
Number of times an Executor tries sending heartbeats to the driver before it gives up and exits (with exit code 56).
Default: 60
For example, with max failures 60 (the default) and spark.executor.heartbeatInterval 10s, then Executor will try to send heartbeats for up to 600s (10 minutes).
The files to be added to a Spark application (that can be defined directly as a configuration property or indirectly using --files option of spark-submit script)
A comma-separated list of directory paths for \"scratch\" space (a temporary storage for map output files, RDDs that get stored on disk, etc.). It is recommended to use paths on fast local disks in your system (e.g. SSDs).
How long to wait until an executor is available for locality-aware delay scheduling (for PROCESS_LOCAL, NODE_LOCAL, and RACK_LOCAL TaskLocalities) unless locality-specific setting is set (i.e., spark.locality.wait.process, spark.locality.wait.node, and spark.locality.wait.rack, respectively)
Fraction of JVM heap space used for execution and storage.
Default: 0.6
The lower the more frequent spills and cached data eviction. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation in the case of sparse, unusually large records. Leaving this at the default value is recommended.
Maximum memory (in bytes) for off-heap memory allocation
Default: 0
This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly.
Must not be negative and be set to a positive value when spark.memory.offHeap.enabled is enabled
Maximum number of remote blocks being fetched per reduce task from a given host port
When a large number of blocks are being requested from a given address in a single fetch or simultaneously, this could crash the serving executor or a Node Manager. This is especially useful to reduce the load on the Node Manager when external shuffle is enabled. You can mitigate the issue by setting it to a lower value.
Default: (unlimited)
Used when:
BlockStoreShuffleReader is requested to read combined records for a reduce task
Maximum number of remote requests to fetch blocks at any given point
When the number of hosts in the cluster increase, it might lead to very large number of inbound connections to one or more nodes, causing the workers to fail under load. By allowing it to limit the number of fetch requests, this scenario can be mitigated
Default: (unlimited)
Used when:
BlockStoreShuffleReader is requested to read combined records for a reduce task
Maximum size of all map outputs to fetch simultaneously from each reduce task (in MiB unless otherwise specified)
Since each output requires us to create a buffer to receive it, this represents a fixed memory overhead per reduce task, so keep it small unless you have a large amount of memory
Default: 48m
Used when:
BlockStoreShuffleReader is requested to read combined records for a reduce task
Controls checksuming of shuffle data. If enabled, Spark will calculate the checksum values for each partition data within the map output file and store the values in a checksum file on the disk. When there's shuffle data corruption detected, Spark will try to diagnose the cause (e.g., network issue, disk issue, etc.) of the corruption by using the checksum file.
If enabled, part of a compressed/encrypted stream will be de-compressed/de-crypted by using extra memory to detect early corruption. Any IOException thrown will cause the task to be retried once and if it fails again with same exception, then FetchFailedException will be thrown to retry previous stage
Default: false
Used when:
BlockStoreShuffleReader is requested to read combined records for a reduce task
Size of the in-memory buffer for each shuffle file output stream, in KiB unless otherwise specified. These buffers reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
Default: 32k
Must be greater than 0 and less than or equal to 2097151 ((Integer.MAX_VALUE - 15) / 1024)
(internal) Multi-thread is used when the number of mappers * shuffle partitions is greater than or equal to this threshold. Note that the actual parallelism is calculated by number of mappers * shuffle partitions / this threshold + 1, so this threshold should be positive.
Default: 10000000
Used when:
MapOutputTrackerMaster is requested for the statistics of a ShuffleDependency
(internal) Minimum number of partitions (threshold) for MapStatus utility to prefer a HighlyCompressedMapStatus (over CompressedMapStatus) (for ShuffleWriters).
Works in conjunction with the server side flag spark.shuffle.push.server.mergedShuffleFileManagerImpl which needs to be set with the appropriate org.apache.spark.network.shuffle.MergedShuffleFileManager implementation for push-based shuffle to be enabled
Used when:
Utils utility is used to determine whether push-based shuffle is enabled or not
If enabled (with spark.shuffle.useOldFetchProtocol disabled and spark.shuffle.service.enabled enabled), shuffle blocks requested from those block managers which are running on the same host are read from the disk directly instead of being fetched as remote blocks over the network.
(internal) The maximum number of elements in memory before forcing the shuffle sorter to spill.
Default: Integer.MAX_VALUE
The default value is to never force the sorter to spill, until Spark reaches some limitations, like the max page size limitation for the pointer array in the sorter.
Used when:
ShuffleExternalSorter is created
Spillable is created
Spark SQL's SortBasedAggregator is requested for an UnsafeKVExternalSorter
Spark SQL's ObjectAggregationMap is requested to dumpToExternalSorter
Spark SQL's UnsafeExternalRowSorter is created
Spark SQL's UnsafeFixedWidthAggregationMap is requested for an UnsafeKVExternalSorter
Controls whether DiskBlockObjectWriter should force outstanding writes to disk while committing a single atomic block (i.e. all operating system buffers should synchronize with the disk to ensure that all changes to a file are in fact recorded in the storage)
Default: false
Used when BlockManager is requested for a DiskBlockObjectWriter
Whether to use the old protocol while doing the shuffle block fetching. It is only enabled while we need the compatibility in the scenario of new Spark version job fetching shuffle blocks from old version external shuffle service.
The max number of executors for which the local dirs are stored. This size is both applied for the driver and both for the executors side to avoid having an unbounded store. This cache will be used to avoid the network in case of fetching disk persisted RDD blocks or shuffle blocks (when spark.shuffle.readHostLocalDisk is set) from the same host.
A Spark driver (aka an application's driver process) is a JVM process that hosts SparkContext.md[SparkContext] for a Spark application. It is the master node in a Spark application.
It is the cockpit of jobs and tasks execution (using scheduler:DAGScheduler.md[DAGScheduler] and scheduler:TaskScheduler.md[Task Scheduler]). It hosts spark-webui.md[Web UI] for the environment.
.Driver with the services image::spark-driver.png[align=\"center\"]
It splits a Spark application into tasks and schedules them to run on executors.
A driver is where the task scheduler lives and spawns tasks across workers.
A driver coordinates workers and overall execution of tasks.
NOTE: spark-shell.md[Spark shell] is a Spark application and the driver. It creates a SparkContext that is available as sc.
Driver requires the additional services (beside the common ones like shuffle:ShuffleManager.md[], memory:MemoryManager.md[], storage:BlockTransferService.md[], BroadcastManager:
Listener Bus
rpc:index.md[]
scheduler:MapOutputTrackerMaster.md[] with the name MapOutputTracker
storage:BlockManagerMaster.md[] with the name BlockManagerMaster
MetricsSystem with the name driver
OutputCommitCoordinator
CAUTION: FIXME Diagram of RpcEnv for a driver (and later executors). Perhaps it should be in the notes about RpcEnv?
High-level control flow of work
Your Spark application runs as long as the Spark driver. ** Once the driver terminates, so does your Spark application.
Creates SparkContext, RDD's, and executes transformations and actions
Launches scheduler:Task.md[tasks]
=== [[driver-memory]] Driver's Memory
It can be set first using spark-submit/index.md#command-line-options[spark-submit's --driver-memory] command-line option or <> and falls back to spark-submit/index.md#environment-variables[SPARK_DRIVER_MEMORY] if not set earlier.
NOTE: It is printed out to the standard error output in spark-submit/index.md#verbose-mode[spark-submit's verbose mode].
It can be set first using spark-submit/index.md#driver-cores[spark-submit's --driver-cores] command-line option for cluster deploy mode.
NOTE: In client deploy mode the driver's memory corresponds to the memory of the JVM process the Spark application runs on.
NOTE: It is printed out to the standard error output in spark-submit/index.md#verbose-mode[spark-submit's verbose mode].
=== [[settings]] Settings
.Spark Properties [cols=\"1,1,2\",options=\"header\",width=\"100%\"] |=== | Spark Property | Default Value | Description | [[spark_driver_blockManager_port]] spark.driver.blockManager.port | storage:BlockManager.md#spark_blockManager_port[spark.blockManager.port] | Port to use for the storage:BlockManager.md[BlockManager] on the driver.
More precisely, spark.driver.blockManager.port is used when core:SparkEnv.md#NettyBlockTransferService[NettyBlockTransferService is created] (while SparkEnv is created for the driver).
| [[spark_driver_memory]] spark.driver.memory | 1g | The driver's memory size (in MiBs).
Refer to <>.
| [[spark_driver_cores]] spark.driver.cores | 1 | The number of CPU cores assigned to the driver in cluster deploy mode.
NOTE: When yarn/spark-yarn-client.md#creating-instance[Client is created] (for Spark on YARN in cluster mode only), it sets the number of cores for ApplicationManager using spark.driver.cores.
spark.driver.appUIAddress is used exclusively in yarn/README.md[Spark on YARN]. It is set when yarn/spark-yarn-client-yarnclientschedulerbackend.md#start[YarnClientSchedulerBackend starts] to yarn/spark-yarn-applicationmaster.md#runExecutorLauncher[run ExecutorLauncher] (and yarn/spark-yarn-applicationmaster.md#registerAM[register ApplicationMaster] for the Spark application).
spark.driver.extraClassPath system property sets the additional classpath entries (e.g. jars and directories) that should be added to the driver's classpath in cluster deploy mode.
For client deploy mode you can use a properties file or command line to set spark.driver.extraClassPath.
Do not use SparkConf.md[SparkConf] since it is too late for client deploy mode given the JVM has already been set up to start a Spark application.
"},{"location":"driver/#refer-to-spark-classmdbuildsparksubmitcommandbuildsparksubmitcommand-internal-method-for-the-very-low-level-details-of-how-it-is-handled-internally","title":"Refer to spark-class.md#buildSparkSubmitCommand[buildSparkSubmitCommand Internal Method] for the very low-level details of how it is handled internally.","text":"
spark.driver.extraClassPath uses a OS-specific path separator.
NOTE: Use spark-submit's spark-submit/index.md#driver-class-path[--driver-class-path command-line option] on command line to override spark.driver.extraClassPath from a spark-properties.md#spark-defaults-conf[Spark properties file].
SparkContext.setLocalProperty lets you set key-value pairs that will be propagated down to tasks and can be accessed there using TaskContext.getLocalProperty.
One of the purposes of local properties is to create logical groups of Spark jobs by means of properties that (regardless of the threads used to submit the jobs) makes the separate jobs launched from different threads belong to a single logical group.
A common use case for the local property concept is to set a local property in a thread, say spark-scheduler-FairSchedulableBuilder.md[spark.scheduler.pool], after which all jobs submitted within the thread will be grouped, say into a pool by FAIR job scheduler.
val data = sc.parallelize(0 to 9)\n\nsc.setLocalProperty(\"spark.scheduler.pool\", \"myPool\")\n\n// these two jobs (one per action) will run in the myPool pool\ndata.count\ndata.collect\n\nsc.setLocalProperty(\"spark.scheduler.pool\", null)\n\n// this job will run in the default pool\ndata.count\n
Apache Spark is an open-source distributed general-purpose cluster computing framework with (mostly) in-memory data processing engine that can do ETL, analytics, machine learning and graph processing on large volumes of data at rest (batch processing) or in motion (streaming processing) with rich concise high-level APIs for the programming languages: Scala, Python, Java, R, and SQL.
You could also describe Spark as a distributed, data processing engine for batch and streaming modes featuring SQL queries, graph processing, and machine learning.
In contrast to Hadoop\u2019s two-stage disk-based MapReduce computation engine, Spark's multi-stage (mostly) in-memory computing engine allows for running most computations in memory, and hence most of the time provides better performance for certain applications, e.g. iterative algorithms or interactive data mining (read Spark officially sets a new record in large-scale sorting).
Spark aims at speed, ease of use, extensibility and interactive analytics.
Spark is a distributed platform for executing complex multi-stage applications, like machine learning algorithms, and interactive ad hoc queries. Spark provides an efficient abstraction for in-memory cluster computing called Resilient Distributed Dataset.
Using Spark Application Frameworks, Spark simplifies access to machine learning and predictive analytics at scale.
Spark is mainly written in http://scala-lang.org/[Scala], but provides developer API for languages like Java, Python, and R.
If you have large amounts of data that requires low latency processing that a typical MapReduce program cannot provide, Spark is a viable alternative.
Access any data type across any data source.
Huge demand for storage and data processing.
The Apache Spark project is an umbrella for https://jaceklaskowski.gitbooks.io/mastering-spark-sql/[SQL] (with Datasets), https://jaceklaskowski.gitbooks.io/spark-structured-streaming/[streaming], http://spark.apache.org/mllib/[machine learning] (pipelines) and http://spark.apache.org/graphx/[graph] processing engines built on top of the Spark Core. You can run them all in a single application using a consistent API.
Spark runs locally as well as in clusters, on-premises or in cloud. It runs on top of Hadoop YARN, Apache Mesos, standalone or in the cloud (Amazon EC2 or IBM Bluemix).
Apache Spark's https://jaceklaskowski.gitbooks.io/spark-structured-streaming/[Structured Streaming] and https://jaceklaskowski.gitbooks.io/mastering-spark-sql/[SQL] programming models with MLlib and GraphX make it easier for developers and data scientists to build applications that exploit machine learning and graph analytics.
At a high level, any Spark application creates RDDs out of some input, run rdd:index.md[(lazy) transformations] of these RDDs to some other form (shape), and finally perform rdd:index.md[actions] to collect or store data. Not much, huh?
You can look at Spark from programmer's, data engineer's and administrator's point of view. And to be honest, all three types of people will spend quite a lot of their time with Spark to finally reach the point where they exploit all the available features. Programmers use language-specific APIs (and work at the level of RDDs using transformations and actions), data engineers use higher-level abstractions like DataFrames or Pipelines APIs or external tools (that connect to Spark), and finally it all can only be possible to run because administrators set up Spark clusters to deploy Spark applications to.
It is Spark's goal to be a general-purpose computing platform with various specialized applications frameworks on top of a single unified engine.
NOTE: When you hear \"Apache Spark\" it can be two things -- the Spark engine aka Spark Core or the Apache Spark open source project which is an \"umbrella\" term for Spark Core and the accompanying Spark Application Frameworks, i.e. Spark SQL, spark-streaming/spark-streaming.md[Spark Streaming], spark-mllib/spark-mllib.md[Spark MLlib] and spark-graphx.md[Spark GraphX] that sit on top of Spark Core and the main data abstraction in Spark called rdd:index.md[RDD - Resilient Distributed Dataset].
Let's list a few of the many reasons for Spark. We are doing it first, and then comes the overview that lends a more technical helping hand.
"},{"location":"overview/#easy-to-get-started","title":"Easy to Get Started","text":"
Spark offers spark-shell that makes for a very easy head start to writing and running Spark applications on the command line on your laptop.
You could then use Spark Standalone built-in cluster manager to deploy your Spark applications to a production-grade cluster to run on a full dataset.
"},{"location":"overview/#unified-engine-for-diverse-workloads","title":"Unified Engine for Diverse Workloads","text":"
As said by Matei Zaharia - the author of Apache Spark - in Introduction to AmpLab Spark Internals video (quoting with few changes):
One of the Spark project goals was to deliver a platform that supports a very wide array of diverse workflows - not only MapReduce batch jobs (there were available in Hadoop already at that time), but also iterative computations like graph algorithms or Machine Learning.
And also different scales of workloads from sub-second interactive jobs to jobs that run for many hours.
Spark combines batch, interactive, and streaming workloads under one rich concise API.
Spark supports near real-time streaming workloads via spark-streaming/spark-streaming.md[Spark Streaming] application framework.
ETL workloads and Analytics workloads are different, however Spark attempts to offer a unified platform for a wide variety of workloads.
Graph and Machine Learning algorithms are iterative by nature and less saves to disk or transfers over network means better performance.
There is also support for interactive workloads using Spark shell.
You should watch the video https://youtu.be/SxAxAhn-BDU[What is Apache Spark?] by Mike Olson, Chief Strategy Officer and Co-Founder at Cloudera, who provides a very exceptional overview of Apache Spark, its rise in popularity in the open source community, and how Spark is primed to replace MapReduce as the general processing engine in Hadoop.
=== Leverages the Best in distributed batch data processing
When you think about distributed batch data processing, varia/spark-hadoop.md[Hadoop] naturally comes to mind as a viable solution.
Spark draws many ideas out of Hadoop MapReduce. They work together well - Spark on YARN and HDFS - while improving on the performance and simplicity of the distributed computing engine.
For many, Spark is Hadoop++, i.e. MapReduce done in a better way.
And it should not come as a surprise, without Hadoop MapReduce (its advances and deficiencies), Spark would not have been born at all.
=== RDD - Distributed Parallel Scala Collections
As a Scala developer, you may find Spark's RDD API very similar (if not identical) to http://www.scala-lang.org/docu/files/collections-api/collections.html[Scala's Collections API].
It is also exposed in Java, Python and R (as well as SQL, i.e. SparkSQL, in a sense).
So, when you have a need for distributed Collections API in Scala, Spark with RDD API should be a serious contender.
=== [[rich-standard-library]] Rich Standard Library
Not only can you use map and reduce (as in Hadoop MapReduce jobs) in Spark, but also a vast array of other higher-level operators to ease your Spark queries and application development.
It expanded on the available computation styles beyond the only map-and-reduce available in Hadoop MapReduce.
=== Unified development and deployment environment for all
Regardless of the Spark tools you use - the Spark API for the many programming languages supported - Scala, Java, Python, R, or spark-shell.md[the Spark shell], or the many Spark Application Frameworks leveraging the concept of rdd:index.md[RDD], i.e. Spark SQL, spark-streaming/spark-streaming.md[Spark Streaming], spark-mllib/spark-mllib.md[Spark MLlib] and spark-graphx.md[Spark GraphX], you still use the same development and deployment environment to for large data sets to yield a result, be it a prediction (spark-mllib/spark-mllib.md[Spark MLlib]), a structured data queries (Spark SQL) or just a large distributed batch (Spark Core) or streaming (Spark Streaming) computation.
It's also very productive of Spark that teams can exploit the different skills the team members have acquired so far. Data analysts, data scientists, Python programmers, or Java, or Scala, or R, can all use the same Spark platform using tailor-made API. It makes for bringing skilled people with their expertise in different programming languages together to a Spark project.
Using spark-shell.md[the Spark shell] you can execute computations to process large amount of data (The Big Data). It's all interactive and very useful to explore the data before final production release.
Also, using the Spark shell you can access any spark-cluster.md[Spark cluster] as if it was your local machine. Just point the Spark shell to a 20-node of 10TB RAM memory in total (using --master) and use all the components (and their abstractions) like Spark SQL, Spark MLlib, spark-streaming/spark-streaming.md[Spark Streaming], and Spark GraphX.
Depending on your needs and skills, you may see a better fit for SQL vs programming APIs or apply machine learning algorithms (Spark MLlib) from data in graph data structures (Spark GraphX).
=== Single Environment
Regardless of which programming language you are good at, be it Scala, Java, Python, R or SQL, you can use the same single clustered runtime environment for prototyping, ad hoc queries, and deploying your applications leveraging the many ingestion data points offered by the Spark platform.
You can be as low-level as using RDD API directly or leverage higher-level APIs of Spark SQL (Datasets), Spark MLlib (ML Pipelines), Spark GraphX (Graphs) or spark-streaming/spark-streaming.md[Spark Streaming] (DStreams).
Or use them all in a single application.
The single programming model and execution engine for different kinds of workloads simplify development and deployment architectures.
=== Data Integration Toolkit with Rich Set of Supported Data Sources
Spark can read from many types of data sources -- relational, NoSQL, file systems, etc. -- using many types of data formats - Parquet, Avro, CSV, JSON.
Both, input and output data sources, allow programmers and data engineers use Spark as the platform with the large amount of data that is read from or saved to for processing, interactively (using Spark shell) or in applications.
=== Tools unavailable then, at your fingertips now
As much and often as it's recommended http://c2.com/cgi/wiki?PickTheRightToolForTheJob[to pick the right tool for the job], it's not always feasible. Time, personal preference, operating system you work on are all factors to decide what is right at a time (and using a hammer can be a reasonable choice).
Spark embraces many concepts in a single unified development and runtime environment.
Machine learning that is so tool- and feature-rich in Python, e.g. SciKit library, can now be used by Scala developers (as Pipeline API in Spark MLlib or calling pipe()).
DataFrames from R are available in Scala, Java, Python, R APIs.
Single node computations in machine learning algorithms are migrated to their distributed versions in Spark MLlib.
This single platform gives plenty of opportunities for Python, Scala, Java, and R programmers as well as data engineers (SparkR) and scientists (using proprietary enterprise data warehouses with spark-sql-thrift-server.md[Thrift JDBC/ODBC Server] in Spark SQL).
Mind the proverb https://en.wiktionary.org/wiki/if_all_you_have_is_a_hammer,_everything_looks_like_a_nail[if all you have is a hammer, everything looks like a nail], too.
=== Low-level Optimizations
Apache Spark uses a scheduler:DAGScheduler.md[directed acyclic graph (DAG) of computation stages] (aka execution DAG). It postpones any processing until really required for actions. Spark's lazy evaluation gives plenty of opportunities to induce low-level optimizations (so users have to know less to do more).
Mind the proverb https://en.wiktionary.org/wiki/less_is_more[less is more].
=== Excels at low-latency iterative workloads
Spark supports diverse workloads, but successfully targets low-latency iterative ones. They are often used in Machine Learning and graph algorithms.
Many Machine Learning algorithms require plenty of iterations before the result models get optimal, like logistic regression. The same applies to graph algorithms to traverse all the nodes and edges when needed. Such computations can increase their performance when the interim partial results are stored in memory or at very fast solid state drives.
Spark can spark-rdd-caching.md[cache intermediate data in memory for faster model building and training]. Once the data is loaded to memory (as an initial step), reusing it multiple times incurs no performance slowdowns.
Also, graph algorithms can traverse graphs one connection per iteration with the partial result in memory.
Less disk access and network can make a huge difference when you need to process lots of data, esp. when it is a BIG Data.
=== ETL done easier
Spark gives Extract, Transform and Load (ETL) a new look with the many programming languages supported - Scala, Java, Python (less likely R). You can use them all or pick the best for a problem.
Scala in Spark, especially, makes for a much less boiler-plate code (comparing to other languages and approaches like MapReduce in Java).
=== [[unified-api]] Unified Concise High-Level API
Spark offers a unified, concise, high-level APIs for batch analytics (RDD API), SQL queries (Dataset API), real-time analysis (DStream API), machine learning (ML Pipeline API) and graph processing (Graph API).
Developers no longer have to learn many different processing engines and platforms, and let the time be spent on mastering framework APIs per use case (atop a single computation engine Spark).
=== Different kinds of data processing using unified API
Spark offers three kinds of data processing using batch, interactive, and stream processing with the unified API and data structures.
=== Little to no disk use for better performance
In the no-so-long-ago times, when the most prevalent distributed computing framework was varia/spark-hadoop.md[Hadoop MapReduce], you could reuse a data between computation (even partial ones!) only after you've written it to an external storage like varia/spark-hadoop.md[Hadoop Distributed Filesystem (HDFS)]. It can cost you a lot of time to compute even very basic multi-stage computations. It simply suffers from IO (and perhaps network) overhead.
One of the many motivations to build Spark was to have a framework that is good at data reuse.
Spark cuts it out in a way to keep as much data as possible in memory and keep it there until a job is finished. It doesn't matter how many stages belong to a job. What does matter is the available memory and how effective you are in using Spark API (so rdd:index.md[no shuffle occur]).
The less network and disk IO, the better performance, and Spark tries hard to find ways to minimize both.
=== Fault Tolerance included
Faults are not considered a special case in Spark, but obvious consequence of being a parallel and distributed system. Spark handles and recovers from faults by default without particularly complex logic to deal with them.
=== Small Codebase Invites Contributors
Spark's design is fairly simple and the code that comes out of it is not huge comparing to the features it offers.
The reasonably small codebase of Spark invites project contributors - programmers who extend the platform and fix bugs in a more steady pace.
== [[i-want-more]] Further reading or watching
(video) https://youtu.be/L029ZNBG7bk[Keynote: Spark 2.0 - Matei Zaharia, Apache Spark Creator and CTO of Databricks]
Push-Based Shuffle is a new feature of Apache Spark 3.2.0 (cf. SPARK-30602) to improve shuffle efficiency.
Push-based shuffle is enabled using spark.shuffle.push.enabled configuration property and can only be used in a Spark application submitted to YARN cluster manager, with external shuffle service enabled, IO encryption disabled, and relocation of serialized objects supported.
"},{"location":"spark-debugging/","title":"Debugging Spark","text":""},{"location":"spark-debugging/#using-spark-shell-and-intellij-idea","title":"Using spark-shell and IntelliJ IDEA","text":"
Start spark-shell with SPARK_SUBMIT_OPTS environment variable that configures the JVM's JDWP.
Use sbt -jvm-debug 5005, connect to the remote JVM at the port 5005 using IntelliJ IDEA, place breakpoints on the desired lines of the source code of Spark.
$ sbt -jvm-debug 5005\nListening for transport dt_socket at address: 5005\n...\n
Run Spark context and the breakpoints get triggered.
scala> val sc = new SparkContext(conf)\n15/11/14 22:58:46 INFO SparkContext: Running Spark version 1.6.0-SNAPSHOT\n
The valid logging levels are log4j's Levels (from most specific to least):
Name Description OFF No events will be logged FATAL A fatal event that will prevent the application from continuing ERROR An error in the application, possibly recoverable WARN An event that might possible lead to an error INFO An event for informational purposes DEBUG A general debugging event TRACE A fine-grained debug message, typically capturing the flow through the application ALL All events should be logged
The names of the logging levels are case-insensitive.
","text":""},{"location":"spark-logging/#turn-logging-off","title":"Turn Logging Off
The following sample conf/log4j2.properties turns all logging of Apache Spark (and Apache Hadoop) off.
# Set to debug or trace if log4j initialization fails\nstatus = warn\n\n# Name of the configuration\nname = exploring-internals\n\n# Console appender configuration\nappender.console.type = Console\nappender.console.name = consoleLogger\nappender.console.layout.type = PatternLayout\nappender.console.layout.pattern = %d{YYYY-MM-dd HH:mm:ss} [%t] %-5p %c:%L - %m%n\nappender.console.target = SYSTEM_OUT\n\nrootLogger.level = off\nrootLogger.appenderRef.stdout.ref = consoleLogger\n\nlogger.spark.name = org.apache.spark\nlogger.spark.level = off\n\nlogger.hadoop.name = org.apache.hadoop\nlogger.hadoop.level = off\n
","text":""},{"location":"spark-properties/","title":"Spark Properties and spark-defaults.conf Properties File","text":"
Spark properties are the means of tuning the execution environment of a Spark application.
The default Spark properties file is <$SPARK_HOME/conf/spark-defaults.conf>> that could be overriden using spark-submit with the spark-submit/index.md#properties-file[--properties-file] command-line option.
spark-defaults.conf (under SPARK_CONF_DIR or $SPARK_HOME/conf) is the default properties file with the Spark properties of your Spark applications.
NOTE: spark-defaults.conf is loaded by spark-AbstractCommandBuilder.md#loadPropertiesFile[AbstractCommandBuilder's loadPropertiesFile internal method].
getDefaultPropertiesFile calculates the absolute path to spark-defaults.conf properties file that can be either in directory specified by SPARK_CONF_DIR environment variable or $SPARK_HOME/conf directory.
NOTE: getDefaultPropertiesFile is part of private[spark]org.apache.spark.util.Utils object.
"},{"location":"spark-tips-and-tricks-access-private-members-spark-shell/","title":"Access private members in Scala in Spark shell","text":"
== Access private members in Scala in Spark shell
If you ever wanted to use private[spark] members in Spark using the Scala programming language, e.g. toy with org.apache.spark.scheduler.DAGScheduler or similar, you will have to use the following trick in Spark shell - use :paste -raw as described in https://issues.scala-lang.org/browse/SI-5299[REPL: support for package definition].
Open spark-shell and execute :paste -raw that allows you to enter any valid Scala code, including package.
The following snippet shows how to access private[spark] member DAGScheduler.RESUBMIT_TIMEOUT:
"},{"location":"spark-tips-and-tricks-running-spark-windows/","title":"Running Spark Applications on Windows","text":"
== Running Spark Applications on Windows
Running Spark applications on Windows in general is no different than running it on other operating systems like Linux or macOS.
NOTE: A Spark application could be spark-shell.md[spark-shell] or your own custom Spark application.
What makes the huge difference between the operating systems is Hadoop that is used internally for file system access in Spark.
You may run into few minor issues when you are on Windows due to the way Hadoop works with Windows' POSIX-incompatible NTFS filesystem.
NOTE: You do not have to install Apache Hadoop to work with Spark or run Spark applications.
TIP: Read the Apache Hadoop project's https://wiki.apache.org/hadoop/WindowsProblems[Problems running Hadoop on Windows].
Among the issues is the infamous java.io.IOException when running Spark Shell (below a stacktrace from Spark 2.0.2 on Windows 10 so the line numbers may be different in your case).
16/12/26 21:34:11 ERROR Shell: Failed to locate the winutils binary in the hadoop binary path\njava.io.IOException: Could not locate executable null\\bin\\winutils.exe in the Hadoop binaries.\n at org.apache.hadoop.util.Shell.getQualifiedBinPath(Shell.java:379)\n at org.apache.hadoop.util.Shell.getWinUtilsPath(Shell.java:394)\n at org.apache.hadoop.util.Shell.<clinit>(Shell.java:387)\n at org.apache.hadoop.hive.conf.HiveConf$ConfVars.findHadoopBinary(HiveConf.java:2327)\n at org.apache.hadoop.hive.conf.HiveConf$ConfVars.<clinit>(HiveConf.java:365)\n at org.apache.hadoop.hive.conf.HiveConf.<clinit>(HiveConf.java:105)\n at java.lang.Class.forName0(Native Method)\n at java.lang.Class.forName(Class.java:348)\n at org.apache.spark.util.Utils$.classForName(Utils.scala:228)\n at org.apache.spark.sql.SparkSession$.hiveClassesArePresent(SparkSession.scala:963)\n at org.apache.spark.repl.Main$.createSparkSession(Main.scala:91)\n
You need to have Administrator rights on your laptop. All the following commands must be executed in a command-line window (cmd) ran as Administrator, i.e. using Run as administrator option while executing cmd.
"},{"location":"spark-tips-and-tricks-running-spark-windows/#read-the-official-document-in-microsoft-technet-httpstechnetmicrosoftcomen-uslibrarycc947813vws10aspxstart-a-command-prompt-as-an-administrator","title":"Read the official document in Microsoft TechNet -- ++https://technet.microsoft.com/en-us/library/cc947813(v=ws.10).aspx++[Start a Command Prompt as an Administrator].","text":"
Download winutils.exe binary from https://github.com/steveloughran/winutils repository.
NOTE: You should select the version of Hadoop the Spark distribution was compiled with, e.g. use hadoop-2.7.1 for Spark 2 (https://github.com/steveloughran/winutils/blob/master/hadoop-2.7.1/bin/winutils.exe[here is the direct link to winutils.exe binary]).
Save winutils.exe binary to a directory of your choice, e.g. c:\\hadoop\\bin.
Set HADOOP_HOME to reflect the directory with winutils.exe (without bin).
set HADOOP_HOME=c:\\hadoop\n
Set PATH environment variable to include %HADOOP_HOME%\\bin as follows:
set PATH=%HADOOP_HOME%\\bin;%PATH%\n
TIP: Define HADOOP_HOME and PATH environment variables in Control Panel so any Windows program would use them.
c:\\tmp\\hive directory is the default value of https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.exec.scratchdir[hive.exec.scratchdir configuration property] in Hive 0.14.0 and later and Spark uses a custom build of Hive 1.2.1.
"},{"location":"spark-tips-and-tricks-running-spark-windows/#you-can-change-hiveexecscratchdir-configuration-property-to-another-directory-as-described-in-wzxhzdk27-configuration-property-in-this-document","title":"You can change hive.exec.scratchdir configuration property to another directory as described in <hive.exec.scratchdir Configuration Property>> in this document.
Execute the following command in cmd that you started using the option Run as administrator.
winutils.exe chmod -R 777 C:\\tmp\\hive\n
Check the permissions (that is one of the commands that are executed under the covers):
winutils.exe ls -F C:\\tmp\\hive\n
Open spark-shell and observe the output (perhaps with few WARN messages that you can simply disregard).
As a verification step, execute the following line to display the content of a DataFrame:
16/12/26 22:05:41 WARN General: Plugin (Bundle) \"org.datanucleus\" is already registered. Ensure you dont have multiple JAR versions of\nthe same plugin in the classpath. The URL \"file:/C:/spark-2.0.2-bin-hadoop2.7/jars/datanucleus-core-3.2.10.jar\" is already registered,\nand you are trying to register an identical plugin located at URL \"file:/C:/spark-2.0.2-bin-hadoop2.7/bin/../jars/datanucleus-core-\n3.2.10.jar.\"\n16/12/26 22:05:41 WARN General: Plugin (Bundle) \"org.datanucleus.api.jdo\" is already registered. Ensure you dont have multiple JAR\nversions of the same plugin in the classpath. The URL \"file:/C:/spark-2.0.2-bin-hadoop2.7/jars/datanucleus-api-jdo-3.2.6.jar\" is already\nregistered, and you are trying to register an identical plugin located at URL \"file:/C:/spark-2.0.2-bin-\nhadoop2.7/bin/../jars/datanucleus-api-jdo-3.2.6.jar.\"\n16/12/26 22:05:41 WARN General: Plugin (Bundle) \"org.datanucleus.store.rdbms\" is already registered. Ensure you dont have multiple JAR\nversions of the same plugin in the classpath. The URL \"file:/C:/spark-2.0.2-bin-hadoop2.7/bin/../jars/datanucleus-rdbms-3.2.9.jar\" is\nalready registered, and you are trying to register an identical plugin located at URL \"file:/C:/spark-2.0.2-bin-\nhadoop2.7/jars/datanucleus-rdbms-3.2.9.jar.\"\n
If you see the above output, you're done. You should now be able to run Spark applications on your Windows. Congrats!
Create a hive-site.xml file with the following content:
<configuration>\n <property>\n <name>hive.exec.scratchdir</name>\n <value>/tmp/mydir</value>\n <description>Scratch space for Hive jobs</description>\n </property>\n</configuration>\n
Start a Spark application, e.g. spark-shell, with HADOOP_CONF_DIR environment variable set to the directory with hive-site.xml.
HADOOP_CONF_DIR=conf ./bin/spark-shell\n
","text":""},{"location":"spark-tips-and-tricks-sparkexception-task-not-serializable/","title":"Task not serializable Exception","text":"
== org.apache.spark.SparkException: Task not serializable
When you run into org.apache.spark.SparkException: Task not serializable exception, it means that you use a reference to an instance of a non-serializable class inside a transformation. See the following example:
\u279c spark git:(master) \u2717 ./bin/spark-shell\nWelcome to\n ____ __\n / __/__ ___ _____/ /__\n _\\ \\/ _ \\/ _ `/ __/ '_/\n /___/ .__/\\_,_/_/ /_/\\_\\ version 1.6.0-SNAPSHOT\n /_/\n\nUsing Scala version 2.11.7 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_66)\nType in expressions to have them evaluated.\nType :help for more information.\n\nscala> class NotSerializable(val num: Int)\ndefined class NotSerializable\n\nscala> val notSerializable = new NotSerializable(10)\nnotSerializable: NotSerializable = NotSerializable@2700f556\n\nscala> sc.parallelize(0 to 10).map(_ => notSerializable.num).count\norg.apache.spark.SparkException: Task not serializable\n at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:304)\n at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:294)\n at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:122)\n at org.apache.spark.SparkContext.clean(SparkContext.scala:2055)\n at org.apache.spark.rdd.RDD$$anonfun$map$1.apply(RDD.scala:318)\n at org.apache.spark.rdd.RDD$$anonfun$map$1.apply(RDD.scala:317)\n at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)\n at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:111)\n at org.apache.spark.rdd.RDD.withScope(RDD.scala:310)\n at org.apache.spark.rdd.RDD.map(RDD.scala:317)\n ... 48 elided\nCaused by: java.io.NotSerializableException: NotSerializable\nSerialization stack:\n - object not serializable (class: NotSerializable, value: NotSerializable@2700f556)\n - field (class: $iw, name: notSerializable, type: class NotSerializable)\n - object (class $iw, $iw@10e542f3)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@729feae8)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@5fc3b20b)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@36dab184)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@5eb974)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@79c514e4)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@5aeaee3)\n - field (class: $iw, name: $iw, type: class $iw)\n - object (class $iw, $iw@2be9425f)\n - field (class: $line18.$read, name: $iw, type: class $iw)\n - object (class $line18.$read, $line18.$read@6311640d)\n - field (class: $iw, name: $line18$read, type: class $line18.$read)\n - object (class $iw, $iw@c9cd06e)\n - field (class: $iw, name: $outer, type: class $iw)\n - object (class $iw, $iw@6565691a)\n - field (class: $anonfun$1, name: $outer, type: class $iw)\n - object (class $anonfun$1, <function1>)\n at org.apache.spark.serializer.SerializationDebugger$.improveException(SerializationDebugger.scala:40)\n at org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:47)\n at org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:101)\n at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:301)\n ... 57 more\n
=== Further reading
https://databricks.gitbooks.io/databricks-spark-knowledge-base/content/troubleshooting/javaionotserializableexception.html[Job aborted due to stage failure: Task not serializable]
https://issues.apache.org/jira/browse/SPARK-5307[Add utility to help with NotSerializableException debugging]
http://stackoverflow.com/q/22592811/1305344[Task not serializable: java.io.NotSerializableException when calling function outside closure only on classes not objects]
"},{"location":"spark-tips-and-tricks/","title":"Spark Tips and Tricks","text":"
= Spark Tips and Tricks
== [[SPARK_PRINT_LAUNCH_COMMAND]] Print Launch Command of Spark Scripts
SPARK_PRINT_LAUNCH_COMMAND environment variable controls whether the Spark launch command is printed out to the standard error output, i.e. System.err, or not.
Spark Command: [here comes the command]\n========================================\n
All the Spark shell scripts use org.apache.spark.launcher.Main class internally that checks SPARK_PRINT_LAUNCH_COMMAND and when set (to any value) will print out the entire command line to launch it.
When you face networking issues when Spark can't resolve your local hostname or IP address, use the preferred SPARK_LOCAL_HOSTNAME environment variable as the custom host name or SPARK_LOCAL_IP as the custom IP that is going to be later resolved to a hostname.
Spark checks them out before using http://docs.oracle.com/javase/8/docs/api/java/net/InetAddress.html#getLocalHost--[java.net.InetAddress.getLocalHost()] (consult https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/util/Utils.scala#L759[org.apache.spark.util.Utils.findLocalInetAddress()] method).
You may see the following WARN messages in the logs when Spark finished the resolving process:
Your hostname, [hostname] resolves to a loopback address: [host-address]; using...\nSet SPARK_LOCAL_IP if you need to bind to another address\n
"},{"location":"spark-tips-and-tricks/#starting-standalone-master-and-workers-on-windows-7","title":"Starting standalone Master and workers on Windows 7","text":"
Windows 7 users can use spark-class to start Spark Standalone as there are no launch scripts for the Windows platform.
"},{"location":"speculative-execution-of-tasks/","title":"Speculative Execution of Tasks","text":"
Speculative tasks (also speculatable tasks or task strugglers) are tasks that run slower than most (FIXME the setting) of the all tasks in a job.
Speculative execution of tasks is a health-check procedure that checks for tasks to be speculated, i.e. running slower in a stage than the median of all successfully completed tasks in a taskset (FIXME the setting). Such slow tasks will be re-submitted to another worker. It will not stop the slow tasks, but run a new copy in parallel.
The thread starts as TaskSchedulerImpl starts in spark-cluster.md[clustered deployment modes] with configuration-properties.md#spark.speculation[spark.speculation] enabled. It executes periodically every configuration-properties.md#spark.speculation.interval[spark.speculation.interval] after the initial spark.speculation.interval passes.
When enabled, you should see the following INFO message in the logs:
It works as scheduler:TaskSchedulerImpl.md#task-scheduler-speculation[task-scheduler-speculation daemon thread pool] (using j.u.c.ScheduledThreadPoolExecutor with core pool size of 1).
The job with speculatable tasks should finish while speculative tasks are running, and it will leave these tasks running - no KILL command yet.
It uses checkSpeculatableTasks method that asks rootPool to check for speculatable tasks. If there are any, SchedulerBackend is called for scheduler:SchedulerBackend.md#reviveOffers[reviveOffers].
CAUTION: FIXME How does Spark handle repeated results of speculative tasks since there are copies launched?
Workers (aka slaves) are running Spark instances where executors live to execute tasks. They are the compute nodes in Spark.
CAUTION: FIXME Are workers perhaps part of Spark Standalone only?
CAUTION: FIXME How many executors are spawned per worker?
A worker receives serialized tasks that it runs in a thread pool.
It hosts a local storage:BlockManager.md[Block Manager] that serves blocks to other workers in a Spark cluster. Workers communicate among themselves using their Block Manager instances.
CAUTION: FIXME Diagram of a driver with workers as boxes.
Explain task execution in Spark and understand Spark\u2019s underlying execution model.
New vocabulary often faced in Spark UI
SparkContext.md[When you create SparkContext], each worker starts an executor. This is a separate process (JVM), and it loads your jar, too. The executors connect back to your driver program. Now the driver can send them commands, like flatMap, map and reduceByKey. When the driver quits, the executors shut down.
A new process is not started for each step. A new process is started on each worker when the SparkContext is constructed.
The executor deserializes the command (this is possible because it has loaded your jar), and executes it on a partition.
Shortly speaking, an application in Spark is executed in three steps:
Create RDD graph, i.e. DAG (directed acyclic graph) of RDDs to represent entire computation.
Create stage graph, i.e. a DAG of stages that is a logical execution plan based on the RDD graph. Stages are created by breaking the RDD graph at shuffle boundaries.
Based on the plan, schedule and execute tasks on workers.
exercises/spark-examples-wordcount-spark-shell.md[In the WordCount example], the RDD graph is as follows:
file -> lines -> words -> per-word count -> global word count -> output
Based on this graph, two stages are created. The stage creation rule is based on the idea of pipelining as many rdd:index.md[narrow transformations] as possible. RDD operations with \"narrow\" dependencies, like map() and filter(), are pipelined together into one set of tasks in each stage.
In the end, every stage will only have shuffle dependencies on other stages, and may compute multiple operations inside it.
In the WordCount example, the narrow transformation finishes at per-word count. Therefore, you get two stages:
file -> lines -> words -> per-word count
global word count -> output
Once stages are defined, Spark will generate scheduler:Task.md[tasks] from scheduler:Stage.md[stages]. The first stage will create scheduler:ShuffleMapTask.md[ShuffleMapTask]s with the last stage creating scheduler:ResultTask.md[ResultTask]s because in the last stage, one action operation is included to produce results.
The number of tasks to be generated depends on how your files are distributed. Suppose that you have 3 three different files in three different nodes, the first stage will generate 3 tasks: one task per partition.
Therefore, you should not map your steps to tasks directly. A task belongs to a stage, and is related to a partition.
The number of tasks being generated in each stage will be equal to the number of partitions.
Accumulators are shared variables that accumulate values from executors on the driver using associative and commutative \"add\" operation.
The main abstraction is AccumulatorV2.
Accumulators are registered (created) using SparkContext with or without a name. Only named accumulators are displayed in web UI.
DAGScheduler is responsible for updating accumulators (from partial values from tasks running on executors every heartbeat).
Accumulators are serializable so they can safely be referenced in the code executed in executors and then safely send over the wire for execution.
// on the driver\nval counter = sc.longAccumulator(\"counter\")\n\nsc.parallelize(1 to 9).foreach { x =>\n // on executors\n counter.add(x) }\n\n// on the driver\nprintln(counter.value)\n
AccumulatorContext is a private[spark] internal object used to track accumulators by Spark itself using an internal originals lookup table. Spark uses the AccumulatorContext object to register and unregister accumulators.
The originals lookup table maps accumulator identifier to the accumulator itself.
Every accumulator has its own unique accumulator id that is assigned using the internal nextId counter.
AccumulatorContext.SQL_ACCUM_IDENTIFIER is an internal identifier for Spark SQL's internal accumulators. The value is sql and Spark uses it to distinguish spark-sql-SparkPlan.md#SQLMetric[Spark SQL metrics] from others.
AccumulatorV2[IN, OUT] is an abstraction of accumulators
AccumulatorV2 is a Java Serializable.
"},{"location":"accumulators/AccumulatorV2/#contract","title":"Contract","text":""},{"location":"accumulators/AccumulatorV2/#adding-value","title":"Adding Value
add(\n v: IN): Unit\n
Accumulates (adds) the given v value to this accumulator
toInfo determines whether the accumulator is internal based on the name (and whether it uses the internal.metrics prefix) and uses it to create an AccumulableInfo.
toInfo\u00a0is used when:
TaskRunner is requested to collectAccumulatorsAndResetStatusOnFailure
DAGScheduler is requested to updateAccumulators
TaskSchedulerImpl is requested to executorHeartbeatReceived
JsonProtocol is requested to taskEndReasonFromJson
SQLAppStatusListener (Spark SQL) is requested to handle a SparkListenerTaskEnd event (onTaskEnd)
readObject is part of the Serializable (Java) abstraction (for special handling during deserialization).
readObject reads the non-static and non-transient fields of the AccumulatorV2 from the given ObjectInputStream.
If the atDriverSide internal flag is turned on, readObject turns it off (to indicate readObject is executed on an executor). Otherwise, atDriverSide internal flag is turned on.
readObject requests the active TaskContext to register this accumulator.
Barrier Execution Mode (Barrier Scheduling) introduces a strong requirement on Spark Scheduler to launch all tasks of a Barrier Stage at the same time or not at all (and consequently wait until required resources are available). Moreover, a failure of a single task of a barrier stage fails the whole stage (and so the other tasks).
Barrier Execution Mode allows for as many tasks to be executed concurrently as ResourceProfile permits (that is enforced upon scheduling a barrier job).
Barrier Execution Mode aims at making Distributed Deep Learning with Apache Spark easier (or even possible).
Rephrasing dmlc/xgboost, Barrier Execution Mode makes sure that:
All tasks of a barrier stage are all launched at once. If there is not enough task slots, the exception will be produced
Tasks either all succeed or fail. Upon a task failure Spark aborts all the other tasks (TaskScheduler will kill all other running tasks) and restarts the whole barrier stage
Spark makes no assumption that tasks don't talk to each other. Actually, it is the opposite. Spark provides BarrierTaskContext which facilitates tasks discovery (e.g., barrier, allGather)
Permits restarting a training from a known state (checkpoint) in case of a failure
From the Design doc: Barrier Execution Mode:
In Spark, a task in a stage doesn't depend on any other task in the same stage, and hence it can be scheduled independently.
That gives Spark a freedom to schedule tasks in as many task batches as needed. So, 5 tasks can be scheduled on 1 CPU core quite easily in 5 consecutive batches. That's unlike MPI (or non-MapReduce scheduling systems) that allows for greater flexibility and inter-task dependency.
Later in Design doc: Barrier Execution Mode:
In MPI, all workers start at the same time and pass messages around.
To embed this workload in Spark, we need to introduce a new scheduling model, tentatively named \"barrier scheduling\", which launches the tasks at the same time and provides users enough information and tooling to embed distributed DL training into a Spark pipeline.
Barrier Execution Mode is based on RDD.barrier operator to indicate that Spark Scheduler must launch the tasks together for the current stage (and mark the current stage as a barrier stage).
barrier(): RDDBarrier[T]\n
RDD.barrier creates a RDDBarrier that comes with the barrier-aware mapPartitions transformation.
Under the covers, RDDBarrier.mapPartitions creates a MapPartitionsRDD like the regular RDD.mapPartitions transformation but with isFromBarrier flag enabled.
Task has a isBarrier flag that says whether this task belongs to a barrier stage (default: false).
Use RDD.mapPartitions transformation to access a BarrierTaskContext.
val barrierRdd = nums\n .barrier\n .mapPartitions { ns =>\n import org.apache.spark.{BarrierTaskContext, TaskContext}\n val ctx = TaskContext.get.asInstanceOf[BarrierTaskContext]\n val tid = ctx.partitionId()\n val port = 10000 + tid\n val host = \"localhost\"\n val message = s\"A message from task $tid, e.g. $host:$port it listens at\"\n val allTaskMessages = ctx.allGather(message)\n\n if (tid == 0) { // only Task 0 prints out status\n println(\">>> Got host:port's from the other tasks\")\n allTaskMessages.foreach(println)\n }\n\n if (tid == 0) { // only Task 0 prints out status\n println(\">>> Starting a distributed training at the nodes...\")\n }\n\n ctx.barrier() // this is BarrierTaskContext.barrier (not RDD.barrier)\n // which can be confusing\n\n if (tid == 0) { // only Task 0 prints out status\n println(\">>> All tasks have finished\")\n }\n\n // return a model after combining (model) pieces from the nodes\n ns\n }\n
Run a distributed computation (using RDD.count action).
barrierRdd.count()\n
There should be INFO and TRACE messages printed out to the console (given ALL logging level for org.apache.spark.BarrierTaskContext logger).
[Executor task launch worker for task 1.0 in stage 5.0 (TID 13)] INFO org.apache.spark.BarrierTaskContext:60 - Task 13 from Stage 5(Attempt 0) has entered the global sync, current barrier epoch is 0.\n...\n[Executor task launch worker for task 1.0 in stage 5.0 (TID 13)] TRACE org.apache.spark.BarrierTaskContext:68 - Current callSite: CallSite($anonfun$runBarrier$2 at Logging.scala:68,org.apache.spark.BarrierTaskContext.$anonfun$runBarrier$2(BarrierTaskContext.scala:61)\n...\n[Executor task launch worker for task 1.0 in stage 5.0 (TID 13)] INFO org.apache.spark.BarrierTaskContext:60 - Task 13 from Stage 5(Attempt 0) finished global sync successfully, waited for 1 seconds, current barrier epoch is 1.\n...\n
XGBoost4J is the JVM package of xgboost (an optimized distributed gradient boosting library with machine learning algorithms for regression and classification under the Gradient Boosting framework).
The heart of distributed training in xgboost4j-spark (that can run distributed xgboost on Apache Spark) is XGBoost.trainDistributed.
There's a familiar line that creates a barrier stage (using RDD.barrier()):
val boostersAndMetrics = trainingRDD.barrier().mapPartitions {\n // distributed training using XGBoost happens here\n}\n
The barrier mapPartitions block finishes is followed by RDD.collect() that gets XGBoost4J-specific metadata (booster and metrics):
val (booster, metrics) = boostersAndMetrics.collect()(0)\n
Within the barrier stage (within mapPartitions block), xgboost4j-spark builds a distributed booster:
Checkpointing, when enabled, happens only by Task 0
All tasks initialize so-called collective Communicator for synchronization
xgboost4j-spark uses XGBoostJNI to talk to XGBoost using JNI
Only Task 0 returns non-empty iterator (and that's why the RDD.collect()(0) gets (booster, metrics))
All tasks execute SXGBoost.train that eventually leads to XGBoost.trainAndSaveCheckpoint
BarrierCoordinator is a ThreadSafeRpcEndpoint that is registered as barrierSync RPC Endpoint when TaskSchedulerImpl is requested to maybeInitBarrierCoordinator.
BarrierCoordinator is responsible for handling RequestToSync messages to coordinate Global Syncs of barrier tasks (using allGather and barrier operators).
In other words, the driver sets up a BarrierCoordinator (TaskSchedulerImpl precisely) upon startup that BarrierTaskContexts talk to using RequestToSync messages. BarrierCoordinator tracks the number of tasks to wait for until a barrier stage is complete and a response can be sent back to the tasks to continue (that are paused for 365 days (!)).
receiveAndReply is part of the RpcEndpoint abstraction.
receiveAndReply handles RequestToSync messages.
Unless already registered, receiveAndReply registers a new ContextBarrierId (for the stageId and the stageAttemptId) in the Barrier States registry.
Multiple Tasks and One BarrierCoordinator
receiveAndReply handles RequestToSync messages, one per task in a barrier stage. Out of all the properties of RequestToSync, numTasks, stageId and stageAttemptId are used.
The very first RequestToSync is used to register the stageId and stageAttemptId (as ContextBarrierId) with numTasks.
receiveAndReply finds the ContextBarrierState for the stage and the stage attempt (in the Barrier States registry) to handle the RequestToSync.
BarrierCoordinatorMessage is an abstraction of RPC messages that tasks can send out using BarrierTaskContext operators for BarrierCoordinator to handle.
BarrierCoordinatorMessage is a Serializable (Java) (so it can be sent from executors to the driver over the wire).
BarrierJobSlotsNumberCheckFailed is a BarrierJobAllocationFailed with the following exception message:
[SPARK-24819]: Barrier execution mode does not allow run a barrier stage that requires more slots than the total number of slots in the cluster currently.\nPlease init a new cluster with more resources(e.g. CPU, GPU) or repartition the input RDD(s) to reduce the number of slots required to run this barrier stage.\n
BarrierJobSlotsNumberCheckFailed can be thrown when DAGScheduler is requested to handle a JobSubmitted event.
runBarrier prints out the following INFO message to the logs:
Task [taskAttemptId] from Stage [stageId](Attempt [stageAttemptNumber]) has entered the global sync, current barrier epoch is [barrierEpoch].\n
runBarrier prints out the following TRACE message to the logs:
Current callSite: [callSite]\n
runBarrier schedules a TimerTask (Java) to print out the following INFO message to the logs every minute:
Task [taskAttemptId] from Stage [stageId](Attempt [stageAttemptNumber]) waiting under the global sync since [startTime],\nhas been waiting for [duration] seconds,\ncurrent barrier epoch is [barrierEpoch].\n
runBarrier requests the Barrier Coordinator RPC Endpoint to send a RequestToSync one-off message and waits 365 days (!) for a response (a collection of responses from all the barrier tasks).
1 Year to Wait for Response from Barrier Coordinator
runBarrier uses 1 year to wait until the response arrives.
runBarrier checks every second if the response \"bundle\" arrived.
runBarrier increments the barrierEpoch.
runBarrier prints out the following INFO message to the logs:
Task [taskAttemptId] from Stage [stageId](Attempt [stageAttemptNumber]) finished global sync successfully,\nwaited for [duration] seconds,\ncurrent barrier epoch is [barrierEpoch].\n
In the end, runBarrier returns the response \"bundle\" (a collection of responses from all the barrier tasks).
In case of a SparkException, runBarrier prints out the following INFO message to the logs and reports (re-throws) the exception up (the call chain):
Task [taskAttemptId] from Stage [stageId](Attempt [stageAttemptNumber]) failed to perform global sync,\nwaited for [duration] seconds,\ncurrent barrier epoch is [barrierEpoch].\n
runBarrier is used when:
BarrierTaskContext is requested to barrier, allGather
requesters is a registry of RpcCallContexts of the barrier tasks (of a barrier stage attempt) pending a reply.
It is only when the number of RpcCallContexts in the requesters reaches the number of tasks expected (while handling RequestToSync requests) that this ContextBarrierState is considered finished successfully.
ContextBarrierState initializes requesters when created to be of number of tasks size.
A new RpcCallContext of a barrier task is added in handleRequest only when the epoch of the barrier task matches the current barrierEpoch.
ContextBarrierState uses a TimerTask (Java) to ensure that a barrier() call can time out.
ContextBarrierState creates a TimerTask (Java) when requested to initTimerTask when requested to handle a RequestToSync message for the first global sync message received (when the requesters is empty). The TimerTask is then immediately scheduled to be executed after spark.barrier.sync.timeout.
spark.barrier.sync.timeout
Since spark.barrier.sync.timeout defaults to 365d (1 year), the TimerTask will run only after one year.
initTimerTask creates a new TimerTask (Java) that, when executed, sends a SparkException to all the requesters with the following message followed by cleanupBarrierStage for this ContextBarrierId.
The coordinator didn't get all barrier sync requests\nfor barrier epoch [barrierEpoch] from [barrierId] within [timeoutInSecs] second(s).\n
The TimerTask is made available as timerTask.
initTimerTask is used when:
ContextBarrierState is requested to handle a RequestToSync message (for the first global sync message received when the requesters is empty)
handleRequest makes sure that the RequestMethod (of the given RequestToSync) is consistent across barrier tasks (using requestMethods registry).
handleRequest asserts that the number of tasks is this numTasks, and so consistent across barrier tasks. Otherwise, handleRequest reports IllegalArgumentException:
Number of tasks of [barrierId] is [numTasks] from Task [taskId], previously it was [numTasks].\n
handleRequest prints out the following INFO message to the logs (with the ContextBarrierId and barrierEpoch):
Current barrier epoch for [barrierId] is [barrierEpoch].\n
For the first sync message received (requesters is empty), handleRequest initializes the TimerTask and schedules it for execution after the timeoutInSecs.
Timeout
Starting the timerTask ensures that a sync may eventually time out (after a configured delay).
handleRequest registers the given requester in the requesters.
handleRequest registers the message of the RequestToSync in the messages for the partitionId.
handleRequest prints out the following INFO message to the logs:
Barrier sync epoch [barrierEpoch] from [barrierId] received update from Task taskId,\ncurrent progress: [requesters]/[numTasks].\n
"},{"location":"barrier-execution-mode/ContextBarrierState/#updates-from-all-barrier-tasks-received","title":"Updates from All Barrier Tasks Received","text":"
When the barrier sync received updates from all barrier tasks (i.e., the number of requesters is the numTasks), handleRequest replies back to all the requesters with the messages.
handleRequest prints out the following INFO message to the logs:
Barrier sync epoch [barrierEpoch] from [barrierId] received all updates from tasks,\nfinished successfully.\n
handleRequest increments the barrierEpoch, clears the requesters and the requestMethods, and then cancelTimerTask.
In case of the epoch of the given RequestToSync being different from this barrierEpoch, handleRequest sends back a failure message (with a SparkException) to the given requester:
The request to sync of [barrierId] with barrier epoch [barrierEpoch] has already finished.\nMaybe task [taskId] is not properly killed.\n
In case of different RequestMethods (in requestMethods registry), handleRequest sends back a failure message to the requesters (incl. the given requester):
Different barrier sync types found for the sync [barrierId]: [requestMethods].\nPlease use the same barrier sync type within a single sync.\n
handleRequest clear.
handleRequest is used when:
BarrierCoordinator is requested to handle a RequestToSync message
RequestMethod represents the allowed request methods of RequestToSyncs (that are sent out from barrier tasks using BarrierTaskContext).
ContextBarrierState tracks RequestMethods (from tasks inside a barrier sync) to make sure that the tasks are all part of a legitimate barrier sync. All tasks should make sure that they're calling the same method within the same barrier sync phase.
From the official documentation about Broadcast Variables:
Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather than shipping a copy of it with tasks.
And later in the document:
Explicitly creating broadcast variables is only useful when tasks across multiple stages need the same data or when caching the data in deserialized form is important.
Spark uses SparkContext to create broadcast variables and BroadcastManager with ContextCleaner to manage their lifecycle.
Not only can Spark developers use broadcast variables for efficient data distribution, but Spark itself uses them quite often too. A very notable use case is when Spark distributes tasks (to executors) for execution.
The idea is to transfer values used in transformations from a driver to executors in a most effective way so they are copied once and used many times by tasks (rather than being copied every time a task is launched).
"},{"location":"broadcast-variables/#lifecycle-of-broadcast-variable","title":"Lifecycle of Broadcast Variable
Broadcast variables (TorrentBroadcasts, actually) are created using SparkContext.broadcast method.
scala> val b = sc.broadcast(1)\nb: org.apache.spark.broadcast.Broadcast[Int] = Broadcast(0)\n
Tip
Enable DEBUG logging level for org.apache.spark.storage.BlockManager logger to debug broadcast method.
With DEBUG logging level enabled, there should be the following messages printed out to the logs:
Put block broadcast_0 locally took 430 ms\nPutting block broadcast_0 without replication took 431 ms\nTold master about block broadcast_0_piece0\nPut block broadcast_0_piece0 locally took 4 ms\nPutting block broadcast_0_piece0 without replication took 4 ms\n
A broadcast variable is stored on the driver's BlockManager as a single value and separately as chunks (of spark.broadcast.blockSize).
When requested for the broadcast value, TorrentBroadcast reads the broadcast block from the local BroadcastManager and, if fails, from the local BlockManager. Only when the local lookups fail, TorrentBroadcast reads the broadcast block chunks (from the BlockMannagers on the other executors), persists them as a single broadcast variable (in the local BlockManager) and caches in BroadcastManager.
scala> b.value\nres0: Int = 1\n
Broadcast.value is the only way to access the value of a broadcast variable in a Spark transformation. You can only access the broadcast value any time until the broadcast variable is destroyed.
With DEBUG logging level enabled, there should be the following messages printed out to the logs:
Getting local block broadcast_0\nLevel for block broadcast_0 is StorageLevel(disk, memory, deserialized, 1 replicas)\n
In the end, broadcast variables should be destroyed to release memory.
b.destroy\n
With DEBUG logging level enabled, there should be the following messages printed out to the logs:
You use broadcast variable to implement map-side join, i.e. a join using a map. For this, lookup tables are distributed across nodes in a cluster using broadcast and then looked up inside map (to do the join implicitly).
When you broadcast a value, it is copied to executors only once (while it is copied multiple times for tasks otherwise). It means that broadcast can help to get your Spark application faster if you have a large value to use in tasks or there are more tasks than executors.
It appears that a Spark idiom emerges that uses broadcast with collectAsMap to create a Map for broadcast. When an RDD is map over to a smaller dataset (column-wise not record-wise), collectAsMap, and broadcast, using the very big RDD to map its elements to the broadcast RDDs is computationally faster.
val acMap = sc.broadcast(myRDD.map { case (a,b,c,b) => (a, c) }.collectAsMap)\nval otherMap = sc.broadcast(myOtherRDD.collectAsMap)\n\nmyBigRDD.map { case (a, b, c, d) =>\n (acMap.value.get(a).get, otherMap.value.get(c).get)\n}.collect\n
Use large broadcasted HashMaps over RDDs whenever possible and leave RDDs with a key to lookup necessary data as demonstrated above.
You're going to use a static mapping of interesting projects with their websites, i.e. Map[String, String] that the tasks, i.e. closures (anonymous functions) in transformations, use.
It works, but is very ineffective as the pws map is sent over the wire to executors while it could have been there already. If there were more tasks that need the pws map, you could improve their performance by minimizing the number of bytes that are going to be sent over the network for task execution.
Semantically, the two computations - with and without the broadcast value - are exactly the same, but the broadcast-based one wins performance-wise when there are more executors spawned to execute many tasks that use pws map.
","text":""},{"location":"broadcast-variables/#further-reading-or-watching","title":"Further Reading or Watching
newBroadcast requests the BroadcastFactory for a new broadcast variable (with the next available broadcast ID).
newBroadcast\u00a0is used when:
SparkContext is requested for a new broadcast variable
MapOutputTracker utility is used to serializeMapStatuses
","text":""},{"location":"broadcast-variables/BroadcastManager/#unique-identifiers-of-broadcast-variables","title":"Unique Identifiers of Broadcast Variables
BroadcastManager tracks broadcast variables and assigns unique and continuous identifiers.
readBroadcastBlock looks up the BroadcastBlockId in (the cache of) BroadcastManager and returns the value if found.
Otherwise, readBroadcastBlock setConf and requests the BlockManager for the locally-stored broadcast data.
If the broadcast block is found locally, readBroadcastBlock requests the BroadcastManager to cache it and returns the value.
If not found locally, readBroadcastBlock multiplies the numBlocks by the blockSize for an estimated size of the broadcast block. readBroadcastBlock prints out the following INFO message to the logs:
Started reading broadcast variable [id] with [numBlocks] pieces\n(estimated total size [estimatedTotalSize])\n
readBroadcastBlock readBlocks and prints out the following INFO message to the logs:
Reading broadcast variable [id] took [time] ms\n
readBroadcastBlock unblockifies the block chunks into an object (using the Serializer and the CompressionCodec).
readBroadcastBlock requests the BlockManager to store the merged copy (so other tasks on this executor don't need to re-fetch it). readBroadcastBlock uses MEMORY_AND_DISK storage level and the tellMaster flag off.
readBroadcastBlock requests the BroadcastManager to cache it and returns the value.
","text":""},{"location":"broadcast-variables/TorrentBroadcast/#unblockifying-broadcast-value","title":"Unblockifying Broadcast Value
readBlocks creates a collection of BlockDatas for numBlocks block chunks.
For every block (randomly-chosen by block ID between 0 and numBlocks), readBlocks creates a BroadcastBlockId for the id (of the broadcast variable) and the chunk (identified by the piece prefix followed by the ID).
readBlocks prints out the following DEBUG message to the logs:
Reading piece [pieceId] of [broadcastId]\n
readBlocks first tries to look up the piece locally by requesting the BlockManager to getLocalBytes and, if found, stores the reference in the local block array (for the piece ID).
If not found in the local BlockManager, readBlocks requests the BlockManager to getRemoteBytes.
With checksumEnabled, readBlocks...FIXME
readBlocks requests the BlockManager to store the chunk (so other tasks on this executor don't need to re-fetch it) using MEMORY_AND_DISK_SER storage level and reporting to the driver (so other executors can pull these chunks from this executor as well).
readBlocks creates a ByteBufferBlockData for the chunk (and stores it in the blocks array).
readBlocks throws a SparkException for blocks neither available locally nor remotely:
writeBlocks returns the number of blocks (chunks) this broadcast variable (was blockified into).
The whole broadcast value is stored in the local BlockManager with MEMORY_AND_DISK storage level while the block chunks with MEMORY_AND_DISK_SER storage level.
writeBlocks\u00a0is used when:
TorrentBroadcast is created (that happens on the driver only)
writeBlocks requests the BlockManager to store the given broadcast value (to be identified as the broadcastId and with the MEMORY_AND_DISK storage level).
writeBlocks blockify the object (into chunks of the block size, the Serializer, and the optional compressionCodec).
With checksumEnabled writeBlocks...FIXME
For every block, writeBlocks creates a BroadcastBlockId for the id and piece[index] identifier, and requests the BlockManager to store the chunk bytes (with MEMORY_AND_DISK_SER storage level and reporting to the driver).
blockifyObject divides (blockifies) the input obj broadcast value into blocks (ByteBuffer chunks). blockifyObject uses the given Serializer to write the value in a serialized format to a ChunkedByteBufferOutputStream of the given blockSize size with the optional CompressionCodec.
doUnpersist removes the persisted state (associated with the broadcast variable) on executors only.
doUnpersist\u00a0is part of the Broadcast abstraction.
","text":""},{"location":"broadcast-variables/TorrentBroadcast/#removing-persisted-state-broadcast-blocks-of-broadcast-variable","title":"Removing Persisted State (Broadcast Blocks) of Broadcast Variable
NettyBlockTransferService is requested to storage:NettyBlockTransferService.md#fetchBlocks[fetchBlocks] (when network:TransportConf.md#io.maxRetries[maxIORetries] is 0)
RetryingBlockFetcher is requested to core:RetryingBlockFetcher.md#fetchAllOutstanding[fetchAllOutstanding]
CleanerListener is an abstraction of listeners that can be core:ContextCleaner.md#attachListener[registered with ContextCleaner] to be informed when <>, <>, <>, <> and <> are cleaned.
ContextCleaner is a Spark service that is responsible for <> (cleanup) of <>, <>, <>, <> and <> that is aimed at reducing the memory requirements of long-running data-heavy Spark applications.
ContextCleaner is created and requested to start when SparkContext is created with configuration-properties.md#spark.cleaner.referenceTracking[spark.cleaner.referenceTracking] configuration property enabled.
registerRDDCheckpointDataForCleanup is used when ContextCleaner is requested to <> (with configuration-properties.md#spark.cleaner.referenceTracking.cleanCheckpoints[spark.cleaner.referenceTracking.cleanCheckpoints] configuration property enabled).
start starts the <> and an action to request the JVM garbage collector (using System.gc()) on regular basis per configuration-properties.md#spark.cleaner.periodicGC.interval[spark.cleaner.periodicGC.interval] configuration property.
The action to request the JVM GC is scheduled on <>.
start is used when SparkContext is created.
== [[periodicGCService]] periodicGCService Single-Thread Executor Service
periodicGCService is an internal single-thread {java-javadoc-url}/java/util/concurrent/ScheduledExecutorService.html[executor service] with the name context-cleaner-periodic-gc to request the JVM garbage collector.
The periodic runs are started when <> and stopped when <>.
== [[registerShuffleForCleanup]] Registering ShuffleDependency for Cleanup
registerForCleanup( objectForCleanup: AnyRef, task: CleanupTask): Unit
registerForCleanup adds the input objectForCleanup to the <> internal queue.
Despite the widest-possible AnyRef type of the input objectForCleanup, the type is really CleanupTaskWeakReference which is a custom Java's {java-javadoc-url}/java/lang/ref/WeakReference.html[java.lang.ref.WeakReference].
registerForCleanup is used when ContextCleaner is requested to <>, <>, <>, <>, and <>.
doCleanupShuffle( shuffleId: Int, blocking: Boolean): Unit
doCleanupShuffle performs a shuffle cleanup which is to remove the shuffle from the current scheduler:MapOutputTrackerMaster.md[MapOutputTrackerMaster] and storage:BlockManagerMaster.md[BlockManagerMaster]. doCleanupShuffle also notifies core:CleanerListener.md[CleanerListeners].
Internally, when executed, doCleanupShuffle prints out the following DEBUG message to the logs:
doCleanupShuffle uses core:SparkEnv.md[SparkEnv] to access the core:SparkEnv.md#mapOutputTracker[MapOutputTracker] to scheduler:MapOutputTracker.md#unregisterShuffle[unregister the given shuffle].
doCleanupShuffle uses core:SparkEnv.md[SparkEnv] to access the core:SparkEnv.md#blockManager[BlockManagerMaster] to storage:BlockManagerMaster.md#removeShuffle[remove the shuffle blocks] (for the given shuffleId).
doCleanupShuffle informs all registered <> that core:CleanerListener.md#shuffleCleaned[shuffle was cleaned].
In the end, doCleanupShuffle prints out the following DEBUG message to the logs:
RetryingBlockFetcher is <> and immediately <> when:
NettyBlockTransferService is requested to storage:NettyBlockTransferService.md#fetchBlocks[fetchBlocks] (when network:TransportConf.md#io.maxRetries[maxIORetries] is greater than 0 which it is by default)
RetryingBlockFetcher uses a <> to core:BlockFetchStarter.md#createAndStart[createAndStart] when requested to <> and later <>.
[[outstandingBlocksIds]] RetryingBlockFetcher uses outstandingBlocksIds internal registry of outstanding block IDs to fetch that is initially the <> when <>.
At <>, RetryingBlockFetcher prints out the following INFO message to the logs (with the number of <>):
Retrying fetch ([retryCount]/[maxRetries]) for [size] outstanding blocks after [retryWaitTime] ms\n
On <> and <>, <> removes the block ID from <>.
[[currentListener]] RetryingBlockFetcher uses a <> to remove block IDs from the <> internal registry.
NettyBlockTransferService is requested to storage:NettyBlockTransferService.md#fetchBlocks[fetchBlocks] (when network:TransportConf.md#io.maxRetries[maxIORetries] is greater than 0 which it is by default)
RetryingBlockFetcher is requested to <>"},{"location":"core/RetryingBlockFetcher/#retryingblockfetchlistener-is-requested-to","title":"* RetryingBlockFetchListener is requested to <>
Enable ALL logging level for org.apache.spark.storage.DiskStore and org.apache.spark.storage.DiskBlockManager loggers to have an even deeper insight on the block storage internals.
"},{"location":"dynamic-allocation/","title":"Dynamic Allocation of Executors","text":"
Dynamic Allocation of Executors (Dynamic Resource Allocation or Elastic Scaling) is a Spark service for adding and removing Spark executors dynamically on demand to match workload.
Unlike the \"traditional\" static allocation where a Spark application reserves CPU and memory resources upfront (irrespective of how much it may eventually use), in dynamic allocation you get as much as needed and no more. It scales the number of executors up and down based on workload, i.e. idle executors are removed, and when there are pending tasks waiting for executors to be launched on, dynamic allocation requests them.
Dynamic Allocation is enabled (and SparkContext creates an ExecutorAllocationManager) when:
spark.dynamicAllocation.enabled configuration property is enabled
spark.master is non-local
SchedulerBackend is an ExecutorAllocationClient
ExecutorAllocationManager is the heart of Dynamic Resource Allocation.
When enabled, it is recommended to use the External Shuffle Service.
Dynamic Allocation comes with the policy of scaling executors up and down as follows:
Scale Up Policy requests new executors when there are pending tasks and increases the number of executors exponentially since executors start slow and Spark application may need slightly more.
Scale Down Policy removes executors that have been idle for spark.dynamicAllocation.executorIdleTimeout seconds.
SparkContext offers a developer API to scale executors up or down.
"},{"location":"dynamic-allocation/#getting-initial-number-of-executors-for-dynamic-allocation","title":"Getting Initial Number of Executors for Dynamic Allocation
getDynamicAllocationInitialExecutors first makes sure that <> is equal or greater than <>.
NOTE: <> falls back to <> if not set. Why to print the WARN message to the logs?
If not, you should see the following WARN message in the logs:
spark.dynamicAllocation.initialExecutors less than spark.dynamicAllocation.minExecutors is invalid, ignoring its setting, please update your configs.\n
getDynamicAllocationInitialExecutors makes sure that executor:Executor.md#spark.executor.instances[spark.executor.instances] is greater than <>.
NOTE: Both executor:Executor.md#spark.executor.instances[spark.executor.instances] and <> fall back to 0 when no defined explicitly.
If not, you should see the following WARN message in the logs:
spark.executor.instances less than spark.dynamicAllocation.minExecutors is invalid, ignoring its setting, please update your configs.\n
getDynamicAllocationInitialExecutors sets the initial number of executors to be the maximum of:
spark.dynamicAllocation.minExecutors
spark.dynamicAllocation.initialExecutors
spark.executor.instances
0
You should see the following INFO message in the logs:
Using initial executors = [initialExecutors], max of spark.dynamicAllocation.initialExecutors, spark.dynamicAllocation.minExecutors and spark.executor.instances\n
getDynamicAllocationInitialExecutors is used when ExecutorAllocationManager is requested to set the initial number of executors.
Requests additional executors from a cluster manager and returns whether the request has been acknowledged by the cluster manager (true) or not (false).
Used when:
SparkContext is requested for additional executors
","text":""},{"location":"dynamic-allocation/ExecutorAllocationClient/#updating-total-executors","title":"Updating Total Executors
Updates a cluster manager with the exact number of executors desired. Returns whether the request has been acknowledged by the cluster manager (true) or not (false).
Used when:
SparkContext is requested to update the number of total executors
ExecutorAllocationManager is requested to start, updateAndSyncNumExecutorsTarget, addExecutors, removeExecutors
ExecutorAllocationListener is a SparkListener.md[] that intercepts events about stages, tasks, and executors, i.e. onStageSubmitted, onStageCompleted, onTaskStart, onTaskEnd, onExecutorAdded, and onExecutorRemoved. Using the events ExecutorAllocationManager can manage the pool of dynamically managed executors.
Internal Class
ExecutorAllocationListener is an internal class of ExecutorAllocationManager with full access to internal registries.
ExecutorAllocationManager uses spark.executor.cores and spark.task.cpus configuration properties for the number of tasks that can be submitted to an executor for full parallelism.
Used when:
maxNumExecutorsNeeded
","text":""},{"location":"dynamic-allocation/ExecutorAllocationManager/#maximum-number-of-executors-needed","title":"Maximum Number of Executors Needed
maxNumExecutorsNeeded(): Int\n
maxNumExecutorsNeeded requests the ExecutorAllocationListener for the number of pending and running tasks.
maxNumExecutorsNeeded is the smallest integer value that is greater than or equal to the multiplication of the total number of pending and running tasks by executorAllocationRatio divided by tasksPerExecutorForFullParallelism.
ExecutorMonitor uses a Java ConcurrentHashMap to track available executors.
An executor is added when (via ensureExecutorIsTracked):
onBlockUpdated
onExecutorAdded
onTaskStart
An executor is removed when onExecutorRemoved.
All executors are removed when reset.
executors is used when:
onOtherEvent (cleanupShuffle)
executorCount
executorsKilled
onUnpersistRDD
onTaskEnd
onJobStart
onJobEnd
pendingRemovalCount
timedOutExecutors
","text":""},{"location":"dynamic-allocation/ExecutorMonitor/#fetchfromshufflesvcenabled-flag","title":"fetchFromShuffleSvcEnabled Flag
fetchFromShuffleSvcEnabled: Boolean\n
ExecutorMonitor initializes fetchFromShuffleSvcEnabled internal flag based on the values of spark.shuffle.service.enabled and spark.shuffle.service.fetch.rdd.enabled configuration properties.
fetchFromShuffleSvcEnabled is enabled (true) when the aforementioned configuration properties are.
fetchFromShuffleSvcEnabled is used when:
onBlockUpdated
","text":""},{"location":"dynamic-allocation/ExecutorMonitor/#shuffletrackingenabled-flag","title":"shuffleTrackingEnabled Flag
shuffleTrackingEnabled: Boolean\n
ExecutorMonitor initializes shuffleTrackingEnabled internal flag based on the values of spark.shuffle.service.enabled and spark.dynamicAllocation.shuffleTracking.enabled configuration properties.
shuffleTrackingEnabled is enabled (true) when the following holds:
spark.shuffle.service.enabled is disabled
spark.dynamicAllocation.shuffleTracking.enabled is enabled
When enabled, shuffleTrackingEnabled is used to skip execution of the following (making them noops):
onJobStart
onJobEnd
When disabled, shuffleTrackingEnabled is used for the following:
Spark applications start one or more Executors for executing tasks.
By default (in Static Allocation of Executors) executors run for the entire lifetime of a Spark application (unlike in Dynamic Allocation).
Executors are managed by ExecutorBackend.
Executors reports heartbeat and partial metrics for active tasks to the HeartbeatReceiver RPC Endpoint on the driver.
Executors provide in-memory storage for RDDs that are cached in Spark applications (via BlockManager).
When started, an executor first registers itself with the driver that establishes a communication channel directly to the driver to accept tasks for execution.
Executor offers are described by executor id and the host on which an executor runs.
Executors can run multiple tasks over their lifetime, both in parallel and sequentially, and track running tasks.
Executors use an Executor task launch worker thread pool for launching tasks.
Executors send metrics (and heartbeats) using the Heartbeat Sender Thread.
When created, Executor prints out the following INFO messages to the logs:
Starting executor ID [executorId] on host [executorHostname]\n
(only for non-local modes) Executor sets SparkUncaughtExceptionHandler as the default handler invoked when a thread abruptly terminates due to an uncaught exception.
(only for non-local modes) Executor requests the BlockManager to initialize (with the Spark application id of the SparkConf).
(only for non-local modes) Executor requests the MetricsSystem to register the following metric sources:
ExecutorSource
JVMCPUSource
ExecutorMetricsSource
ShuffleMetricsSource (of the BlockManager)
Executor uses SparkEnv to access the MetricsSystem and BlockManager.
Executor creates a task class loader (optionally with REPL support) and requests the system Serializer to use as the default classloader (for deserializing tasks).
Executor starts sending heartbeats with the metrics of active tasks.
updateDependencies fetches missing or outdated extra files (in the given newFiles). For every name-timestamp pair that...FIXME..., updateDependencies prints out the following INFO message to the logs:
Fetching [name] with timestamp [timestamp]\n
updateDependencies fetches missing or outdated extra jars (in the given newJars). For every name-timestamp pair that...FIXME..., updateDependencies prints out the following INFO message to the logs:
Fetching [name] with timestamp [timestamp]\n
updateDependencies fetches the file to the SparkFiles root directory.
Executor uses the spark.driver.maxResultSize for TaskRunner when requested to run a task (and decide on a serialized task result).
","text":""},{"location":"executor/Executor/#maximum-size-of-direct-results","title":"Maximum Size of Direct Results
Executor uses the minimum of spark.task.maxDirectResultSize and spark.rpc.message.maxSize when TaskRunner is requested to run a task (and decide on the type of a serialized task result).
","text":""},{"location":"executor/Executor/#islocal-flag","title":"isLocal Flag
Executor is given the isLocal flag when created to indicate a non-local mode (whether the executor and the Spark application runs with local or cluster-specific master URL).
isLocal is disabled (false) by default and is off explicitly when CoarseGrainedExecutorBackend is requested to handle a RegisteredExecutor message.
isLocal is enabled (true) when LocalEndpoint is created
Executor is given user-defined jars when created. No jars are assumed by default.
The jars are specified using spark.executor.extraClassPath configuration property (via --user-class-path command-line option of CoarseGrainedExecutorBackend).
launchTask creates a TaskRunner (with the given ExecutorBackend, the TaskDescription and the PluginContainer) and adds it to the runningTasks internal registry.
launchTask requests the \"Executor task launch worker\" thread pool to execute the TaskRunner (sometime in the future).
In case the decommissioned flag is enabled, launchTask prints out the following ERROR message to the logs:
Launching a task while in decommissioned state.\n
launchTask is used when:
CoarseGrainedExecutorBackend is requested to handle a LaunchTask message
LocalEndpoint RPC endpoint (of LocalSchedulerBackend) is requested to reviveOffers
","text":""},{"location":"executor/Executor/#sending-heartbeats-and-active-tasks-metrics","title":"Sending Heartbeats and Active Tasks Metrics
Executors keep sending metrics for active tasks to the driver every spark.executor.heartbeatInterval (defaults to 10s with some random initial delay so the heartbeats from different executors do not pile up on the driver).
An executor sends heartbeats using the Heartbeat Sender Thread.
For each task in TaskRunner (in runningTasks internal registry), the task's metrics are computed and become part of the heartbeat (with accumulators).
A blocking Heartbeat message that holds the executor id, all accumulator updates (per task id), and BlockManagerId is sent to HeartbeatReceiver RPC endpoint.
If the response requests to re-register BlockManager, Executor prints out the following INFO message to the logs:
Told to re-register on heartbeat\n
BlockManager is requested to reregister.
The internal heartbeatFailures counter is reset.
If there are any issues with communicating with the driver, Executor prints out the following WARN message to the logs:
Issue communicating with driver in heartbeater\n
The internal heartbeatFailures is incremented and checked to be less than the spark.executor.heartbeat.maxFailures. If the number is greater, the following ERROR is printed out to the logs:
Exit as unable to send heartbeats to driver more than [HEARTBEAT_MAX_FAILURES] times\n
The executor exits (using System.exit and exit code 56).
The amount of memory per executor is configured using spark.executor.memory configuration property. It sets the available memory equally for all executors per application.
You can find the value displayed as Memory per Node in the web UI of the standalone Master.
","text":""},{"location":"executor/Executor/#heartbeating-with-partial-metrics-for-active-tasks-to-driver","title":"Heartbeating With Partial Metrics For Active Tasks To Driver
reportHeartBeat(): Unit\n
reportHeartBeat collects TaskRunners for currently running tasks (active tasks) with their tasks deserialized (i.e. either ready for execution or already started).
TaskRunner has task deserialized when it runs the task.
For every running task, reportHeartBeat takes the TaskMetrics and:
Requests ShuffleRead metrics to be merged
Sets jvmGCTime metrics
reportHeartBeat then records the latest values of internal and external accumulators for every task.
Note
Internal accumulators are a task's metrics while external accumulators are a Spark application's accumulators that a user has created.
reportHeartBeat sends a blocking Heartbeat message to the HeartbeatReceiver (on the driver). reportHeartBeat uses the value of spark.executor.heartbeatInterval configuration property for the RPC timeout.
Note
A Heartbeat message contains the executor identifier, the accumulator updates, and the identifier of the BlockManager.
If the response (from HeartbeatReceiver) is to re-register the BlockManager, reportHeartBeat prints out the following INFO message to the logs and requests the BlockManager to re-register (which will register the blocks the BlockManager manages with the driver).
Told to re-register on heartbeat\n
HeartbeatResponse requests the BlockManager to re-register when either TaskScheduler or HeartbeatReceiver know nothing about the executor.
When posting the Heartbeat was successful, reportHeartBeat resets heartbeatFailures internal counter.
In case of a non-fatal exception, you should see the following WARN message in the logs (followed by the stack trace).
Issue communicating with driver in heartbeater\n
Every failure reportHeartBeat increments heartbeat failures up to spark.executor.heartbeat.maxFailures configuration property. When the heartbeat failures reaches the maximum, reportHeartBeat prints out the following ERROR message to the logs and the executor terminates with the error code: 56.
Exit as unable to send heartbeats to driver more than [HEARTBEAT_MAX_FAILURES] times\n
reportHeartBeat is used when:
Executor is requested to schedule reporting heartbeat and partial metrics for active tasks to the driver (that happens every spark.executor.heartbeatInterval).
getCurrentMetrics gives metric values for every metric getter.
Given that one metric getter (type) can report multiple metrics, the length of the result collection is the number of metrics (and at least the number of metric getters). The order matters and is exactly as metricGetters.
Name Description threadpool.activeTasks Approximate number of threads that are actively executing tasks (based on ThreadPoolExecutor.getActiveCount) others","text":""},{"location":"executor/ShuffleReadMetrics/","title":"ShuffleReadMetrics","text":"
ShuffleReadMetrics is a collection of metrics (accumulators) on reading shuffle data.
val maxResultSize = sc.getConf.get(\"spark.driver.maxResultSize\")\nassert(maxResultSize == \"1m\")\n
val rddOver1m = sc.range(0, 1024 * 1024 + 10, 1)\n
scala> rddOver1m.collect\nERROR TaskSetManager: Total size of serialized results of 2 tasks (1030.8 KiB) is bigger than spark.driver.maxResultSize (1024.0 KiB)\nERROR TaskSetManager: Total size of serialized results of 3 tasks (1546.2 KiB) is bigger than spark.driver.maxResultSize (1024.0 KiB)\nERROR TaskSetManager: Total size of serialized results of 4 tasks (2.0 MiB) is bigger than spark.driver.maxResultSize (1024.0 KiB)\nWARN TaskSetManager: Lost task 7.0 in stage 0.0 (TID 7, 192.168.68.105, executor driver): TaskKilled (Tasks result size has exceeded maxResultSize)\nWARN TaskSetManager: Lost task 5.0 in stage 0.0 (TID 5, 192.168.68.105, executor driver): TaskKilled (Tasks result size has exceeded maxResultSize)\nWARN TaskSetManager: Lost task 12.0 in stage 0.0 (TID 12, 192.168.68.105, executor driver): TaskKilled (Tasks result size has exceeded maxResultSize)\nERROR TaskSetManager: Total size of serialized results of 5 tasks (2.5 MiB) is bigger than spark.driver.maxResultSize (1024.0 KiB)\nWARN TaskSetManager: Lost task 8.0 in stage 0.0 (TID 8, 192.168.68.105, executor driver): TaskKilled (Tasks result size has exceeded maxResultSize)\n...\norg.apache.spark.SparkException: Job aborted due to stage failure: Total size of serialized results of 2 tasks (1030.8 KiB) is bigger than spark.driver.maxResultSize (1024.0 KiB)\n at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2059)\n ...\n
","text":""},{"location":"executor/TaskRunner/#thread-name","title":"Thread Name
TaskRunner uses the following thread name (with the taskId of the TaskDescription):
run initializes the threadId internal registry as the current thread identifier (using Thread.getId).
run sets the name of the current thread of execution as the threadName.
run creates a TaskMemoryManager (for the current MemoryManager and taskId). run uses SparkEnv to access the current MemoryManager.
run starts tracking the time to deserialize a task and sets the current thread's context classloader.
run creates a closure Serializer. run uses SparkEnv to access the closure Serializer.
run prints out the following INFO message to the logs (with the taskName and taskId):
Running [taskName] (TID [taskId])\n
run notifies the ExecutorBackend that the status of the task has changed to RUNNING (for the taskId).
run computes the total amount of time this JVM process has spent in garbage collection.
run uses the addedFiles and addedJars (of the given TaskDescription) to update dependencies.
run takes the serializedTask of the given TaskDescription and requests the closure Serializer to deserialize the task. run sets the task internal reference to hold the deserialized task.
For non-local environments, run prints out the following DEBUG message to the logs before requesting the MapOutputTrackerWorker to update the epoch (using the epoch of the Task to be executed). run uses SparkEnv to access the MapOutputTrackerWorker.
Task [taskId]'s epoch is [epoch]\n
run requests the metricsPoller...FIXME
run records the current time as the task's start time (taskStartTimeNs).
run requests the Task to run (with taskAttemptId as taskId, attemptNumber from TaskDescription, and metricsSystem as the current MetricsSystem).
Note
run uses SparkEnv to access the MetricsSystem.
Note
The task runs inside a \"monitored\" block (try-finally block) to detect any memory and lock leaks after the task's run finishes regardless of the final outcome - the computed value or an exception thrown.
run creates a Serializer and requests it to serialize the task result (valueBytes).
Note
run uses SparkEnv to access the Serializer.
run updates the metrics of the Task executed.
run updates the metric counters in the ExecutorSource.
run requests the Task executed for accumulator updates and the ExecutorMetricsPoller for metric peaks.
","text":""},{"location":"executor/TaskRunner/#serialized-task-result","title":"Serialized Task Result
run creates a DirectTaskResult (with the serialized task result, the accumulator updates and the metric peaks) and requests the closure Serializer to serialize it.
Note
The serialized DirectTaskResult is a java.nio.ByteBuffer.
run selects between the DirectTaskResult and an IndirectTaskResult based on the size of the serialized task result (limit of this serializedDirectResult byte buffer):
With the size above spark.driver.maxResultSize, run prints out the following WARN message to the logs and serializes an IndirectTaskResult with a TaskResultBlockId.
Finished [taskName] (TID [taskId]). Result is larger than maxResultSize ([resultSize] > [maxResultSize]), dropping it.\n
With the size above maxDirectResultSize, run creates an TaskResultBlockId and requests the BlockManager to store the task result locally (with MEMORY_AND_DISK_SER). run prints out the following INFO message to the logs and serializes an IndirectTaskResult with a TaskResultBlockId.
Finished [taskName] (TID [taskId]). [resultSize] bytes result sent via BlockManager)\n
run prints out the following INFO message to the logs and uses the DirectTaskResult created earlier.
Finished [taskName] (TID [taskId]). [resultSize] bytes result sent to driver\n
Note
serializedResult is either a IndirectTaskResult (possibly with the block stored in BlockManager) or a DirectTaskResult.
When shuffle:FetchFailedException.md[FetchFailedException] is reported while running a task, run <>.
run shuffle:FetchFailedException.md#toTaskFailedReason[requests FetchFailedException for the TaskFailedReason], serializes it and ExecutorBackend.md#statusUpdate[notifies ExecutorBackend that the task has failed] (with <>, TaskState.FAILED, and a serialized reason).
NOTE: ExecutorBackend was specified when <>.
NOTE: run uses a closure serializer:Serializer.md[Serializer] to serialize the failure reason. The Serializer was created before run ran the task.
run then <> and ExecutorBackend.md#statusUpdate[notifies ExecutorBackend that the task has been killed] (with <>, TaskState.KILLED, and a serialized TaskKilled object).","text":""},{"location":"executor/TaskRunner/#interruptedexception-with-task-killed","title":"InterruptedException (with Task Killed)
When InterruptedException is reported while running a task, and the task has been killed, you should see the following INFO message in the logs:
Executor interrupted and killed [taskName] (TID [taskId]), reason: [killReason]\n
run then <> and ExecutorBackend.md#statusUpdate[notifies ExecutorBackend that the task has been killed] (with <>, TaskState.KILLED, and a serialized TaskKilled object).
NOTE: The difference between this InterruptedException and <> is the INFO message in the logs.","text":""},{"location":"executor/TaskRunner/#commitdeniedexception","title":"CommitDeniedException
When CommitDeniedException is reported while running a task, run <> and ExecutorBackend.md#statusUpdate[notifies ExecutorBackend that the task has failed] (with <>, TaskState.FAILED, and a serialized TaskKilled object).
NOTE: The difference between this CommitDeniedException and <> is just the reason being sent to ExecutorBackend.","text":""},{"location":"executor/TaskRunner/#throwable","title":"Throwable
When run catches a Throwable, you should see the following ERROR message in the logs (followed by the exception).
Exception in [taskName] (TID [taskId])\n
run then records the following task metrics (only when <> is available):
run then scheduler:Task.md#collectAccumulatorUpdates[collects the latest values of internal and external accumulators] (with taskFailed flag enabled to inform that the collection is for a failed task).
Otherwise, when <> is not available, the accumulator collection is empty.
run converts the task accumulators to collection of AccumulableInfo, creates a ExceptionFailure (with the accumulators), and serializer:Serializer.md#serialize[serializes them].
NOTE: run uses a closure serializer:Serializer.md[Serializer] to serialize the ExceptionFailure.
CAUTION: FIXME Why does run create new ExceptionFailure(t, accUpdates).withAccums(accums), i.e. accumulators occur twice in the object.
run <> and ExecutorBackend.md#statusUpdate[notifies ExecutorBackend that the task has failed] (with <>, TaskState.FAILED, and the serialized ExceptionFailure).
run may also trigger SparkUncaughtExceptionHandler.uncaughtException(t) if this is a fatal error.
NOTE: The difference between this most Throwable case and other FAILED cases (i.e. <> and <>) is just the serialized ExceptionFailure vs a reason being sent to ExecutorBackend, respectively.","text":""},{"location":"executor/TaskRunner/#collectaccumulatorsandresetstatusonfailure","title":"collectAccumulatorsAndResetStatusOnFailure
kill marks the TaskRunner as <> and scheduler:Task.md#kill[kills the task] (if available and not <> already).
NOTE: kill passes the input interruptThread on to the task itself while killing it.
When executed, you should see the following INFO message in the logs:
Executor is trying to kill [taskName] (TID [taskId]), reason: [reason]\n
NOTE: <> flag is checked periodically in <> to stop executing the task. Once killed, the task will eventually stop.","text":""},{"location":"executor/TaskRunner/#logging","title":"Logging
Enable ALL logging level for org.apache.spark.executor.Executor logger to see what happens inside.
","text":""},{"location":"executor/TaskRunner/#internal-properties","title":"Internal Properties","text":""},{"location":"executor/TaskRunner/#finished-flag","title":"finished Flag
finished flag says whether the <> has finished (true) or not (false)
Default: false
Enabled (true) after TaskRunner has been requested to <>
Used when TaskRunner is requested to <>","text":""},{"location":"executor/TaskRunner/#reasonifkilled","title":"reasonIfKilled
Timestamp (which is really the Executor.md#computeTotalGcTime[total amount of time this Executor JVM process has already spent in garbage collection]) that is used to mark the GC \"zero\" time (when <>) and then compute the JVM GC time metric when:
TaskRunner is requested to <> and <>
Executor is requested to Executor.md#reportHeartBeat[reportHeartBeat]
Executor is requested to Executor.md#reportHeartBeat[reportHeartBeat]
","text":""},{"location":"executor/TaskRunner/#task-name","title":"Task Name
The name of the task (of the TaskDescription) that is used exclusively for <> purposes when TaskRunner is requested to <> and <> the task","text":""},{"location":"executor/TaskRunner/#thread-id","title":"Thread Id
Current thread ID
Default: -1
Set immediately when TaskRunner is requested to <> and used exclusively when TaskReaper is requested for the thread info of the current thread (aka thread dump)","text":""},{"location":"exercises/spark-examples-wordcount-spark-shell/","title":"WordCount using Spark shell","text":"
== WordCount using Spark shell
It is like any introductory big data example should somehow demonstrate how to count words in distributed fashion.
In the following example you're going to count the words in README.md file that sits in your Spark distribution and save the result under README.count directory.
You're going to use spark-shell.md[the Spark shell] for the example. Execute spark-shell.
<1> Read the text file - refer to spark-io.md[Using Input and Output (I/O)]. <2> Split each line into words and flatten the result. <3> Map each word into a pair and count them by word (key). <4> Save the result into text files - one per partition.
After you have executed the example, see the contents of the README.count directory:
$ ls -lt README.count\ntotal 16\n-rw-r--r-- 1 jacek staff 0 9 pa\u017a 13:36 _SUCCESS\n-rw-r--r-- 1 jacek staff 1963 9 pa\u017a 13:36 part-00000\n-rw-r--r-- 1 jacek staff 1663 9 pa\u017a 13:36 part-00001\n
The files part-0000x contain the pairs of word and the count.
Please read the questions and give answers first before looking at the link given.
Why are there two files under the directory?
How could you have only one?
How to filter out words by name?
How to count words?
Please refer to the chapter spark-rdd-partitions.md[Partitions] to find some of the answers.
"},{"location":"exercises/spark-exercise-custom-scheduler-listener/","title":"Developing Custom SparkListener to monitor DAGScheduler in Scala","text":"
== Exercise: Developing Custom SparkListener to monitor DAGScheduler in Scala
The example shows how to develop a custom Spark Listener. You should read SparkListener.md[] first to understand the motivation for the example.
=== Requirements
https://www.jetbrains.com/idea/[IntelliJ IDEA] (or eventually http://www.scala-sbt.org/[sbt] alone if you're adventurous).
Access to Internet to download Apache Spark's dependencies.
=== Setting up Scala project using IntelliJ IDEA
Create a new project custom-spark-listener.
Add the following line to build.sbt (the main configuration file for the sbt project) that adds the dependency on Apache Spark.
Create a Scala class -- CustomSparkListener -- for your custom SparkListener. It should be under src/main/scala directory (create one if it does not exist).
The aim of the class is to intercept scheduler events about jobs being started and tasks completed.
class CustomSparkListener extends SparkListener { override def onJobStart(jobStart: SparkListenerJobStart) { println(s\"Job started with ${jobStart.stageInfos.size} stages: $jobStart\") }
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { println(s\"Stage ${stageCompleted.stageInfo.stageId} completed with ${stageCompleted.stageInfo.numTasks} tasks.\") } }
=== Creating deployable package
Package the custom Spark listener. Execute sbt package command in the custom-spark-listener project's main directory.
$ sbt package\n[info] Loading global plugins from /Users/jacek/.sbt/0.13/plugins\n[info] Loading project definition from /Users/jacek/dev/workshops/spark-workshop/solutions/custom-spark-listener/project\n[info] Updating {file:/Users/jacek/dev/workshops/spark-workshop/solutions/custom-spark-listener/project/}custom-spark-listener-build...\n[info] Resolving org.fusesource.jansi#jansi;1.4 ...\n[info] Done updating.\n[info] Set current project to custom-spark-listener (in build file:/Users/jacek/dev/workshops/spark-workshop/solutions/custom-spark-listener/)\n[info] Updating {file:/Users/jacek/dev/workshops/spark-workshop/solutions/custom-spark-listener/}custom-spark-listener...\n[info] Resolving jline#jline;2.12.1 ...\n[info] Done updating.\n[info] Compiling 1 Scala source to /Users/jacek/dev/workshops/spark-workshop/solutions/custom-spark-listener/target/scala-2.11/classes...\n[info] Packaging /Users/jacek/dev/workshops/spark-workshop/solutions/custom-spark-listener/target/scala-2.11/custom-spark-listener_2.11-1.0.jar ...\n[info] Done packaging.\n[success] Total time: 8 s, completed Oct 27, 2016 11:23:50 AM\n
You should find the result jar file with the custom scheduler listener ready under target/scala-2.11 directory, e.g. target/scala-2.11/custom-spark-listener_2.11-1.0.jar.
=== Activating Custom Listener in Spark shell
Start ../spark-shell.md[spark-shell] with additional configurations for the extra custom listener and the jar that includes the class.
What are the pros and cons of using the command line version vs inside a Spark application?
"},{"location":"exercises/spark-exercise-dataframe-jdbc-postgresql/","title":"Working with Datasets from JDBC Data Sources (and PostgreSQL)","text":"
== Working with Datasets from JDBC Data Sources (and PostgreSQL)
Start spark-shell with the JDBC driver for the database you want to use. In our case, it is PostgreSQL JDBC Driver.
NOTE: Download the jar for PostgreSQL JDBC Driver 42.1.1 directly from the http://central.maven.org/maven2/org/postgresql/postgresql/42.1.1/postgresql-42.1.1.jar[Maven repository].
// Note the number of partition (aka numPartitions) scala> df.explain == Physical Plan == *Scan JDBCRelation(projects) [numPartitions=1] [id#0,name#1,website#2] ReadSchema: struct
// use jdbc method with predicates to define partitions import java.util.Properties val df4parts = spark. read. jdbc( url = \"jdbc:postgresql:sparkdb\", table = \"projects\", predicates = Array(\"id=1\", \"id=2\", \"id=3\", \"id=4\"), connectionProperties = new Properties())
// Note the number of partitions (aka numPartitions) scala> df4parts.explain == Physical Plan == *Scan JDBCRelation(projects) [numPartitions=4] [id#16,name#17,website#18] ReadSchema: struct
If things can go wrong, they sooner or later go wrong. Here is a list of possible issues and their solutions.
==== java.sql.SQLException: No suitable driver
Ensure that the JDBC driver sits on the CLASSPATH. Use spark-submit/index.md#driver-class-path[--driver-class-path] as described above (--packages or --jars do not work).
scala> val df = spark.\n | read.\n | format(\"jdbc\").\n | options(opts).\n | load\njava.sql.SQLException: No suitable driver\n at java.sql.DriverManager.getDriver(DriverManager.java:315)\n at org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions$$anonfun$7.apply(JDBCOptions.scala:84)\n at org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions$$anonfun$7.apply(JDBCOptions.scala:84)\n at scala.Option.getOrElse(Option.scala:121)\n at org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions.<init>(JDBCOptions.scala:83)\n at org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions.<init>(JDBCOptions.scala:34)\n at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:32)\n at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:301)\n at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:190)\n at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:158)\n ... 52 elided\n
=== PostgreSQL Setup
NOTE: I'm on Mac OS X so YMMV (aka Your Mileage May Vary).
Use the sections to have a properly configured PostgreSQL database.
<>
<>
<>
<>
<>
<>
<>
==== [[installation]] Installation
Install PostgreSQL as described in...TK
CAUTION: This page serves as a cheatsheet for the author so he does not have to search Internet to find the installation steps.
$ initdb /usr/local/var/postgres -E utf8\nThe files belonging to this database system will be owned by user \"jacek\".\nThis user must also own the server process.\n\nThe database cluster will be initialized with locale \"pl_pl.utf-8\".\ninitdb: could not find suitable text search configuration for locale \"pl_pl.utf-8\"\nThe default text search configuration will be set to \"simple\".\n\nData page checksums are disabled.\n\ncreating directory /usr/local/var/postgres ... ok\ncreating subdirectories ... ok\nselecting default max_connections ... 100\nselecting default shared_buffers ... 128MB\nselecting dynamic shared memory implementation ... posix\ncreating configuration files ... ok\ncreating template1 database in /usr/local/var/postgres/base/1 ... ok\ninitializing pg_authid ... ok\ninitializing dependencies ... ok\ncreating system views ... ok\nloading system objects' descriptions ... ok\ncreating collations ... ok\ncreating conversions ... ok\ncreating dictionaries ... ok\nsetting privileges on built-in objects ... ok\ncreating information schema ... ok\nloading PL/pgSQL server-side language ... ok\nvacuuming database template1 ... ok\ncopying template1 to template0 ... ok\ncopying template1 to postgres ... ok\nsyncing data to disk ... ok\n\nWARNING: enabling \"trust\" authentication for local connections\nYou can change this by editing pg_hba.conf or using the option -A, or\n--auth-local and --auth-host, the next time you run initdb.\n\nSuccess. You can now start the database server using:\n\n pg_ctl -D /usr/local/var/postgres -l logfile start\n
==== [[starting-database-server]] Starting Database Server
NOTE: Consult http://www.postgresql.org/docs/current/static/server-start.html[17.3. Starting the Database Server] in the official documentation.
Enable all logs in PostgreSQL to see query statements.
log_statement = 'all'\n
"},{"location":"exercises/spark-exercise-dataframe-jdbc-postgresql/#add-log_statement-all-to-usrlocalvarpostgrespostgresqlconf-on-mac-os-x-with-postgresql-installed-using-brew","title":"Add log_statement = 'all' to /usr/local/var/postgres/postgresql.conf on Mac OS X with PostgreSQL installed using brew.","text":"
Alternatively, you can run the database server using postgres.
$ postgres -D /usr/local/var/postgres\n
==== [[creating-database]] Create Database
$ createdb sparkdb\n
TIP: Consult http://www.postgresql.org/docs/current/static/app-createdb.html[createdb] in the official documentation.
==== Accessing Database
Use psql sparkdb to access the database.
$ psql sparkdb\npsql (9.6.2)\nType \"help\" for help.\n\nsparkdb=#\n
Execute SELECT version() to know the version of the database server you have connected to.
sparkdb=# SELECT version();\n version\n--------------------------------------------------------------------------------------------------------------\n PostgreSQL 9.6.2 on x86_64-apple-darwin14.5.0, compiled by Apple LLVM version 7.0.2 (clang-700.1.81), 64-bit\n(1 row)\n
Use \\h for help and \\q to leave a session.
==== Creating Table
Create a table using CREATE TABLE command.
CREATE TABLE projects (\n id SERIAL PRIMARY KEY,\n name text,\n website text\n);\n
Insert rows to initialize the table with data.
INSERT INTO projects (name, website) VALUES ('Apache Spark', 'http://spark.apache.org');\nINSERT INTO projects (name, website) VALUES ('Apache Hive', 'http://hive.apache.org');\nINSERT INTO projects VALUES (DEFAULT, 'Apache Kafka', 'http://kafka.apache.org');\nINSERT INTO projects VALUES (DEFAULT, 'Apache Flink', 'http://flink.apache.org');\n
Execute select * from projects; to ensure that you have the following records in projects table:
TIP: Consult http://www.postgresql.org/docs/current/static/app-dropdb.html[dropdb] in the official documentation.
==== Stopping Database Server
pg_ctl -D /usr/local/var/postgres stop\n
"},{"location":"exercises/spark-exercise-failing-stage/","title":"Causing Stage to Fail","text":"
== Exercise: Causing Stage to Fail
The example shows how Spark re-executes a stage in case of stage failure.
=== Recipe
Start a Spark cluster, e.g. 1-node Hadoop YARN.
start-yarn.sh\n
// 2-stage job -- it _appears_ that a stage can be failed only when there is a shuffle\nsc.parallelize(0 to 3e3.toInt, 2).map(n => (n % 2, n)).groupByKey.count\n
Use 2 executors at least so you can kill one and keep the application up and running (on one executor).
NOTE: It is not possible to start another instance of standalone Master on the same machine using ./sbin/start-master.sh. The reason is that the script assumes one instance per machine only. We're going to change the script to make it possible.
Wait till the Spark shell connects to an active standalone Master.
Find out which standalone Master is active (there can only be one). Kill it. Observe how the other standalone Master takes over and lets the Spark shell register with itself. Check out the master's UI.
Optionally, kill the worker, make sure it goes away instantly in the active master's logs.
"},{"location":"exercises/spark-exercise-take-multiple-jobs/","title":"Learning Jobs and Partitions Using take Action","text":"
== Exercise: Learning Jobs and Partitions Using take Action
The exercise aims for introducing take action and using spark-shell and web UI. It should introduce you to the concepts of partitions and jobs.
The following snippet creates an RDD of 16 elements with 16 partitions.
scala> val r1 = sc.parallelize(0 to 15, 16)\nr1: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[26] at parallelize at <console>:18\n\nscala> r1.partitions.size\nres63: Int = 16\n\nscala> r1.foreachPartition(it => println(\">>> partition size: \" + it.size))\n...\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n... // the machine has 8 cores\n... // so first 8 tasks get executed immediately\n... // with the others after a core is free to take on new tasks.\n>>> partition size: 1\n...\n>>> partition size: 1\n...\n>>> partition size: 1\n...\n>>> partition size: 1\n>>> partition size: 1\n...\n>>> partition size: 1\n>>> partition size: 1\n>>> partition size: 1\n
All 16 partitions have one element.
When you execute r1.take(1) only one job gets run since it is enough to compute one task on one partition.
CAUTION: FIXME Snapshot from web UI - note the number of tasks
However, when you execute r1.take(2) two jobs get run as the implementation assumes one job with one partition, and if the elements didn't total to the number of elements requested in take, quadruple the partitions to work on in the following jobs.
CAUTION: FIXME Snapshot from web UI - note the number of tasks
Can you guess how many jobs are run for r1.take(15)? How many tasks per job?
CAUTION: FIXME Snapshot from web UI - note the number of tasks
Answer: 3.
"},{"location":"exercises/spark-first-app/","title":"Your first complete Spark application (using Scala and sbt)","text":"
== Your first Spark application (using Scala and sbt)
This page gives you the exact steps to develop and run a complete Spark application using http://www.scala-lang.org/[Scala] programming language and http://www.scala-sbt.org/[sbt] as the build tool.
[TIP] Refer to Quick Start's http://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/quick-start.html#self-contained-applications[Self-Contained Applications] in the official documentation.
The sample application called SparkMe App is...FIXME
=== Overview
You're going to use http://www.scala-sbt.org/[sbt] as the project build tool. It uses build.sbt for the project's description as well as the dependencies, i.e. the version of Apache Spark and others.
The application's main code is under src/main/scala directory, in SparkMeApp.scala file.
With the files in a directory, executing sbt package results in a package that can be deployed onto a Spark cluster using spark-submit.
In this example, you're going to use Spark's local/spark-local.md[local mode].
=== Project's build - build.sbt
Any Scala project managed by sbt uses build.sbt as the central place for configuration, including project dependencies denoted as libraryDependencies.
<1> Use the development version of Spark 1.6.0-SNAPSHOT
=== SparkMe Application
The application uses a single command-line parameter (as args(0)) that is the file to process. The file is read and the number of lines printed out.
package pl.japila.spark\n\nimport org.apache.spark.{SparkContext, SparkConf}\n\nobject SparkMeApp {\n def main(args: Array[String]) {\n val conf = new SparkConf().setAppName(\"SparkMe Application\")\n val sc = new SparkContext(conf)\n\n val fileName = args(0)\n val lines = sc.textFile(fileName).cache\n\n val c = lines.count\n println(s\"There are $c lines in $fileName\")\n }\n}\n
=== sbt version - project/build.properties
sbt (launcher) uses project/build.properties file to set (the real) sbt up
sbt.version=0.13.9\n
TIP: With the file the build is more predictable as the version of sbt doesn't depend on the sbt launcher.
=== Packaging Application
Execute sbt package to package the application.
\u279c sparkme-app sbt package\n[info] Loading global plugins from /Users/jacek/.sbt/0.13/plugins\n[info] Loading project definition from /Users/jacek/dev/sandbox/sparkme-app/project\n[info] Set current project to SparkMe Project (in build file:/Users/jacek/dev/sandbox/sparkme-app/)\n[info] Compiling 1 Scala source to /Users/jacek/dev/sandbox/sparkme-app/target/scala-2.11/classes...\n[info] Packaging /Users/jacek/dev/sandbox/sparkme-app/target/scala-2.11/sparkme-project_2.11-1.0.jar ...\n[info] Done packaging.\n[success] Total time: 3 s, completed Sep 23, 2015 12:47:52 AM\n
The application uses only classes that comes with Spark so package is enough.
In target/scala-2.11/sparkme-project_2.11-1.0.jar there is the final application ready for deployment.
=== Submitting Application to Spark (local)
NOTE: The application is going to be deployed to local[*]. Change it to whatever cluster you have available (refer to spark-cluster.md[Running Spark in cluster]).
spark-submit the SparkMe application and specify the file to process (as it is the only and required input parameter to the application), e.g. build.sbt of the project.
NOTE: build.sbt is sbt's build definition and is only used as an input file for demonstration purposes. Any file is going to work fine.
\u279c sparkme-app ~/dev/oss/spark/bin/spark-submit --master \"local[*]\" --class pl.japila.spark.SparkMeApp target/scala-2.11/sparkme-project_2.11-1.0.jar build.sbt\nUsing Spark's repl log4j profile: org/apache/spark/log4j-defaults-repl.properties\nTo adjust logging level use sc.setLogLevel(\"INFO\")\n15/09/23 01:06:02 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable\n15/09/23 01:06:04 WARN MetricsSystem: Using default name DAGScheduler for source because spark.app.id is not set.\nThere are 8 lines in build.sbt\n
NOTE: Disregard the two above WARN log messages.
You're done. Sincere congratulations!
"},{"location":"exercises/spark-hello-world-using-spark-shell/","title":"Spark's Hello World using Spark shell and Scala","text":"
== Exercise: Spark's Hello World using Spark shell and Scala
Run Spark shell and count the number of words in a file using MapReduce pattern.
Use sc.textFile to read the file into memory
Use RDD.flatMap for a mapper step
Use reduceByKey for a reducer step
"},{"location":"exercises/spark-sql-hive-orc-example/","title":"Using Spark SQL to update data in Hive using ORC files","text":"
== Using Spark SQL to update data in Hive using ORC files
The example has showed up on Spark's users mailing list.
FIXME Load ORC files into dataframe ** val df = hiveContext.read.format(\"orc\").load(to/path) ====
Solution was to use Hive in ORC format with partitions:
A table in Hive stored as an ORC file (using partitioning)
Using SQLContext.sql to insert data into the table
Using SQLContext.sql to periodically run ALTER TABLE...CONCATENATE to merge your many small files into larger files optimized for your HDFS block size ** Since the CONCATENATE command operates on files in place it is transparent to any downstream processing
Hive solution is just to concatenate the files ** it does not alter or change records. ** it's possible to update data in Hive using ORC format ** With transactional tables in Hive together with insert, update, delete, it does the \"concatenate \" for you automatically in regularly intervals. Currently this works only with tables in orc.format (stored as orc) ** Alternatively, use Hbase with Phoenix as the SQL layer on top ** Hive was originally not designed for updates, because it was.purely warehouse focused, the most recent one can do updates, deletes etc in a transactional way.
Criteria:
spark-streaming/spark-streaming.md[Spark Streaming] jobs are receiving a lot of small events (avg 10kb)
Events are stored to HDFS, e.g. for Pig jobs
There are a lot of small files in HDFS (several millions)
External Shuffle Service is a Spark service to serve RDD and shuffle blocks outside and for Executors.
ExternalShuffleService can be started as a command-line application or automatically as part of a worker node in a Spark cluster (e.g. Spark Standalone).
External Shuffle Service is enabled in a Spark application using spark.shuffle.service.enabled configuration property.
ExternalShuffleBlockResolver uses spark.shuffle.service.fetch.rdd.enabled configuration property to control whether or not to remove cached RDD files (alongside shuffle output files).
ExternalBlockHandler is requested to handle a RegisterExecutor message and reregisterExecutor
","text":""},{"location":"external-shuffle-service/ExternalShuffleBlockResolver/#cleaning-up-local-directories-for-removed-executor","title":"Cleaning Up Local Directories for Removed Executor
executorRemoved prints out the following INFO message to the logs:
Clean up non-shuffle and non-RDD files associated with the finished executor [executorId]\n
executorRemoved looks up the executor in the executors internal registry.
When found, executorRemoved prints out the following INFO message to the logs and requests the Directory Cleaner Executor to execute asynchronous deletion of the executor's local directories (on a separate thread).
Cleaning up non-shuffle and non-RDD files in executor [AppExecId]'s [localDirs] local dirs\n
When not found, executorRemoved prints out the following INFO message to the logs:
Executor is not registered (appId=[appId], execId=[executorId])\n
executorRemoved\u00a0is used when:
ExternalBlockHandler is requested to executorRemoved
ExternalShuffleService is a Spark service that can serve RDD and shuffle blocks.
ExternalShuffleService manages shuffle output files so they are available to executors. As the shuffle output files are managed externally to the executors it offers an uninterrupted access to the shuffle output files regardless of executors being killed or down (esp. with Dynamic Allocation of Executors).
ExternalShuffleService can be launched from command line.
ExternalShuffleService is enabled on the driver and executors using spark.shuffle.service.enabled configuration property.
Note
Spark on YARN uses a custom external shuffle service (YarnShuffleService).
","text":""},{"location":"external-shuffle-service/ExternalShuffleService/#main-entry-point","title":"main Entry Point
main(\n args: Array[String]): Unit\n
main is the entry point of ExternalShuffleService standalone application.
main prints out the following INFO message to the logs:
Started daemon with process name: [name]\n
main registers signal handlers for TERM, HUP, INT signals.
main loads the default Spark properties.
main creates a SecurityManager.
main turns spark.shuffle.service.enabled to true explicitly (since this service is started from the command line for a reason).
main creates an ExternalShuffleService and starts it.
main prints out the following DEBUG message to the logs:
Adding shutdown hook\n
main registers a shutdown hook. When triggered, the shutdown hook prints the following INFO message to the logs and requests the ExternalShuffleService to stop.
ExternalShuffleService uses spark.shuffle.service.enabled configuration property to control whether or not is enabled (and should be started when requested).
ExternalShuffleService creates an ExternalBlockHandler when created.
With spark.shuffle.service.db.enabled and spark.shuffle.service.enabled configuration properties enabled, the ExternalBlockHandler is given a local directory with a registeredExecutors.ldb file.
blockHandler\u00a0is used to create a TransportContext that creates the TransportServer.
findRegisteredExecutorsDBFile returns one of the local directories (defined using spark.local.dir configuration property) with the input dbName file or null when no directories defined.
findRegisteredExecutorsDBFile searches the local directories (defined using spark.local.dir configuration property) for the input dbName file. Unless found, findRegisteredExecutorsDBFile takes the first local directory.
With no local directories defined in spark.local.dir configuration property, findRegisteredExecutorsDBFile prints out the following WARN message to the logs and returns null.
'spark.local.dir' should be set first when we use db in ExternalShuffleService. Note that this only affects standalone mode.\n
Enables ExternalShuffleService for fetching disk persisted RDD blocks.
When enabled with Dynamic Resource Allocation executors having only disk persisted blocks are considered idle after spark.dynamicAllocation.executorIdleTimeout and will be released accordingly.
Default: false
Used when:
ExternalShuffleBlockResolver is created
SparkEnv utility is requested to create a \"base\" SparkEnv
StorageUtils utility is requested for the port of an external shuffle service
","text":""},{"location":"features/","title":"Features","text":""},{"location":"history-server/","title":"Spark History Server","text":"
Spark History Server is the web UI of Spark applications with event log collection enabled (based on spark.eventLog.enabled configuration property).
Spark History Server is an extension of Spark's web UI.
Spark History Server can be started using start-history-server.sh and stopped using stop-history-server.sh shell scripts.
Spark History Server supports custom configuration properties that can be defined using --properties-file [propertiesFile] command-line option. The properties file can have any valid spark.-prefixed Spark property.
$SPARK_HOME/sbin/start-history-server.sh shell script (where SPARK_HOME is the directory of your Spark installation) is used to start a Spark History Server instance.
$ ./sbin/start-history-server.sh\nstarting org.apache.spark.deploy.history.HistoryServer, logging to .../spark/logs/spark-jacek-org.apache.spark.deploy.history.HistoryServer-1-japila.out\n
Using the more explicit approach with spark-class to start Spark History Server could be easier to trace execution by seeing the logs printed out to the standard output and hence terminal directly.
When started, start-history-server.sh prints out the following INFO message to the logs:
Started daemon with process name: [processName]\n
start-history-server.sh registers signal handlers (using SignalUtils) for TERM, HUP, INT to log their execution:
RECEIVED SIGNAL [signal]\n
start-history-server.sh inits security if enabled (based on spark.history.kerberos.enabled configuration property).
start-history-server.sh creates a SecurityManager.
start-history-server.sh creates a ApplicationHistoryProvider (based on spark.history.provider configuration property).
In the end, start-history-server.sh creates a HistoryServer and requests it to bind to the port (based on spark.history.ui.port configuration property).
Note
The host's IP can be specified using SPARK_LOCAL_IP environment variable (defaults to 0.0.0.0).
start-history-server.sh prints out the following INFO message to the logs:
Bound HistoryServer to [host], and started at [webUrl]\n
start-history-server.sh registers a shutdown hook to call stop on the HistoryServer instance.
$SPARK_HOME/sbin/stop-history-server.sh shell script (where SPARK_HOME is the directory of your Spark installation) is used to stop a running instance of Spark History Server.
ApplicationCache is <> exclusively when HistoryServer is HistoryServer.md#appCache[created].
ApplicationCache uses https://github.com/google/guava/wiki/Release14[Google Guava 14.0.1] library for the internal <>.
[[internal-registries]] .ApplicationCache's Internal Properties (e.g. Registries, Counters and Flags) [cols=\"1,2\",options=\"header\",width=\"100%\"] |=== | Name | Description
| appLoader | [[appLoader]] Google Guava's https://google.github.io/guava/releases/14.0/api/docs/com/google/common/cache/CacheLoader.html[CacheLoader] with a custom ++https://google.github.io/guava/releases/14.0/api/docs/com/google/common/cache/CacheLoader.html#load(K)++[load] which is simply <>.
Used when...FIXME
| removalListener | [[removalListener]]
| appCache a| [[appCache]] Google Guava's https://google.github.io/guava/releases/14.0/api/docs/com/google/common/cache/LoadingCache.html[LoadingCache] of CacheKey keys and CacheEntry entries
Used when ApplicationCache is requested for the following:
NOTE: load is part of Google Guava's https://google.github.io/guava/releases/14.0/api/docs/com/google/common/cache/CacheLoader.html[CacheLoader] to retrieve a CacheEntry, based on a CacheKey, for <>.
load simply relays to <> with the appId and attemptId of the input CacheKey.
=== [[get]] Requesting Cached UI of Spark Application (CacheEntry) -- get Method
| getAppUI | [[getAppUI]] spark-webui-SparkUI.md[SparkUI] (the UI of a Spark application)
Used exclusively when ApplicationCache is requested for ApplicationCache.md#loadApplicationEntry[loadApplicationEntry]
| attachSparkUI | [[attachSparkUI]]
| detachSparkUI | [[detachSparkUI]] |===
[[implementations]] NOTE: HistoryServer.md[HistoryServer] is the one and only known implementation of <> in Apache Spark."},{"location":"history-server/ApplicationHistoryProvider/","title":"ApplicationHistoryProvider","text":"
ApplicationHistoryProvider is an abstraction of history providers.
EventLoggingListener is a SparkListener that writes out JSON-encoded events of a Spark application with event logging enabled (based on spark.eventLog.enabled configuration property).
FsHistoryProvider takes the following to be created:
SparkConf
Clock (default: SystemClock)
FsHistoryProvider is created\u00a0when HistoryServer standalone application is started (and no spark.history.provider configuration property was defined).
"},{"location":"history-server/FsHistoryProvider/#path-of-application-history-cache","title":"Path of Application History Cache
storePath: Option[File]\n
FsHistoryProvider uses spark.history.store.path configuration property for the directory to cache application history.
With storePath defined, FsHistoryProvider uses a LevelDB as the KVStore. Otherwise, a InMemoryStore.
With storePath defined, FsHistoryProvider uses a HistoryServerDiskManager as the disk manager.
HistoryServer is an extension of the web UI for reviewing event logs of running (active) and completed Spark applications with event log collection enabled (based on spark.eventLog.enabled configuration property).
This is however deprecated since Spark 1.1.0 and you should see the following WARN message in the logs:
WARN HistoryServerArguments: Setting log directory through the command line is deprecated as of Spark 1.1.0. Please set this through spark.history.fs.logDirectory instead.\n
The same WARN message shows up for --dir and -d command-line options.
--properties-file [propertiesFile] command-line option specifies the file with the custom spark-properties.md[Spark properties].
NOTE: When not specified explicitly, History Server uses the default configuration file, i.e. spark-properties.md#spark-defaults-conf[spark-defaults.conf].
"},{"location":"history-server/HistoryServerArguments/#refer-to-spark-loggingmdlogging","title":"Refer to spark-logging.md[Logging].","text":""},{"location":"history-server/HistoryServerDiskManager/","title":"HistoryServerDiskManager","text":"
HistoryServerDiskManager is a disk manager for FsHistoryProvider.
replay reads JSON-encoded SparkListener.md#SparkListenerEvent[SparkListenerEvent] events from logData (one event per line) and posts them to all registered SparkListenerInterfaces.
replay uses spark-history-server:JsonProtocol.md#sparkEventFromJson[JsonProtocol to convert JSON-encoded events to SparkListenerEvent objects].
NOTE: replay uses jackson from http://json4s.org/[json4s] library to parse the AST for JSON.
When there is an exception parsing a JSON event, you may see the following WARN message in the logs (for the last line) or a JsonParseException.
WARN Got JsonParseException from log file $sourceName at line [lineNumber], the file might not have finished writing cleanly.\n
Any other non-IO exceptions end up with the following ERROR messages in the logs:
SQLHistoryListener is a custom spark-sql-SQLListener.md[SQLListener] for index.md[History Server]. It attaches spark-sql-webui.md#creating-instance[SQL tab] to History Server's web UI only when the first spark-sql-SQLListener.md#SparkListenerSQLExecutionStart[SparkListenerSQLExecutionStart] arrives and shuts <> off. It also handles <>.
NOTE: Support for SQL UI in History Server was added in SPARK-11206 Support SQL UI on the history server.
When SparkListenerSQLExecutionStart event comes, onOtherEvent attaches spark-sql-webui.md#creating-instance[SQL tab] to web UI and passes the call to the parent spark-sql-SQLListener.md[SQLListener].
SQLHistoryListener is created using a (private[sql]) SQLHistoryListenerFactory class (which is SparkHistoryListenerFactory).
The SQLHistoryListenerFactory class is registered when spark-webui-SparkUI.md#createHistoryUI[SparkUI creates a web UI for History Server] as a Java service in META-INF/services/org.apache.spark.scheduler.SparkHistoryListenerFactory:
NOTE: Loading the service uses Java's https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html#load-java.lang.Class-java.lang.ClassLoader-[ServiceLoader.load] method.
The codec used to compress event log (with spark.eventLog.compress enabled). By default, Spark provides four codecs: lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify the codec.
Spark local is one of the available runtime environments in Apache Spark. It is the only available runtime with no need for a proper cluster manager (and hence many call it a pseudo-cluster, however such concept do exist in Spark and is a bit different).
Spark local is used for the following master URLs (as specified using <<../SparkConf.md#, SparkConf.setMaster>> method or <<../configuration-properties.md#spark.master, spark.master>> configuration property):
local (with exactly 1 CPU core)
local[n] (with exactly n CPU cores)
++local[]++* (with the total number of CPU cores that is the number of available CPU cores on the local machine)
local[n, m] (with exactly n CPU cores and m retries when a task fails)
++local[, m]++* (with the total number of CPU cores that is the number of available CPU cores on the local machine)
Internally, Spark local uses <> as the <<../SchedulerBackend.md#, SchedulerBackend>> and executor:ExecutorBackend.md[].
.Architecture of Spark local image::../diagrams/spark-local-architecture.png[align=\"center\"]
In this non-distributed multi-threaded runtime environment, Spark spawns all the main execution components - the spark-driver.md[driver] and an executor:Executor.md[] - in the same single JVM.
The default parallelism is the number of threads as specified in the <>. This is the only mode where a driver is used for execution (as it acts both as the driver and the only executor).
The local mode is very convenient for testing, debugging or demonstration purposes as it requires no earlier setup to launch Spark applications.
This mode of operation is also called http://spark.apache.org/docs/latest/programming-guide.html#initializing-spark[Spark in-process] or (less commonly) a local version of Spark.
SparkContext.isLocal returns true when Spark runs in local mode.
scala> sc.isLocal\nres0: Boolean = true\n
Spark shell defaults to local mode with local[*] as the the master URL.
scala> sc.master\nres0: String = local[*]\n
Tasks are not re-executed on failure in local mode (unless <> is used).
The scheduler:TaskScheduler.md[task scheduler] in local mode works with local/spark-LocalSchedulerBackend.md[LocalSchedulerBackend] task scheduler backend.
You can run Spark in local mode using local, local[n] or the most general local[*] for the master URL.
The URL says how many threads can be used in total:
local uses 1 thread only.
local[n] uses n threads.
local[*] uses as many threads as the number of processors available to the Java virtual machine (it uses https://docs.oracle.com/javase/8/docs/api/java/lang/Runtime.html#availableProcessors--[Runtime.getRuntime.availableProcessors()] to know the number).
NOTE: What happens when there are less cores than n in local[n] master URL? \"Breaks\" scheduling as Spark assumes more CPU cores available to execute tasks.
[[local-with-retries]] local[N, maxFailures] (called local-with-retries) with N being * or the number of threads to use (as explained above) and maxFailures being the value of <<../configuration-properties.md#spark.task.maxFailures, spark.task.maxFailures>> configuration property.
.TaskSchedulerImpl.submitTasks in local mode image::taskscheduler-submitTasks-local-mode.png[align=\"center\"]
When ReviveOffers or StatusUpdate messages are received, local/spark-LocalEndpoint.md[LocalEndpoint] places an offer to TaskSchedulerImpl (using TaskSchedulerImpl.resourceOffers).
If there is one or more tasks that match the offer, they are launched (using executor.launchTask method).
The number of tasks to be launched is controlled by the number of threads as specified in <>. The executor uses threads to spawn the tasks."},{"location":"local/LauncherBackend/","title":"LauncherBackend","text":"
Used exclusively when LauncherBackend is requested to <> (to access configuration-properties.md#spark.launcher.port[spark.launcher.port] and configuration-properties.md#spark.launcher.secret[spark.launcher.secret] configuration properties)
Spark Standalone's StandaloneSchedulerBackend is requested to <> (in client deploy mode)
Spark local's LocalSchedulerBackend is <>
Spark on Mesos' MesosCoarseGrainedSchedulerBackend is requested to <> (in client deploy mode)"},{"location":"local/LauncherBackend/#spark-on-yarns-client-is-requested-to","title":"* Spark on YARN's Client is requested to <>
LocalEndpoint is <> exclusively when LocalSchedulerBackend is requested to <>.
Put simply, LocalEndpoint is the communication channel between <> and <>. LocalEndpoint is a (thread-safe) rpc:RpcEndpoint.md[RpcEndpoint] that hosts an <> (with driver ID and localhost hostname) for Spark local mode.
When <>, LocalEndpoint requests the <> to scheduler:TaskSchedulerImpl.md#statusUpdate[handle a task status update] (given the taskId, the task state and the data).
If the given scheduler:Task.md#TaskState[TaskState] is a finished state (one of FINISHED, FAILED, KILLED, LOST states), LocalEndpoint adds scheduler:TaskSchedulerImpl.md#CPUS_PER_TASK[spark.task.cpus] configuration (default: 1) to the <> registry followed by <>.
NOTE: StatusUpdate RPC message is sent out exclusively when LocalSchedulerBackend is requested to <>.
When <>, LocalEndpoint requests the single <> to executor:Executor.md#killTask[kill a task] (given the taskId, the interruptThread flag and the reason).
NOTE: KillTask RPC message is sent out exclusively when LocalSchedulerBackend is requested to <>.
When <>, LocalEndpoint requests the single <> to executor:Executor.md#stop[stop] and requests the given RpcCallContext to reply with true (as the response).
NOTE: StopExecutor RPC message is sent out exclusively when LocalSchedulerBackend is requested to <>."},{"location":"local/LocalSchedulerBackend/","title":"LocalSchedulerBackend","text":"
LocalSchedulerBackend is a SchedulerBackend and an ExecutorBackend for Spark local deployment.
Master URL Total CPU Cores local 1 local[n]nlocal[*] The number of available CPU cores on the local machine local[n, m]n CPU cores and m task retries local[*, m] The number of available CPU cores on the local machine and m task retries "},{"location":"local/LocalSchedulerBackend/#creating-instance","title":"Creating Instance","text":"
LocalSchedulerBackend takes the following to be created:
SparkConf
TaskSchedulerImpl
Total number of CPU cores
LocalSchedulerBackend is created when:
SparkContext is requested to create a Spark Scheduler (for local master URL)
KubernetesClusterManager (Spark on Kubernetes) is requested for a SchedulerBackend
"},{"location":"local/LocalSchedulerBackend/#maxNumConcurrentTasks","title":"Maximum Number of Concurrent Tasks","text":"SchedulerBackend
MemoryAllocator is an abstraction of memory allocators that TaskMemoryManager uses to allocate and release memory.
MemoryAllocator creates the available MemoryAllocators to be available under the names HEAP and UNSAFE.
A MemoryAllocator to use is selected when MemoryManager is created (based on MemoryMode).
"},{"location":"memory/MemoryAllocator/#contract","title":"Contract","text":""},{"location":"memory/MemoryAllocator/#allocating-contiguous-block-of-memory","title":"Allocating Contiguous Block of Memory
MemoryBlock allocate(\n long size)\n
Used when:
TaskMemoryManager is requested to allocate a memory page
MemoryConsumer is an abstraction of memory consumers (of TaskMemoryManager) that support spilling.
MemoryConsumers correspond to individual operators and data structures within a task. TaskMemoryManager receives memory allocation requests from MemoryConsumers and issues callbacks to consumers in order to trigger spilling when running low on memory.
A MemoryConsumer basically tracks how much memory is allocated.
MemoryManager is an abstraction of memory managers that can share available memory between tasks (TaskMemoryManager) and storage (BlockManager).
MemoryManager splits assigned memory into two regions:
Execution Memory for shuffles, joins, sorts and aggregations
Storage Memory for caching and propagating internal data across Spark nodes (in on- and off-heap modes)
MemoryManager is used to create BlockManager (and MemoryStore) and TaskMemoryManager.
"},{"location":"memory/MemoryManager/#contract","title":"Contract","text":""},{"location":"memory/MemoryManager/#acquiring-execution-memory-for-task","title":"Acquiring Execution Memory for Task
MemoryManager is available as SparkEnv.memoryManager on the driver and executors.
import org.apache.spark.SparkEnv\nval mm = SparkEnv.get.memoryManager\n
// MemoryManager is private[spark]\n// the following won't work unless within org.apache.spark package\n// import org.apache.spark.memory.MemoryManager\n// assert(mm.isInstanceOf[MemoryManager])\n\n// we have to revert to string comparision \ud83d\ude14\nassert(\"UnifiedMemoryManager\".equals(mm.getClass.getSimpleName))\n
"},{"location":"memory/MemoryManager/#associating-memorystore-with-storage-memory-pools","title":"Associating MemoryStore with Storage Memory Pools
setMemoryStore(\n store: MemoryStore): Unit\n
setMemoryStore requests the on-heap and off-heap storage memory pools to use the given MemoryStore.
MemoryManager creates an ExecutionMemoryPool for ON_HEAP memory mode when created and immediately requests it to incrementPoolSize to onHeapExecutionMemory.
MemoryManager creates a StorageMemoryPool for ON_HEAP memory mode when created and immediately requests it to incrementPoolSize to onHeapExecutionMemory.
onHeapStorageMemoryPool is requested to setMemoryStore when MemoryManager is requested to setMemoryStore.
onHeapStorageMemoryPool is requested to release memory when MemoryManager is requested to release on-heap storage memory.
onHeapStorageMemoryPool is requested to release all memory when MemoryManager is requested to release all storage memory.
onHeapStorageMemoryPool is used when:
MemoryManager is requested for the storageMemoryUsed and onHeapStorageMemoryUsed
UnifiedMemoryManager is requested to acquire on-heap execution and storage memory
MemoryManager creates a StorageMemoryPool for OFF_HEAP memory mode when created and immediately requested it to incrementPoolSize to offHeapStorageMemory.
MemoryManager requests the MemoryPools to use a given MemoryStore when requested to setMemoryStore.
MemoryManager requests the MemoryPools to release memory when requested to releaseStorageMemory.
MemoryManager requests the MemoryPools to release all memory when requested to release all storage memory.
MemoryManager requests the MemoryPools for the memoryUsed when requested for storageMemoryUsed.
offHeapStorageMemoryPool is used when:
MemoryManager is requested for the offHeapStorageMemoryUsed
UnifiedMemoryManager is requested to acquire off-heap execution and storage memory
","text":""},{"location":"memory/MemoryManager/#total-storage-memory-used","title":"Total Storage Memory Used
storageMemoryUsed: Long\n
storageMemoryUsed is the sum of the memory used of the on-heap and off-heap storage memory pools.
tungstenMemoryMode tracks whether Tungsten memory will be allocated on the JVM heap or off-heap (using sun.misc.Unsafe).
final val
tungstenMemoryMode is a final value so initialized once when MemoryManager is created.
tungstenMemoryMode is OFF_HEAP when the following are all met:
spark.memory.offHeap.enabled configuration property is enabled
spark.memory.offHeap.size configuration property is greater than 0
JVM supports unaligned memory access (aka unaligned Unsafe, i.e. sun.misc.Unsafe package is available and the underlying system has unaligned-access capability)
Otherwise, tungstenMemoryMode is ON_HEAP.
Note
Given that spark.memory.offHeap.enabled configuration property is turned off by default and spark.memory.offHeap.size configuration property is 0 by default, Apache Spark seems to encourage using Tungsten memory allocated on the JVM heap (ON_HEAP).
tungstenMemoryMode is used when:
MemoryManager is created (and initializes the pageSizeBytes and tungstenMemoryAllocator internal properties)
defaultPageSizeBytes is a Scala lazy value to guarantee that the code to initialize it is executed once only (when accessed for the first time) and the computed value never changes afterwards.
"},{"location":"memory/MemoryPool/#contract","title":"Contract","text":""},{"location":"memory/MemoryPool/#size-of-memory-used","title":"Size of Memory Used
memoryUsed: Long\n
Used when:
MemoryPool is requested for the amount of free memory and decrementPoolSize
long acquireExecutionMemory(\n long required,\n MemoryConsumer consumer)\n
acquireExecutionMemory allocates up to required execution memory (bytes) for the MemoryConsumer (from the MemoryManager).
When not enough memory could be allocated initially, acquireExecutionMemory requests every consumer (with the same MemoryMode, itself including) to spill.
acquireExecutionMemory returns the amount of memory allocated.
acquireExecutionMemory\u00a0is used when:
MemoryConsumer is requested to acquire execution memory
TaskMemoryManager is requested to allocate a page
acquireExecutionMemory requests the MemoryManager to acquire execution memory (with required bytes, the taskAttemptId and the MemoryMode of the MemoryConsumer).
In the end, acquireExecutionMemory registers the MemoryConsumer (and adds it to the consumers registry) and prints out the following DEBUG message to the logs:
Task [taskAttemptId] acquired [got] for [consumer]\n
In case MemoryManager will have offerred less memory than required, acquireExecutionMemory finds the MemoryConsumers (in the consumers registry) with the MemoryMode and non-zero memory used, sorts them by memory usage, requests them (one by one) to spill until enough memory is acquired or there are no more consumers to release memory from (by spilling).
When a MemoryConsumer releases memory, acquireExecutionMemory prints out the following DEBUG message to the logs:
Task [taskAttemptId] released [released] from [c] for [consumer]\n
In case there is still not enough memory (less than required), acquireExecutionMemory requests the MemoryConsumer (to acquire memory for) to spill.
acquireExecutionMemory prints out the following DEBUG message to the logs:
Task [taskAttemptId] released [released] from itself ([consumer])\n
showMemoryUsage prints out the following INFO message to the logs (with the taskAttemptId):
Memory used in task [taskAttemptId]\n
showMemoryUsage requests every MemoryConsumer to report memory used. For consumers with non-zero memory usage, showMemoryUsage prints out the following INFO message to the logs:
Acquired by [consumer]: [memUsage]\n
showMemoryUsage requests the MemoryManager to getExecutionMemoryUsageForTask to calculate memory not accounted for (that is not associated with a specific consumer).
showMemoryUsage prints out the following INFO messages to the logs:
[memoryNotAccountedFor] bytes of memory were used by task [taskAttemptId] but are not associated with specific consumers\n
showMemoryUsage requests the MemoryManager for the executionMemoryUsed and storageMemoryUsed and prints out the following INFO message to the logs:
[executionMemoryUsed] bytes of memory are used for execution and\n[storageMemoryUsed] bytes of memory are used for storage\n
showMemoryUsage is used when:
MemoryConsumer is requested to throw an OutOfMemoryError
","text":""},{"location":"memory/TaskMemoryManager/#cleaning-up-all-allocated-memory","title":"Cleaning Up All Allocated Memory
long cleanUpAllAllocatedMemory()\n
The consumers collection is then cleared.
cleanUpAllAllocatedMemory finds all the registered MemoryConsumers (in the consumers registry) that still keep some memory used and, for every such consumer, prints out the following DEBUG message to the logs:
unreleased [getUsed] memory from [consumer]\n
cleanUpAllAllocatedMemory removes all the consumers.
For every MemoryBlock in the pageTable, cleanUpAllAllocatedMemory prints out the following DEBUG message to the logs:
unreleased page: [page] in task [taskAttemptId]\n
cleanUpAllAllocatedMemory marks the pages to be freed (FREED_IN_TMM_PAGE_NUMBER) and requests the MemoryManager for the tungstenMemoryAllocator to free up the MemoryBlock.
cleanUpAllAllocatedMemory clears the pageTable registry (by assigning null values).
cleanUpAllAllocatedMemory requests the MemoryManager to release execution memory that is not used by any consumer (with the acquiredButNotUsed and the tungstenMemoryMode).
In the end, cleanUpAllAllocatedMemory requests the MemoryManager to release all execution memory for the task.
cleanUpAllAllocatedMemory\u00a0is used when:
TaskRunner is requested to run a task (and the task has finished successfully)
MemoryBlock allocatePage(\n long size,\n MemoryConsumer consumer)\n
allocatePage allocates a block of memory (page) that is:
Below MAXIMUM_PAGE_SIZE_BYTES maximum size
For MemoryConsumers with the same MemoryMode as the TaskMemoryManager
allocatePage acquireExecutionMemory (for the size and the MemoryConsumer). allocatePage returns immediately (with null) when this allocation ended up with 0 or less bytes.
allocatePage allocates the first clear bit in the allocatedPages (unless the whole page table is taken and allocatePage throws an IllegalStateException).
allocatePage requests the MemoryManager for the tungstenMemoryAllocator that is requested to allocate the acquired memory.
allocatePage registers the page in the pageTable.
In the end, allocatePage prints out the following TRACE message to the logs and returns the MemoryBlock allocated.
Allocate page number [pageNumber] ([acquired] bytes)\n
Requesting the tungstenMemoryAllocator to allocate the acquired memory may throw an OutOfMemoryError. If so, allocatePage prints out the following WARN message to the logs:
Failed to allocate a page ([acquired] bytes), try again.\n
allocatePage adds the acquired memory to the acquiredButNotUsed and removes the page from the allocatedPages (by clearing the bit).
In the end, allocatePage tries to allocate the page again (recursively).
UnifiedMemoryManager is a MemoryManager (with the onHeapExecutionMemory being the Maximum Heap Memory with the onHeapStorageRegionSize taken out).
UnifiedMemoryManager allows for soft boundaries between storage and execution memory (allowing requests for memory in one region to be fulfilled by borrowing memory from the other).
apply creates a UnifiedMemoryManager with the Maximum Heap Memory and the size of the on-heap storage region as spark.memory.storageFraction of the Maximum Memory.
apply\u00a0is used when:
SparkEnv utility is used to create a base SparkEnv (for the driver and executors)
UnifiedMemoryManager is given the maximum heap memory to use (for execution and storage) when created (that uses apply factory method which uses getMaxMemory).
UnifiedMemoryManager makes sure that the driver's system memory is at least 1.5 of the Reserved System Memory. Otherwise, getMaxMemory throws an IllegalArgumentException:
System memory [systemMemory] must be at least [minSystemMemory].\nPlease increase heap size using the --driver-memory option or spark.driver.memory in Spark configuration.\n
UnifiedMemoryManager makes sure that the executor memory (spark.executor.memory) is at least the Reserved System Memory. Otherwise, getMaxMemory throws an IllegalArgumentException:
Executor memory [executorMemory] must be at least [minSystemMemory].\nPlease increase executor memory using the --executor-memory option or spark.executor.memory in Spark configuration.\n
UnifiedMemoryManager considers \"usable\" memory to be the system memory without the reserved memory.
UnifiedMemoryManager uses the fraction (based on spark.memory.fraction configuration property) of the \"usable\" memory for the maximum heap memory.
In the end, acquireExecutionMemory requests the ExecutionMemoryPool to acquire memory of numBytes bytes (with the maybeGrowExecutionPool and the maximum size of execution pool functions).
acquireExecutionMemory\u00a0is part of the MemoryManager abstraction.
","text":""},{"location":"memory/UnifiedMemoryManager/#maximum-size-of-execution-pool","title":"Maximum Size of Execution Pool
computeMaxExecutionPoolSize(): Long\n
computeMaxExecutionPoolSize takes the minimum size of the storage memory regions (based on the memory mode, ON_HEAP or OFF_HEAP, respectively):
Memory used of the on-heap or the off-heap storage memory pool
On-heap or the off-heap storage memory size
In the end, computeMaxExecutionPoolSize returns the size of the remaining memory space of the maximum memory (the maxHeapMemory or the maxOffHeapMemory for ON_HEAP or OFF_HEAP memory mode, respectively) without (the minimum size of) the storage memory region.
Spark Metrics gives you execution metrics of Spark subsystems (metrics instances, e.g. the driver of a Spark application or the master of a Spark Standalone cluster).
Spark Metrics uses Dropwizard Metrics Java library for the metrics infrastructure.
Metrics is a Java library which gives you unparalleled insight into what your code does in production.
Metrics provides a powerful toolkit of ways to measure the behavior of critical components in your production environment.
MetricsConfig is the configuration of the MetricsSystem (i.e. metrics spark-metrics-Source.md[sources] and spark-metrics-Sink.md[sinks]).
metrics.properties is the default metrics configuration file. It is configured using spark-metrics-properties.md#spark.metrics.conf[spark.metrics.conf] configuration property. The file is first loaded from the path directly before using Spark's CLASSPATH.
MetricsConfig also accepts a metrics configuration using spark.metrics.conf.-prefixed configuration properties.
Spark comes with conf/metrics.properties.template file that is a template of metrics configuration.
Among the metrics sinks is spark-metrics-MetricsServlet.md[MetricsServlet] that is used when sink.servlet metrics sink is configured in spark-metrics-MetricsConfig.md[metrics configuration].
CAUTION: FIXME Describe configuration files and properties
NOTE: You can access a Spark subsystem's MetricsSystem using its corresponding \"leading\" port, e.g. 4040 for the driver, 8080 for Spark Standalone's master and applications.
NOTE: You have to use the trailing slash (/) to have the output.
MetricsConfig is the configuration of the MetricsSystem (i.e. metrics sources and sinks).
MetricsConfig is <> when MetricsSystem is.
MetricsConfig uses metrics.properties as the default metrics configuration file. It is configured using spark-metrics-properties.md#spark.metrics.conf[spark.metrics.conf] configuration property. The file is first loaded from the path directly before using Spark's CLASSPATH.
MetricsConfig accepts a metrics configuration using spark.metrics.conf.-prefixed configuration properties.
Spark comes with conf/metrics.properties.template file that is a template of metrics configuration.
subProperties takes prop properties and destructures keys given regex. subProperties takes the matching prefix (of a key per regex) and uses it as a new key with the value(s) being the matching suffix(es).
NOTE: subProperties is used when MetricsConfig <> (to apply the default metrics configuration) and when MetricsSystem registers metrics sources and sinks.
MetricsServlet is a metrics sink that gives metrics snapshots in JSON format.
MetricsServlet is a \"special\" sink as it is only available to the metrics instances with a web UI:
Driver of a Spark application
Spark Standalone's Master and Worker
You can access the metrics from MetricsServlet at /metrics/json URI by default. The entire URL depends on a metrics instance, e.g. http://localhost:4040/metrics/json/ for a running Spark application.
MetricsServlet is <> exclusively when MetricsSystem is started (and requested to register metrics sinks).
MetricsServlet can be configured using configuration properties with sink.servlet prefix (in spark-metrics-MetricsConfig.md[metrics configuration]). That is not required since MetricsConfig spark-metrics-MetricsConfig.md#setDefaultProperties[makes sure] that MetricsServlet is always configured.
MetricsServlet uses https://fasterxml.github.io/jackson-databind/[jackson-databind], the general data-binding package for Jackson (as <>) with Dropwizard Metrics library (i.e. registering a Coda Hale MetricsModule).
| path | /metrics/json/ | [[path]] Path URI prefix to bind to
| sample | false | [[sample]] Whether to show entire set of samples for histograms |===
[[internal-registries]] .MetricsServlet's Internal Properties (e.g. Registries, Counters and Flags) [cols=\"1,2\",options=\"header\",width=\"100%\"] |=== | Name | Description
| mapper | [[mapper]] Jaxson's https://fasterxml.github.io/jackson-databind/javadoc/2.6/com/fasterxml/jackson/databind/ObjectMapper.html[com.fasterxml.jackson.databind.ObjectMapper] that \"provides functionality for reading and writing JSON, either to and from basic POJOs (Plain Old Java Objects), or to and from a general-purpose JSON Tree Model (JsonNode), as well as related functionality for performing conversions.\"
When created, mapper is requested to register a Coda Hale com.codahale.metrics.json.MetricsModule.
Used exclusively when MetricsServlet is requested to <>.
| servletPath | [[servletPath]] Value of <> configuration property
| servletShowSample | [[servletShowSample]] Flag to control whether to show samples (true) or not (false).
servletShowSample is the value of <> configuration property (if defined) or false.
Used when <> is requested to register a Coda Hale com.codahale.metrics.json.MetricsModule. |==="},{"location":"metrics/MetricsServlet/#creating-instance","title":"Creating Instance","text":"
getMetricsSnapshot simply requests the <> to serialize the <> to a JSON string (using ++https://fasterxml.github.io/jackson-databind/javadoc/2.6/com/fasterxml/jackson/databind/ObjectMapper.html#writeValueAsString-java.lang.Object-++[ObjectMapper.writeValueAsString]).
NOTE: getMetricsSnapshot is used exclusively when MetricsServlet is requested to <>.
MetricsSystem creates a PrometheusServlet when requested to registerSinks for an instance with sink.prometheusServlet configuration.
MetricsSystem requests the PrometheusServlet for URL handlers when requested for servlet handlers (so it can be attached to a web UI and serve HTTP requests).
MetricsServlet JSON metrics sink that is only available for the <> with a web UI (i.e. the driver of a Spark application and Spark Standalone's Master).
MetricsSystem may have at most one MetricsServlet JSON metrics sink (which is registered by default).
Initialized when MetricsSystem registers <> (and finds a configuration entry with servlet sink name).
Used when MetricsSystem is requested for a <>.","text":""},{"location":"metrics/MetricsSystem/#creating-metricssystem","title":"Creating MetricsSystem
buildRegistryName uses spark-metrics-properties.md#spark.metrics.namespace[spark.metrics.namespace] and executor:Executor.md#spark.executor.id[spark.executor.id] Spark properties to differentiate between a Spark application's driver and executors, and the other Spark framework's components.
(only when <> is driver or executor) buildRegistryName builds metrics source name that is made up of spark-metrics-properties.md#spark.metrics.namespace[spark.metrics.namespace], executor:Executor.md#spark.executor.id[spark.executor.id] and the name of the source.
FIXME Finish for the other components.
buildRegistryName is used when MetricsSystem is requested to register or remove a metrics source.
","text":""},{"location":"metrics/MetricsSystem/#registering-metrics-sources-for-spark-instance","title":"Registering Metrics Sources for Spark Instance
registerSources(): Unit\n
registerSources finds <> configuration for the <>.
NOTE: instance is defined when MetricsSystem <>.
registerSources finds the configuration of all the spark-metrics-Source.md[metrics sources] for the subsystem (as described with source. prefix).
For every metrics source, registerSources finds class property, creates an instance, and in the end <>.
When registerSources fails, you should see the following ERROR message in the logs followed by the exception.
Source class [classPath] cannot be instantiated\n
registerSources is used when MetricsSystem is requested to start.
registerSinks requests the <> for the spark-metrics-MetricsConfig.md#getInstance[configuration] of the <>.
registerSinks requests the <> for the spark-metrics-MetricsConfig.md#subProperties[configuration] of all metrics sinks (i.e. configuration entries that match ^sink\\\\.(.+)\\\\.(.+) regular expression).
For every metrics sink configuration, registerSinks takes class property and (if defined) creates an instance of the metric sink using an constructor that takes the configuration, <> and <>.
For a single servlet metrics sink, registerSinks converts the sink to a spark-metrics-MetricsServlet.md[MetricsServlet] and sets the <> internal registry.
For all other metrics sinks, registerSinks adds the sink to the <> internal registry.
In case of an Exception, registerSinks prints out the following ERROR message to the logs:
Sink class [classPath] cannot be instantiated\n
registerSinks is used when MetricsSystem is requested to start.
Default: Spark Application ID (i.e. spark.app.id configuration property)
Since a Spark application's ID changes with every execution of a Spark application, a custom namespace can be specified for an easier metrics reporting.
Used when MetricsSystem is requested for a metrics source identifier (metrics namespace)
The given module is shuffle most of the time except:
rpc for NettyRpcEnv
files for NettyRpcEnv
Only defined in NettyRpcEnv to be either driver or executor
fromSparkConf makes a copy (clones) the given SparkConf.
fromSparkConf sets the following configuration properties (for the given module):
spark.[module].io.serverThreads
spark.[module].io.clientThreads
The values are taken using the following properties in the order and until one is found (with suffix being serverThreads or clientThreads, respectively):
spark.[role].[module].io.[suffix]
spark.[module].io.[suffix]
Unless found, fromSparkConf defaults to the default number of threads (based on the given numUsableCores and not more than 8).
In the end, fromSparkConf creates a TransportConf (for the given module and the updated SparkConf).
fromSparkConf\u00a0is used when:
SparkEnv utility is used to create a SparkEnv (with the spark.shuffle.service.enabled configuration property enabled)
ExternalShuffleService is created
NettyBlockTransferService is requested to init
NettyRpcEnv is created and requested for a downloadClient
IndexShuffleBlockResolver is created
ShuffleBlockPusher is requested to initiateBlockPush
BlockManager is requested to readDiskBlockFromSameHostExecutor
While being created, TransportClientFactory requests the given TransportContext for the TransportConf that is used to access the values of the following (configuration) properties:
Plugin Framework is an API for registering custom extensions (plugins) to be executed on the driver and executors.
Plugin Framework uses separate PluginContainers for the driver and executors, and spark.plugins configuration property for SparkPlugins to be registered.
Plugin Framework was introduced in Spark 2.4.4 (with an API for executors) with further changes in Spark 3.0.0 (to cover the driver).
executorPlugins finds all the configuration properties with spark.plugins.internal.conf. prefix (in the SparkConf) for extra configuration of every ExecutorPlugin of the given SparkPlugins.
For every SparkPlugin (in the given SparkPlugins) that defines an ExecutorPlugin, executorPlugins creates a PluginContextImpl, requests the ExecutorPlugin to init (with the PluginContextImpl and the extra configuration) and the PluginContextImpl to registerMetrics.
In the end, executorPlugins prints out the following INFO message to the logs (for every ExecutorPlugin):
Initialized executor component for plugin [name].\n
PluginContainer is an abstraction of plugin containers that can register metrics (for the driver and executors).
PluginContainer is created for the driver and executors using apply utility.
"},{"location":"plugins/PluginContainer/#contract","title":"Contract","text":""},{"location":"plugins/PluginContainer/#listening-to-task-failures","title":"Listening to Task Failures
apply creates a PluginContainer for the driver or executors (based on the type of the first input argument, i.e. SparkContext or SparkEnv, respectively).
apply first loads the SparkPlugins defined by spark.plugins configuration property.
Only when there was at least one plugin loaded, apply creates a DriverPluginContainer or ExecutorPluginContainer.
Resilient Distributed Dataset (aka RDD) is the primary data abstraction in Apache Spark and the core of Spark (that I often refer to as \"Spark Core\").
.The origins of RDD
The original paper that gave birth to the concept of RDD is https://cs.stanford.edu/~matei/papers/2012/nsdi_spark.pdf[Resilient Distributed Datasets: A Fault-Tolerant Abstraction for In-Memory Cluster Computing] by Matei Zaharia, et al.
An RDD is a description of a fault-tolerant and resilient computation over a distributed collection of records (spread over <>).
NOTE: One could compare RDDs to collections in Scala, i.e. a RDD is computed on many JVMs while a Scala collection lives on a single JVM.
Using RDD Spark hides data partitioning and so distribution that in turn allowed them to design parallel computational framework with a higher-level programming interface (API) for four mainstream programming languages.
The features of RDDs (decomposing the name):
Resilient, i.e. fault-tolerant with the help of <> and so able to recompute missing or damaged partitions due to node failures.
Distributed with data residing on multiple nodes in a spark-cluster.md[cluster].
Dataset is a collection of spark-rdd-partitions.md[partitioned data] with primitive values or values of values, e.g. tuples or other objects (that represent records of the data you work with).
.RDDs image::spark-rdds.png[align=\"center\"]
From the scaladoc of http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.RDD[org.apache.spark.rdd.RDD]:
A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, partitioned collection of elements that can be operated on in parallel.
From the original paper about RDD - https://cs.stanford.edu/~matei/papers/2012/nsdi_spark.pdf[Resilient Distributed Datasets: A Fault-Tolerant Abstraction for In-Memory Cluster Computing]:
Resilient Distributed Datasets (RDDs) are a distributed memory abstraction that lets programmers perform in-memory computations on large clusters in a fault-tolerant manner.
Beside the above traits (that are directly embedded in the name of the data abstraction - RDD) it has the following additional traits:
In-Memory, i.e. data inside RDD is stored in memory as much (size) and long (time) as possible.
Immutable or Read-Only, i.e. it does not change once created and can only be transformed using transformations to new RDDs.
Lazy evaluated, i.e. the data inside RDD is not available or transformed until an action is executed that triggers the execution.
Cacheable, i.e. you can hold all the data in a persistent \"storage\" like memory (default and the most preferred) or disk (the least preferred due to access speed).
Parallel, i.e. process data in parallel.
Typed -- RDD records have types, e.g. Long in RDD[Long] or (Int, String) in RDD[(Int, String)].
Partitioned -- records are partitioned (split into logical partitions) and distributed across nodes in a cluster.
Location-Stickiness -- RDD can define <> to compute partitions (as close to the records as possible).
NOTE: Preferred location (aka locality preferences or placement preferences or locality info) is information about the locations of RDD records (that Spark's scheduler:DAGScheduler.md#preferred-locations[DAGScheduler] uses to place computing partitions on to have the tasks as close to the data as possible).
Computing partitions in a RDD is a distributed process by design and to achieve even data distribution as well as leverage data locality (in distributed systems like HDFS or Apache Kafka in which data is partitioned by default), they are partitioned to a fixed number of spark-rdd-partitions.md[partitions] - logical chunks (parts) of data. The logical division is for processing only and internally it is not divided whatsoever. Each partition comprises of records.
spark-rdd-partitions.md[Partitions are the units of parallelism]. You can control the number of partitions of a RDD using spark-rdd-partitions.md#repartition[repartition] or spark-rdd-partitions.md#coalesce[coalesce] transformations. Spark tries to be as close to data as possible without wasting time to send data across network by means of RDD shuffling, and creates as many partitions as required to follow the storage layout and thus optimize data access. It leads to a one-to-one mapping between (physical) data in distributed data storage, e.g. HDFS or Cassandra, and partitions.
RDDs support two kinds of operations:
<> - lazy operations that return another RDD.
<> - operations that trigger computation and return values.
The motivation to create RDD were (https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf[after the authors]) two types of applications that current computing frameworks handle inefficiently:
iterative algorithms in machine learning and graph computations.
interactive data mining tools as ad-hoc queries on the same dataset.
The goal is to reuse intermediate in-memory results across multiple data-intensive workloads with no need for copying large amounts of data over the network.
Technically, RDDs follow the <> defined by the five main intrinsic properties:
An array of spark-rdd-partitions.md[partitions] that a dataset is divided to.
A rdd:RDD.md#compute[compute] function to do a computation on partitions.
An optional rdd:Partitioner.md[Partitioner] that defines how keys are hashed, and the pairs partitioned (for key-value RDDs)
Optional <> (aka locality info), i.e. hosts for a partition where the records live or are the closest to read from.
This RDD abstraction supports an expressive set of operations without having to modify scheduler for each one.
[[context]] An RDD is a named (by name) and uniquely identified (by id) entity in a SparkContext.md[] (available as context property).
RDDs live in one and only one SparkContext.md[] that creates a logical boundary.
NOTE: RDDs cannot be shared between SparkContexts (see SparkContext.md#sparkcontext-and-rdd[SparkContext and RDDs]).
An RDD can optionally have a friendly name accessible using name that can be changed using =:
scala> val ns = sc.parallelize(0 to 10)\nns: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[2] at parallelize at <console>:24\n\nscala> ns.id\nres0: Int = 2\n\nscala> ns.name\nres1: String = null\n\nscala> ns.name = \"Friendly name\"\nns.name: String = Friendly name\n\nscala> ns.name\nres2: String = Friendly name\n\nscala> ns.toDebugString\nres3: String = (8) Friendly name ParallelCollectionRDD[2] at parallelize at <console>:24 []\n
RDDs are a container of instructions on how to materialize big (arrays of) distributed data, and how to split it into partitions so Spark (using executor:Executor.md[executors]) can hold some of them.
In general data distribution can help executing processing in parallel so a task processes a chunk of data that it could eventually keep in memory.
Spark does jobs in parallel, and RDDs are split into partitions to be processed and written in parallel. Inside a partition, data is processed sequentially.
Saving partitions results in part-files instead of one single file (unless there is a single partition).
== [[transformations]] Transformations
A transformation is a lazy operation on a RDD that returns another RDD, e.g. map, flatMap, filter, reduceByKey, join, cogroup, etc.
Find out more in rdd:spark-rdd-transformations.md[Transformations].
== [[actions]] Actions
An action is an operation that triggers execution of <> and returns a value (to a Spark driver - the user program).
TIP: Go in-depth in the section spark-rdd-actions.md[Actions].
== [[creating-rdds]] Creating RDDs
=== SparkContext.parallelize
One way to create a RDD is with SparkContext.parallelize method. It accepts a collection of elements as shown below (sc is a SparkContext instance):
scala> val rdd = sc.parallelize(1 to 1000)\nrdd: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[0] at parallelize at <console>:25\n
You may also want to randomize the sample data:
scala> val data = Seq.fill(10)(util.Random.nextInt)\ndata: Seq[Int] = List(-964985204, 1662791, -1820544313, -383666422, -111039198, 310967683, 1114081267, 1244509086, 1797452433, 124035586)\n\nscala> val rdd = sc.parallelize(data)\nrdd: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[0] at parallelize at <console>:29\n
Given the reason to use Spark to process more data than your own laptop could handle, SparkContext.parallelize is mainly used to learn Spark in the Spark shell. SparkContext.parallelize requires all the data to be available on a single machine - the Spark driver - that eventually hits the limits of your laptop.
=== SparkContext.makeRDD
CAUTION: FIXME What's the use case for makeRDD?
scala> sc.makeRDD(0 to 1000)\nres0: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[1] at makeRDD at <console>:25\n
=== SparkContext.textFile
One of the easiest ways to create an RDD is to use SparkContext.textFile to read files.
You can use the local README.md file (and then flatMap over the lines inside to have an RDD of words):
scala> val words = sc.textFile(\"README.md\").flatMap(_.split(\"\\\\W+\")).cache\nwords: org.apache.spark.rdd.RDD[String] = MapPartitionsRDD[27] at flatMap at <console>:24\n
NOTE: You spark-rdd-caching.md[cache] it so the computation is not performed every time you work with words.
== [[creating-rdds-from-input]] Creating RDDs from Input
Refer to spark-io.md[Using Input and Output (I/O)] to learn about the IO API to create RDDs.
=== Transformations
RDD transformations by definition transform an RDD into another RDD and hence are the way to create new ones.
Refer to <> section to learn more.
== RDDs in Web UI
It is quite informative to look at RDDs in the Web UI that is at http://localhost:4040 for spark-shell.md[Spark shell].
Execute the following Spark application (type all the lines in spark-shell):
val ints = sc.parallelize(1 to 100) // <1> ints.setName(\"Hundred ints\") // <2> ints.cache // <3> ints.count // <4>
<1> Creates an RDD with hundred of numbers (with as many partitions as possible) <2> Sets the name of the RDD <3> Caches the RDD for performance reasons that also makes it visible in Storage tab in the web UI <4> Executes action (and materializes the RDD)
With the above executed, you should see the following in the Web UI:
.RDD with custom name image::spark-ui-rdd-name.png[align=\"center\"]
Click the name of the RDD (under RDD Name) and you will get the details of how the RDD is cached.
.RDD Storage Info image::spark-ui-storage-hundred-ints.png[align=\"center\"]
Execute the following Spark job and you will see how the number of partitions decreases.
ints.repartition(2).count\n
.Number of tasks after repartition image::spark-ui-repartition-2.png[align=\"center\"]
Aggregator is a set of <> used to aggregate data using rdd:PairRDDFunctions.md#combineByKeyWithClassTag[PairRDDFunctions.combineByKeyWithClassTag] transformation.
Aggregator[K, V, C] is a parameterized type of K keys, V values, and C combiner (partial) values.
[[creating-instance]][[aggregation-functions]] Aggregator transforms an RDD[(K, V)] into an RDD[(K, C)] (for a \"combined type\" C) using the functions:
[[createCombiner]] createCombiner: V => C
[[mergeValue]] mergeValue: (C, V) => C
[[mergeCombiners]] mergeCombiners: (C, C) => C
Aggregator is used to create a ShuffleDependency and ExternalSorter.
combineValuesByKey creates a new shuffle:ExternalAppendOnlyMap.md[ExternalAppendOnlyMap] (with the <>).
combineValuesByKey requests the ExternalAppendOnlyMap to shuffle:ExternalAppendOnlyMap.md#insertAll[insert all key-value pairs] from the given iterator (that is the values of a partition).
combineValuesByKey <>.
In the end, combineValuesByKey requests the ExternalAppendOnlyMap for an shuffle:ExternalAppendOnlyMap.md#iterator[iterator of \"combined\" pairs].
combineValuesByKey is used when:
rdd:PairRDDFunctions.md#combineByKeyWithClassTag[PairRDDFunctions.combineByKeyWithClassTag] transformation is used (with the same Partitioner as the RDD's)
BlockStoreShuffleReader is requested to shuffle:BlockStoreShuffleReader.md#read[read combined records for a reduce task] (with the Map-Size Partial Aggregation Flag off)
combineCombinersByKey is used when BlockStoreShuffleReader is requested to shuffle:BlockStoreShuffleReader.md#read[read combined records for a reduce task] (with the Map-Size Partial Aggregation Flag on).
scala> myRdd.dependencies.map(_.rdd).foreach(println)\nMapPartitionsRDD[6] at groupBy at <console>:39\n
RDD.toDebugString is used to print out the RDD lineage in a developer-friendly way.
scala> println(myRdd.toDebugString)\n(16) ShuffledRDD[7] at groupBy at <console>:39 []\n +-(16) MapPartitionsRDD[6] at groupBy at <console>:39 []\n | ParallelCollectionRDD[5] at parallelize at <console>:39 []\n
https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.HadoopRDD[HadoopRDD] is an RDD that provides core functionality for reading data stored in HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI using the older MapReduce API (https://hadoop.apache.org/docs/current/api/org/apache/hadoop/mapred/package-summary.html[org.apache.hadoop.mapred]).
HadoopRDD is created as a result of calling the following methods in SparkContext.md[]:
hadoopFile
textFile (the most often used in examples!)
sequenceFile
Partitions are of type HadoopPartition.
When an HadoopRDD is computed, i.e. an action is called, you should see the INFO message Input split: in the logs.
scala> sc.textFile(\"README.md\").count\n...\n15/10/10 18:03:21 INFO HadoopRDD: Input split: file:/Users/jacek/dev/oss/spark/README.md:0+1784\n15/10/10 18:03:21 INFO HadoopRDD: Input split: file:/Users/jacek/dev/oss/spark/README.md:1784+1784\n...\n
The following properties are set upon partition execution:
mapred.tip.id - task id of this task's attempt
mapred.task.id - task attempt's id
mapred.task.is.map as true
mapred.task.partition - split id
mapred.job.id
Spark settings for HadoopRDD:
spark.hadoop.cloneConf (default: false) - shouldCloneJobConf - should a Hadoop job configuration JobConf object be cloned before spawning a Hadoop job. Refer to https://issues.apache.org/jira/browse/SPARK-2546[[SPARK-2546] Configuration object thread safety issue]. When true, you should see a DEBUG message Cloning Hadoop Configuration.
You can register callbacks on TaskContext.
HadoopRDDs are not checkpointed. They do nothing when checkpoint() is called.
The number of partition for HadoopRDD, i.e. the return value of getPartitions, is calculated using InputFormat.getSplits(jobConf, minPartitions) where minPartitions is only a hint of how many partitions one may want at minimum. As a hint it does not mean the number of partitions will be exactly the number given.
For SparkContext.textFile the input format class is https://hadoop.apache.org/docs/current/api/org/apache/hadoop/mapred/TextInputFormat.html[org.apache.hadoop.mapred.TextInputFormat].
The https://hadoop.apache.org/docs/current/api/org/apache/hadoop/mapred/FileInputFormat.html[javadoc of org.apache.hadoop.mapred.FileInputFormat] says:
FileInputFormat is the base class for all file-based InputFormats. This provides a generic implementation of getSplits(JobConf, int). Subclasses of FileInputFormat can also override the isSplitable(FileSystem, Path) method to ensure input-files are not split-up and are processed as a whole by Mappers.
TIP: You may find https://github.com/apache/hadoop/blob/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java#L319[the sources of org.apache.hadoop.mapred.FileInputFormat.getSplits] enlightening.
"},{"location":"rdd/HadoopRDD/#whats-hadoop-split-input-splits-for-hadoop-reads-see-inputformatgetsplits","title":"What's Hadoop Split? input splits for Hadoop reads? See InputFormat.getSplits","text":""},{"location":"rdd/HashPartitioner/","title":"HashPartitioner","text":"
HashPartitioner is a Partitioner for hash-based partitioning.
Important
HashPartitioner places null keys in 0th partition.
HashPartitioner is used as the default Partitioner.
HashPartitioner takes the following to be created:
Number of partitions"},{"location":"rdd/HashPartitioner/#number-of-partitions","title":"Number of Partitions
numPartitions: Int\n
numPartitions returns the given number of partitions.
numPartitions\u00a0is part of the Partitioner abstraction.
","text":""},{"location":"rdd/HashPartitioner/#partition-for-key","title":"Partition for Key
getPartition(\n key: Any): Int\n
For null keys getPartition simply returns 0.
For non-null keys, getPartition uses the Object.hashCode of the key modulo the number of partitions. For negative results, getPartition adds the number of partitions to make it non-negative.
getPartition\u00a0is part of the Partitioner abstraction.
compute merely throws an SparkException (that explains the reason):
Checkpoint block [RDDBlockId] not found! Either the executor\nthat originally checkpointed this partition is no longer alive, or the original RDD is\nunpersisted. If this problem persists, you may consider using `rdd.checkpoint()`\ninstead, which is slower than local checkpointing but more fault-tolerant.\"\n
doCheckpoint\u00a0is part of the RDDCheckpointData abstraction.
doCheckpoint creates a LocalCheckpointRDD with the RDD. doCheckpoint triggers caching any missing partitions (by checking availability of the RDDBlockIds for the partitions in the BlockManagerMaster).
Extra Spark Job
If there are any missing partitions (RDDBlockIds) doCheckpoint requests the SparkContext to run a Spark job with the RDD and the missing partitions.
doCheckpointmakes sure that the StorageLevel of the RDD uses disk (among other persistence storages). If not, doCheckpoint\u00a0throws an AssertionError:
Storage level [level] is not appropriate for local checkpointing\n
isBarrier_ is enabled (true) when either this MapPartitionsRDD is isFromBarrier or any of the parent RDDs is isBarrier. Otherwise, isBarrier_ is disabled (false).
NarrowDependency[T] is an extension of the Dependency abstraction for narrow dependencies (of RDD[T]s) where each partition of the child RDD depends on a small number of partitions of the parent RDD.
PruneDependency is a NarrowDependency that represents a dependency between the PartitionPruningRDD and the parent RDD (with a subset of partitions of the parents).
NewHadoopRDD initializes the <>."},{"location":"rdd/OrderedRDDFunctions/","title":"OrderedRDDFunctions","text":"
class OrderedRDDFunctions[\n K: Ordering : ClassTag,\n V: ClassTag,\n P <: Product2[K, V] : ClassTag]\n
OrderedRDDFunctions adds extra operators to RDDs of (key, value) pairs (RDD[(K, V)]) where the K key is sortable (i.e. any key type K that has an implicit Ordering[K] in scope).
Tip
Learn more about Ordering in the Scala Standard Library documentation.
Uses a HashPartitioner (with the given numPartitions)
combineByKeyWithClassTag creates an Aggregator for the given aggregation functions.
combineByKeyWithClassTag branches off per the given Partitioner.
If the input partitioner and the RDD's are the same, combineByKeyWithClassTag simply mapPartitions on the RDD with the following arguments:
Iterator of the Aggregator
preservesPartitioning flag turned on
If the input partitioner is different than the RDD's, combineByKeyWithClassTag creates a ShuffledRDD (with the Serializer, the Aggregator, and the mapSideCombine flag).
saveAsNewAPIHadoopDataset creates a new HadoopMapReduceWriteConfigUtil (with the given Configuration) and writes the RDD out.
Configuration should have all the relevant output params set (an output format, output paths, e.g. a table name to write to) in the same way as it would be configured for a Hadoop MapReduce job.
"},{"location":"rdd/RDD/","title":"RDD \u2014 Description of Distributed Computation","text":"
RDD[T] is an abstraction of fault-tolerant resilient distributed datasets that are mere descriptions of computations over a distributed collection of records (of type T).
DAGScheduler is requested to submitMissingTasks (that are either ShuffleMapStages to create ShuffleMapTasks or ResultStage to create ResultTasks)
RDDInfo is created
ShuffleDependency is requested to canShuffleMergeBeEnabled
DAGScheduler is requested to checkBarrierStageWithRDDChainPattern, checkBarrierStageWithDynamicAllocation, checkBarrierStageWithNumSlots, handleTaskCompletion (FetchFailed case to mark a map stage as broken)
doCheckpoint turns the doCheckpointCalled flag on (to prevent multiple executions).
doCheckpoint branches off based on whether a RDDCheckpointData is defined or not:
With the RDDCheckpointData defined, doCheckpoint checks out the checkpointAllMarkedAncestors flag and if enabled, doCheckpoint requests the Dependencies for the RDD that are in turn requested to doCheckpoint themselves. Otherwise, doCheckpoint requests the RDDCheckpointData to checkpoint.
With the RDDCheckpointData undefined, doCheckpoint requests the Dependencies for the RDD that are in turn requested to doCheckpoint themselves.
In other words, With the RDDCheckpointData defined, requesting doCheckpointing of the Dependencies is guarded by checkpointAllMarkedAncestors flag.
doCheckpoint skips execution if called earlier.
doCheckpoint is used when:
SparkContext is requested to run a job synchronously
val wordCount = sc.textFile(\"README.md\")\n .flatMap(_.split(\"\\\\s+\"))\n .map((_, 1))\n .reduceByKey(_ + _)\n\nscala> println(wordCount.toDebugString)\n(2) ShuffledRDD[21] at reduceByKey at <console>:24 []\n +-(2) MapPartitionsRDD[20] at map at <console>:24 []\n | MapPartitionsRDD[19] at flatMap at <console>:24 []\n | README.md MapPartitionsRDD[18] at textFile at <console>:24 []\n | README.md HadoopRDD[17] at textFile at <console>:24 []\n
toDebugString uses indentations to indicate a shuffle boundary.
The numbers in round brackets show the level of parallelism at each stage, e.g. (2) in the above output.
scala> println(wordCount.getNumPartitions)\n2\n
With spark.logLineage enabled, toDebugString is printed out when executing an action.
$ ./bin/spark-shell --conf spark.logLineage=true\n\nscala> sc.textFile(\"README.md\", 4).count\n...\n15/10/17 14:46:42 INFO SparkContext: Starting job: count at <console>:25\n15/10/17 14:46:42 INFO SparkContext: RDD's recursive dependencies:\n(4) MapPartitionsRDD[1] at textFile at <console>:25 []\n | README.md HadoopRDD[0] at textFile at <console>:25 []\n
RangePartitioner takes the following to be created:
Hint for the number of partitions
Key-Value RDD (RDD[_ <: Product2[K, V]])
ascending flag (default: true)
samplePointsPerPartitionHint (default: 20)"},{"location":"rdd/RangePartitioner/#number-of-partitions","title":"Number of Partitions
numPartitions: Int\n
numPartitions\u00a0is part of the Partitioner abstraction.
numPartitions is 1 more than the length of the range bounds (since the number of range bounds is 0 for 0 or 1 partitions).
","text":""},{"location":"rdd/RangePartitioner/#partition-for-key","title":"Partition for Key
getPartition(\n key: Any): Int\n
getPartition\u00a0is part of the Partitioner abstraction.
getPartition branches off based on the length of the range bounds.
For up to 128 range bounds, getPartition is either the first range bound (from the rangeBounds) for which the key value is greater than the value of the range bound or 128 (if no value was found among the rangeBounds). getPartition starts finding a candidate partition number from 0 and walks over the rangeBounds until a range bound for which the given key value is greater than the value of the range bound is found or there are no more rangeBounds. getPartition increments the candidate partition candidate every iteration.
For the number of the rangeBounds above 128, getPartition...FIXME
In the end, getPartition returns the candidate partition number for the ascending enabled, or flips it (to be the number of the rangeBounds minus the candidate partition number), otheriwse.
For the number of partitions up to and including 1, rangeBounds is an empty array.
For more than 1 partitions, rangeBounds determines the sample size per partitions. The total sample size is the samplePointsPerPartitionHint multiplied by the number of partitions capped by 1e6. rangeBounds allows for 3x over-sample per partition.
rangeBounds sketches the keys of the input rdd (with the sampleSizePerPartition).
writePartitionerToCheckpointDir( sc: SparkContext, partitioner: Partitioner, checkpointDirPath: Path): Unit
writePartitionerToCheckpointDir creates the <> with the buffer size based on configuration-properties.md#spark.buffer.size[spark.buffer.size] configuration property.
writePartitionerToCheckpointDir requests the core:SparkEnv.md#serializer[default Serializer] for a new serializer:Serializer.md#newInstance[SerializerInstance].
writePartitionerToCheckpointDir requests the SerializerInstance to serializer:SerializerInstance.md#serializeStream[serialize the output stream] and serializer:DeserializationStream.md#writeObject[writes] the given Partitioner.
In the end, writePartitionerToCheckpointDir prints out the following DEBUG message to the logs:
"},{"location":"rdd/ReliableCheckpointRDD/#sourceplaintext","title":"[source,plaintext]","text":""},{"location":"rdd/ReliableCheckpointRDD/#written-partitioner-to-partitionerfilepath","title":"Written partitioner to [partitionerFilePath]","text":"
In case of any non-fatal exception, writePartitionerToCheckpointDir prints out the following DEBUG message to the logs:
"},{"location":"rdd/ReliableCheckpointRDD/#sourceplaintext_1","title":"[source,plaintext]","text":""},{"location":"rdd/ReliableCheckpointRDD/#error-writing-partitioner-partitioner-to-checkpointdirpath","title":"Error writing partitioner [partitioner] to [checkpointDirPath]","text":"
writePartitionerToCheckpointDir is used when ReliableCheckpointRDD is requested to <>.
== [[readCheckpointedPartitionerFile]] Reading Partitioner from Checkpointed Directory
readCheckpointedPartitionerFile opens the <> with the buffer size based on configuration-properties.md#spark.buffer.size[spark.buffer.size] configuration property.
readCheckpointedPartitionerFile requests the core:SparkEnv.md#serializer[default Serializer] for a new serializer:Serializer.md#newInstance[SerializerInstance].
readCheckpointedPartitionerFile requests the SerializerInstance to serializer:SerializerInstance.md#deserializeStream[deserialize the input stream] and serializer:DeserializationStream.md#readObject[read the Partitioner] from the partitioner file.
readCheckpointedPartitionerFile prints out the following DEBUG message to the logs and returns the partitioner.
"},{"location":"rdd/ReliableCheckpointRDD/#sourceplaintext_2","title":"[source,plaintext]","text":""},{"location":"rdd/ReliableCheckpointRDD/#read-partitioner-from-partitionerfilepath","title":"Read partitioner from [partitionerFilePath]","text":"
In case of FileNotFoundException or any non-fatal exceptions, readCheckpointedPartitionerFile prints out a corresponding message to the logs and returns None.
readCheckpointedPartitionerFile is used when ReliableCheckpointRDD is requested for the <>.
== [[logging]] Logging
Enable ALL logging level for org.apache.spark.rdd.ReliableCheckpointRDD$ logger to see what happens inside.
ReliableRDDCheckpointData creates a subdirectory of the SparkContext.md#checkpointDir[application-wide checkpoint directory] for <> the given <>.
The name of the subdirectory uses the rdd:RDD.md#id[unique identifier] of the <>:"},{"location":"rdd/ReliableRDDCheckpointData/#sourceplaintext","title":"[source,plaintext]","text":""},{"location":"rdd/ReliableRDDCheckpointData/#rdd-id","title":"rdd-[id]","text":"
doCheckpoint rdd:ReliableCheckpointRDD.md#writeRDDToCheckpointDirectory[writes] the <> to the <> (that creates a new RDD).
With configuration-properties.md#spark.cleaner.referenceTracking.cleanCheckpoints[spark.cleaner.referenceTracking.cleanCheckpoints] configuration property enabled, doCheckpoint requests the SparkContext.md#cleaner[ContextCleaner] to core:ContextCleaner.md#registerRDDCheckpointDataForCleanup[registerRDDCheckpointDataForCleanup] for the new RDD.
In the end, doCheckpoint prints out the following INFO message to the logs and returns the new RDD.
"},{"location":"rdd/ReliableRDDCheckpointData/#sourceplaintext_1","title":"[source,plaintext]","text":""},{"location":"rdd/ReliableRDDCheckpointData/#done-checkpointing-rdd-id-to-cpdir-new-parent-is-rdd-id","title":"Done checkpointing RDD [id] to [cpDir], new parent is RDD [id]","text":"
doCheckpoint is part of the rdd:RDDCheckpointData.md#doCheckpoint[RDDCheckpointData] abstraction.
ShuffleDependency is a Dependency on the output of a ShuffleMapStage of a key-value RDD.
ShuffleDependency uses the RDD to know the number of (map-side/pre-shuffle) partitions and the Partitioner for the number of (reduce-size/post-shuffle) partitions.
ShuffleDependency takes the following to be created:
RDD (RDD[_ <: Product2[K, V]])
Partitioner
Serializer (default: SparkEnv.get.serializer)
Optional Key Ordering (default: undefined)
Optional Aggregator
mapSideCombine
ShuffleWriteProcessor
ShuffleDependency is created\u00a0when:
CoGroupedRDD is requested for the dependencies (for RDDs with different partitioners)
ShuffledRDD is requested for the dependencies
SubtractedRDD is requested for the dependencies (for an RDD with different partitioner)
ShuffleExchangeExec (Spark SQL) physical operator is requested to prepare a ShuffleDependency
When created, ShuffleDependency gets the shuffle id.
ShuffleDependency registers itself with the ShuffleManager and gets a ShuffleHandle (available as shuffleHandle). ShuffleDependency uses SparkEnv to access the ShuffleManager.
In the end, ShuffleDependency registers itself with the ContextCleaner (if configured) and the ShuffleDriverComponents.
ShuffleDependency registers itself with the ShuffleManager when created.
The ShuffleHandle is used when:
CoGroupedRDDs, ShuffledRDD, SubtractedRDD, and ShuffledRowRDD (Spark SQL) are requested to compute a partition (to get a ShuffleReader for a ShuffleDependency)
ShuffleMapTask is requested to run (to get a ShuffleWriter for a ShuffleDependency).
ShuffledRDD is an RDD of key-value pairs that represents a shuffle step in a RDD lineage (and indicates start of a new stage).
When requested to compute a partition, ShuffledRDD uses the one and only ShuffleDependency for a ShuffleHandle for a ShuffleReader (from the system ShuffleManager) that is used to read the (combined) key-value pairs.
getDependencies uses the user-specified Serializer, if defined, or requests the current SerializerManager for one.
getDependencies uses the mapSideCombine internal flag for the types of the keys and values (i.e. K and C or K and V when the flag is enabled or not, respectively).
In the end, getDependencies creates a single ShuffleDependency (with the previous RDD, the Partitioner, and the Serializer).
setMapSideCombine is used for PairRDDFunctions.combineByKeyWithClassTag transformation (which defaults to the flag enabled).
","text":"","tags":["DeveloperApi"]},{"location":"rdd/ShuffledRDD/#placement-preferences-of-partition","title":"Placement Preferences of Partition Signature
userSpecifiedSerializer is undefined (None) by default and can be changed using setSerializer method (that is used for PairRDDFunctions.combineByKeyWithClassTag transformation).
","text":"","tags":["DeveloperApi"]},{"location":"rdd/ShuffledRDD/#demos","title":"Demos","text":"","tags":["DeveloperApi"]},{"location":"rdd/ShuffledRDD/#shuffledrdd-and-coalesce","title":"ShuffledRDD and coalesce
val data = sc.parallelize(0 to 9)\nval coalesced = data.coalesce(numPartitions = 4, shuffle = true)\nscala> println(coalesced.toDebugString)\n(4) MapPartitionsRDD[9] at coalesce at <pastie>:75 []\n | CoalescedRDD[8] at coalesce at <pastie>:75 []\n | ShuffledRDD[7] at coalesce at <pastie>:75 []\n +-(16) MapPartitionsRDD[6] at coalesce at <pastie>:75 []\n | ParallelCollectionRDD[5] at parallelize at <pastie>:74 []\n
","text":"","tags":["DeveloperApi"]},{"location":"rdd/ShuffledRDD/#shuffledrdd-and-sortbykey","title":"ShuffledRDD and sortByKey
val data = sc.parallelize(0 to 9)\nval grouped = rdd.groupBy(_ % 2)\nval sorted = grouped.sortByKey(numPartitions = 2)\nscala> println(sorted.toDebugString)\n(2) ShuffledRDD[15] at sortByKey at <console>:74 []\n +-(4) ShuffledRDD[12] at groupBy at <console>:74 []\n +-(4) MapPartitionsRDD[11] at groupBy at <console>:74 []\n | MapPartitionsRDD[9] at coalesce at <pastie>:75 []\n | CoalescedRDD[8] at coalesce at <pastie>:75 []\n | ShuffledRDD[7] at coalesce at <pastie>:75 []\n +-(16) MapPartitionsRDD[6] at coalesce at <pastie>:75 []\n | ParallelCollectionRDD[5] at parallelize at <pastie>:74 []\n
You call SparkContext.setCheckpointDir(directory: String) to set the checkpoint directory - the directory where RDDs are checkpointed. The directory must be a HDFS path if running on a cluster. The reason is that the driver may attempt to reconstruct the checkpointed RDD from its own local file system, which is incorrect because the checkpoint files are actually on the executor machines.
You mark an RDD for checkpointing by calling RDD.checkpoint(). The RDD will be saved to a file inside the checkpoint directory and all references to its parent RDDs will be removed. This function has to be called before any job has been executed on this RDD.
NOTE: It is strongly recommended that a checkpointed RDD is persisted in memory, otherwise saving it on a file will require recomputation.
When an action is called on a checkpointed RDD, the following INFO message is printed out in the logs:
Done checkpointing RDD 5 to [path], new parent is RDD [id]\n
== [[local-checkpointing]] Local Checkpointing
localCheckpoint allows to truncate RDD lineage graph while skipping the expensive step of replicating the materialized data to a reliable distributed file system.
This is useful for RDDs with long lineages that need to be truncated periodically, e.g. GraphX.
Local checkpointing trades fault-tolerance for performance.
NOTE: The checkpoint directory set through SparkContext.setCheckpointDir is not used.
scala> rdd.checkpoint org.apache.spark.SparkException: Checkpoint directory has not been set in the SparkContext at org.apache.spark.rdd.RDD.checkpoint(RDD.scala:1599) ... 49 elided
sc.setCheckpointDir(\"/tmp/rdd-checkpoint\")
// Creates a subdirectory for this SparkContext $ ls /tmp/rdd-checkpoint/ fc21e1d1-3cd9-4d51-880f-58d1dd07f783
// Mark the RDD to checkpoint at the earliest action rdd.checkpoint
// Check out the checkpoint directory // You should find a directory for the checkpointed RDD, e.g. rdd-2 // The number of part-000* files is exactly the number of partitions $ ls -ltra /tmp/rdd-checkpoint/fc21e1d1-3cd9-4d51-880f-58d1dd07f783/rdd-2/part-000* | wc -l 16
Logical Execution Plan starts with the earliest RDDs (those with no dependencies on other RDDs or reference cached data) and ends with the RDD that produces the result of the action that has been called to execute.
Note
A logical plan (a DAG) is materialized and executed when SparkContext is requested to run a Spark job.
RDD Actions are RDD operations that produce concrete non-RDD values. They materialize a value in a Spark program. In other words, a RDD operation that returns a value of any type but RDD[T] is an action.
action: RDD => a value\n
NOTE: Actions are synchronous. You can use <> to release a calling thread while calling actions.
They trigger execution of <> to return values. Simply put, an action evaluates the RDD lineage graph.
You can think of actions as a valve and until action is fired, the data to be processed is not even in the pipes, i.e. transformations. Only actions can materialize the entire processing pipeline with real data.
aggregate
collect
count
countApprox*
countByValue*
first
fold
foreach
foreachPartition
max
min
reduce
saveAs* (e.g. saveAsTextFile, saveAsHadoopFile)
take
takeOrdered
takeSample
toLocalIterator
top
treeAggregate
treeReduce
Actions run jobs using SparkContext.runJob or directly DAGScheduler.runJob.
scala> :type words\n\nscala> words.count // <1>\nres0: Long = 502\n
TIP: You should cache RDDs you work with when you want to execute two or more actions on it for a better performance. Refer to spark-rdd-caching.md[RDD Caching and Persistence].
Before calling an action, Spark does closure/function cleaning (using SparkContext.clean) to make it ready for serialization and sending over the wire to executors. Cleaning can throw a SparkException if the computation cannot be cleaned.
NOTE: Spark uses ClosureCleaner to clean closures.
=== [[AsyncRDDActions]] AsyncRDDActions
AsyncRDDActions class offers asynchronous actions that you can use on RDDs (thanks to the implicit conversion rddToAsyncRDDActions in RDD class). The methods return a <>.
The following asynchronous methods are available:
countAsync
collectAsync
takeAsync
foreachAsync
foreachPartitionAsync
"},{"location":"rdd/spark-rdd-caching/","title":"Caching and Persistence","text":"
== RDD Caching and Persistence
Caching or persistence are optimisation techniques for (iterative and interactive) Spark computations. They help saving interim partial results so they can be reused in subsequent stages. These interim results as RDDs are thus kept in memory (default) or more solid storages like disk and/or replicated.
RDDs can be cached using <> operation. They can also be persisted using <> operation.
The difference between cache and persist operations is purely syntactic. cache is a synonym of persist or persist(MEMORY_ONLY), i.e. cache is merely persist with the default storage level MEMORY_ONLY.
NOTE: Due to the very small and purely syntactic difference between caching and persistence of RDDs the two terms are often used interchangeably and I will follow the \"pattern\" here.
RDDs can also be <> to remove RDD from a permanent storage like memory and/or disk.
persist marks a RDD for persistence using newLevel storage:StorageLevel.md[storage level].
You can only change the storage level once or persist reports an UnsupportedOperationException:
Cannot change storage level of an RDD after it was already assigned a level\n
NOTE: You can pretend to change the storage level of an RDD with already-assigned storage level only if the storage level is the same as it is currently assigned.
If the RDD is marked as persistent the first time, the RDD is core:ContextCleaner.md#registerRDDForCleanup[registered to ContextCleaner] (if available) and SparkContext.md#persistRDD[SparkContext].
The internal storageLevel attribute is set to the input newLevel storage level.
When called, unpersist prints the following INFO message to the logs:
INFO [RddName]: Removing RDD [id] from persistence list\n
It then calls SparkContext.md#unpersist[SparkContext.unpersistRDD(id, blocking)] and sets storage:StorageLevel.md[NONE storage level] as the current storage level.
RDDs have two types of operations: spark-rdd-transformations.md[transformations] and spark-rdd-actions.md[actions].
NOTE: Operators are also called operations.
=== Gotchas - things to watch for
Even if you don't access it explicitly it cannot be referenced inside a closure as it is serialized and carried around across executors.
See https://issues.apache.org/jira/browse/SPARK-5063
"},{"location":"rdd/spark-rdd-partitions/","title":"Partitions and Partitioning","text":"
== Partitions and Partitioning
=== Introduction
Depending on how you look at Spark (programmer, devop, admin), an RDD is about the content (developer's and data scientist's perspective) or how it gets spread out over a cluster (performance), i.e. how many partitions an RDD represents.
A partition (aka split) is a logical chunk of a large distributed data set.
How does the number of partitions map to the number of tasks? How to verify it?
Spark manages data using partitions that helps parallelize distributed data processing with minimal network traffic for sending data between executors.
By default, Spark tries to read data into an RDD from the nodes that are close to it. Since Spark usually accesses distributed partitioned data, to optimize transformation operations it creates partitions to hold the data chunks.
There is a one-to-one correspondence between how data is laid out in data storage like HDFS or Cassandra (it is partitioned for the same reasons).
Features:
size
number
partitioning scheme
node distribution
repartitioning
"},{"location":"rdd/spark-rdd-partitions/#how-does-the-mapping-between-partitions-and-tasks-correspond-to-data-locality-if-any","title":"How does the mapping between partitions and tasks correspond to data locality if any?","text":""},{"location":"rdd/spark-rdd-partitions/#tip","title":"[TIP]","text":"
Read the following documentations to learn what experts say on the topic:
https://databricks.gitbooks.io/databricks-spark-knowledge-base/content/performance_optimization/how_many_partitions_does_an_rdd_have.html[How Many Partitions Does An RDD Have?]
By default, a partition is created for each HDFS partition, which by default is 64MB (from http://spark.apache.org/docs/latest/programming-guide.html#external-datasets[Spark's Programming Guide]).
RDDs get partitioned automatically without programmer intervention. However, there are times when you'd like to adjust the size and number of partitions or the partitioning scheme according to the needs of your application.
You use def getPartitions: Array[Partition] method on a RDD to know the set of partitions in this RDD.
As noted in https://github.com/databricks/spark-knowledgebase/blob/master/performance_optimization/how_many_partitions_does_an_rdd_have.md#view-task-execution-against-partitions-using-the-ui[View Task Execution Against Partitions Using the UI]:
When a stage executes, you can see the number of partitions for a given stage in the Spark UI.
Start spark-shell and see it yourself!
scala> sc.parallelize(1 to 100).count\nres0: Long = 100\n
When you execute the Spark job, i.e. sc.parallelize(1 to 100).count, you should see the following in http://localhost:4040/jobs[Spark shell application UI].
.The number of partition as Total tasks in UI image::spark-partitions-ui-stages.png[align=\"center\"]
The reason for 8 Tasks in Total is that I'm on a 8-core laptop and by default the number of partitions is the number of all available cores.
$ sysctl -n hw.ncpu\n8\n
You can request for the minimum number of partitions, using the second input parameter to many transformations.
scala> sc.parallelize(1 to 100, 2).count\nres1: Long = 100\n
.Total tasks in UI shows 2 partitions image::spark-partitions-ui-stages-2-partitions.png[align=\"center\"]
You can always ask for the number of partitions using partitions method of a RDD:
scala> val ints = sc.parallelize(1 to 100, 4)\nints: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[1] at parallelize at <console>:24\n\nscala> ints.partitions.size\nres2: Int = 4\n
In general, smaller/more numerous partitions allow work to be distributed among more workers, but larger/fewer partitions allow work to be done in larger chunks, which may result in the work getting done more quickly as long as all workers are kept busy, due to reduced overhead.
Increasing partitions count will make each partition to have less data (or not at all!)
Spark can only run 1 concurrent task for every partition of an RDD, up to the number of cores in your cluster. So if you have a cluster with 50 cores, you want your RDDs to at least have 50 partitions (and probably http://spark.apache.org/docs/latest/tuning.html#level-of-parallelism[2-3x times that]).
As far as choosing a \"good\" number of partitions, you generally want at least as many as the number of executors for parallelism. You can get this computed value by calling sc.defaultParallelism.
Also, the number of partitions determines how many files get generated by actions that save RDDs to files.
The maximum size of a partition is ultimately limited by the available memory of an executor.
In the first RDD transformation, e.g. reading from a file using sc.textFile(path, partition), the partition parameter will be applied to all further transformations and actions on this RDD.
Partitions get redistributed among nodes whenever shuffle occurs. Repartitioning may cause shuffle to occur in some situations, but it is not guaranteed to occur in all cases. And it usually happens during action stage.
When creating an RDD by reading a file using rdd = SparkContext().textFile(\"hdfs://.../file.txt\") the number of partitions may be smaller. Ideally, you would get the same number of blocks as you see in HDFS, but if the lines in your file are too long (longer than the block size), there will be fewer partitions.
Preferred way to set up the number of partitions for an RDD is to directly pass it as the second input parameter in the call like rdd = sc.textFile(\"hdfs://.../file.txt\", 400), where 400 is the number of partitions. In this case, the partitioning makes for 400 splits that would be done by the Hadoop's TextInputFormat, not Spark and it would work much faster. It's also that the code spawns 400 concurrent tasks to try to load file.txt directly into 400 partitions.
It will only work as described for uncompressed files.
When using textFile with compressed files (file.txt.gz not file.txt or similar), Spark disables splitting that makes for an RDD with only 1 partition (as reads against gzipped files cannot be parallelized). In this case, to change the number of partitions you should do <>.
Some operations, e.g. map, flatMap, filter, don't preserve partitioning.
map, flatMap, filter operations apply a function to every partition.
"},{"location":"rdd/spark-rdd-partitions/#httpssparkapacheorgdocslatesttuninghtmltuning-spark-the-official-documentation-of-spark","title":"https://spark.apache.org/docs/latest/tuning.html[Tuning Spark] (the official documentation of Spark)","text":""},{"location":"rdd/spark-rdd-partitions/#source-scala","title":"[source, scala]","text":""},{"location":"rdd/spark-rdd-partitions/#repartitionnumpartitions-intimplicit-ord-orderingt-null-rddt","title":"repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T]","text":"
repartition is <> with numPartitions and shuffle enabled.
With the following computation you can see that repartition(5) causes 5 tasks to be started using NODE_LOCAL data locality.
scala> lines.repartition(5).count\n...\n15/10/07 08:10:00 INFO DAGScheduler: Submitting 5 missing tasks from ResultStage 7 (MapPartitionsRDD[19] at repartition at <console>:27)\n15/10/07 08:10:00 INFO TaskSchedulerImpl: Adding task set 7.0 with 5 tasks\n15/10/07 08:10:00 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 17, localhost, partition 0,NODE_LOCAL, 2089 bytes)\n15/10/07 08:10:00 INFO TaskSetManager: Starting task 1.0 in stage 7.0 (TID 18, localhost, partition 1,NODE_LOCAL, 2089 bytes)\n15/10/07 08:10:00 INFO TaskSetManager: Starting task 2.0 in stage 7.0 (TID 19, localhost, partition 2,NODE_LOCAL, 2089 bytes)\n15/10/07 08:10:00 INFO TaskSetManager: Starting task 3.0 in stage 7.0 (TID 20, localhost, partition 3,NODE_LOCAL, 2089 bytes)\n15/10/07 08:10:00 INFO TaskSetManager: Starting task 4.0 in stage 7.0 (TID 21, localhost, partition 4,NODE_LOCAL, 2089 bytes)\n...\n
You can see a change after executing repartition(1) causes 2 tasks to be started using PROCESS_LOCAL data locality.
scala> lines.repartition(1).count\n...\n15/10/07 08:14:09 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[20] at repartition at <console>:27)\n15/10/07 08:14:09 INFO TaskSchedulerImpl: Adding task set 8.0 with 2 tasks\n15/10/07 08:14:09 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 22, localhost, partition 0,PROCESS_LOCAL, 2058 bytes)\n15/10/07 08:14:09 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 23, localhost, partition 1,PROCESS_LOCAL, 2058 bytes)\n...\n
Please note that Spark disables splitting for compressed files and creates RDDs with only 1 partition. In such cases, it's helpful to use sc.textFile('demo.gz') and do repartitioning using rdd.repartition(100) as follows:
With the lines, you end up with rdd to be exactly 100 partitions of roughly equal in size.
rdd.repartition(N) does a shuffle to split data to match N ** partitioning is done on round robin basis
TIP: If partitioning scheme doesn't work for you, you can write your own custom partitioner.
TIP: It's useful to get familiar with https://hadoop.apache.org/docs/current/api/org/apache/hadoop/mapred/TextInputFormat.html[Hadoop's TextInputFormat].
The coalesce transformation is used to change the number of partitions. It can trigger shuffling depending on the shuffle flag (disabled by default, i.e. false).
In the following sample, you parallelize a local 10-number sequence and coalesce it first without and then with shuffling (note the shuffle parameter being false and true, respectively).
Tip
Use toDebugString to check out the RDD lineage graph.
scala> val rdd = sc.parallelize(0 to 10, 8)\nrdd: org.apache.spark.rdd.RDD[Int] = ParallelCollectionRDD[0] at parallelize at <console>:24\n\nscala> rdd.partitions.size\nres0: Int = 8\n\nscala> rdd.coalesce(numPartitions=8, shuffle=false) // <1>\nres1: org.apache.spark.rdd.RDD[Int] = CoalescedRDD[1] at coalesce at <console>:27\n\nscala> res1.toDebugString\nres2: String =\n(8) CoalescedRDD[1] at coalesce at <console>:27 []\n | ParallelCollectionRDD[0] at parallelize at <console>:24 []\n\nscala> rdd.coalesce(numPartitions=8, shuffle=true)\nres3: org.apache.spark.rdd.RDD[Int] = MapPartitionsRDD[5] at coalesce at <console>:27\n\nscala> res3.toDebugString\nres4: String =\n(8) MapPartitionsRDD[5] at coalesce at <console>:27 []\n | CoalescedRDD[4] at coalesce at <console>:27 []\n | ShuffledRDD[3] at coalesce at <console>:27 []\n +-(8) MapPartitionsRDD[2] at coalesce at <console>:27 []\n | ParallelCollectionRDD[0] at parallelize at <console>:24 []\n
<1> shuffle is false by default and it's explicitly used here for demo purposes. Note the number of partitions that remains the same as the number of partitions in the source RDD rdd."},{"location":"rdd/spark-rdd-transformations/","title":"Transformations -- Lazy Operations on RDD (to Create One or More RDDs)","text":"
Transformations are lazy operations on an rdd:RDD.md[RDD] that create one or many new RDDs.
// T and U are Scala types\ntransformation: RDD[T] => RDD[U]\ntransformation: RDD[T] => Seq[RDD[U]]\n
In other words, transformations are functions that take an RDD as the input and produce one or many RDDs as the output. Transformations do not change the input RDD (since rdd:index.md#introduction[RDDs are immutable] and hence cannot be modified), but produce one or more new RDDs by applying the computations they represent.
By applying transformations you incrementally build a RDD lineage with all the parent RDDs of the final RDD(s).
Transformations are lazy, i.e. are not executed immediately. Only after calling an action are transformations executed.
After executing a transformation, the result RDD(s) will always be different from their parents and can be smaller (e.g. filter, count, distinct, sample), bigger (e.g. flatMap, union, cartesian) or the same size (e.g. map).
CAUTION: There are transformations that may trigger jobs, e.g. sortBy, <>, etc.
.From SparkContext by transformations to the result image::rdd-sparkcontext-transformations-action.png[align=\"center\"]
Certain transformations can be pipelined which is an optimization that Spark uses to improve performance of computations.
Narrow transformations are the result of map, filter and such that is from the data from a single partition only, i.e. it is self-sustained.
An output RDD has partitions with records that originate from a single partition in the parent RDD. Only a limited subset of partitions used to calculate the result.
Spark groups narrow transformations as a stage which is called pipelining.
=== [[wide-transformations]] Wide Transformations
Wide transformations are the result of groupByKey and reduceByKey. The data required to compute the records in a single partition may reside in many partitions of the parent RDD.
NOTE: Wide transformations are also called shuffle transformations as they may or may not depend on a shuffle.
All of the tuples with the same key must end up in the same partition, processed by the same task. To satisfy these operations, Spark must execute a RDD shuffle, which transfers data across cluster and results in a new stage with a new set of partitions.
= Status REST API -- Monitoring Spark Applications Using REST API
Status REST API is a collection of REST endpoints under /api/v1 URI path in the spark-api-UIRoot.md[root containers for application UI information]:
[[SparkUI]] spark-webui-SparkUI.md[SparkUI] - Application UI for an active Spark application (i.e. a Spark application that is still running)
[[HistoryServer]] spark-history-server:HistoryServer.md[HistoryServer] - Application UI for active and completed Spark applications (i.e. Spark applications that are still running or have already finished)
Status REST API uses spark-api-ApiRootResource.md[ApiRootResource] main resource class that registers /api/v1 URI <>.
| [[applications]] applications | [[ApplicationListResource]] Delegates to the spark-api-ApplicationListResource.md[ApplicationListResource] resource class
| [[applications_appId]] applications/\\{appId} | [[OneApplicationResource]] Delegates to the spark-api-OneApplicationResource.md[OneApplicationResource] resource class
| [[version]] version | Creates a VersionInfo with the current version of Spark |===
Status REST API uses the following components:
https://jersey.github.io/[Jersey RESTful Web Services framework] with support for the https://github.com/jax-rs[Java API for RESTful Web Services] (JAX-RS API)
https://www.eclipse.org/jetty/[Eclipse Jetty] as the lightweight HTTP server and the https://jcp.org/en/jsr/detail?id=369[Java Servlet] container
== [[ApiRootResource]] ApiRootResource -- /api/v1 URI Handler
ApiRootResource is the spark-api-ApiRequestContext.md[ApiRequestContext] for the /v1 URI path.
ApiRootResource uses @Path(\"/v1\") annotation at the class level. It is a partial URI path template relative to the base URI of the server on which the resource is deployed, the context root of the application, and the URL pattern to which the JAX-RS runtime responds.
TIP: Learn more about @Path annotation in https://docs.oracle.com/cd/E19798-01/821-1841/6nmq2cp26/index.html[The @Path Annotation and URI Path Templates].
ApiRootResource <> the /api/* context handler (with the REST resources and providers in org.apache.spark.status.api.v1 package).
With the @Path(\"/v1\") annotation and after <> the /api/* context handler, ApiRootResource serves HTTP requests for <> under the /api/v1 URI paths for spark-webui-SparkUI.md#initialize[SparkUI] and spark-history-server:HistoryServer.md#initialize[HistoryServer].
ApiRootResource gives the metrics of a Spark application in JSON format (using JAX-RS API).
[[applications]] applications [[ApplicationListResource]] Delegates to the spark-api-ApplicationListResource.md[ApplicationListResource] resource class [[applications_appId]] applications/\\{appId} [[OneApplicationResource]] Delegates to the spark-api-OneApplicationResource.md[OneApplicationResource] resource class
| [[version]] version | GET | Creates a VersionInfo with the current version of Spark |===
getServletHandler creates a Jetty ServletContextHandler for /api context path.
NOTE: The Jetty ServletContextHandler created does not support HTTP sessions as REST API is stateless.
getServletHandler creates a Jetty ServletHolder with the resources and providers in org.apache.spark.status.api.v1 package. It then registers the ServletHolder to serve /* context path (under the ServletContextHandler for /api).
getServletHandler requests UIRootFromServletContext to spark-api-UIRootFromServletContext.md#setUiRoot[setUiRoot] with the ServletContextHandler and the input spark-api-UIRoot.md[UIRoot].
NOTE: getServletHandler is used when spark-webui-SparkUI.md#initialize[SparkUI] and spark-history-server:HistoryServer.md#initialize[HistoryServer] are requested to initialize.
== [[ApplicationListResource]] ApplicationListResource -- applications URI Handler
ApplicationListResource is a spark-api-ApiRequestContext.md[ApiRequestContext] that spark-api-ApiRootResource.md#applications[ApiRootResource] uses to handle <> URI path.
OneApplicationAttemptResource is a spark-api-AbstractApplicationResource.md[AbstractApplicationResource] (and so a spark-api-ApiRequestContext.md[ApiRequestContext] indirectly).
OneApplicationAttemptResource is used when AbstractApplicationResource is requested to spark-api-AbstractApplicationResource.md#applicationAttempt[applicationAttempt].
// start spark-shell\n// there should be a single Spark application -- the spark-shell itself\n// CAUTION: FIXME Demo of OneApplicationAttemptResource in Action\n
getAttempt requests the spark-api-ApiRequestContext.md#uiRoot[UIRoot] for the spark-api-UIRoot.md#getApplicationInfo[application info] (given the spark-api-BaseAppResource.md#appId[appId]) and finds the spark-api-BaseAppResource.md#attemptId[attemptId] among the available attempts.
NOTE: spark-api-BaseAppResource.md#appId[appId] and spark-api-BaseAppResource.md#attemptId[attemptId] are path parameters.
In the end, getAttempt returns the ApplicationAttemptInfo if available or reports a NotFoundException:
== [[OneApplicationResource]] OneApplicationResource -- applications/appId URI Handler
OneApplicationResource is a spark-api-AbstractApplicationResource.md[AbstractApplicationResource] (and so a spark-api-ApiRequestContext.md[ApiRequestContext] indirectly) that spark-api-ApiRootResource.md#applications_appId[ApiRootResource] uses to handle <> URI path.
getApp requests the spark-api-ApiRequestContext.md#uiRoot[UIRoot] for the spark-api-UIRoot.md#getApplicationInfo[application info] (given the spark-api-BaseAppResource.md#appId[appId]).
In the end, getApp returns the ApplicationInfo if available or reports a NotFoundException:
| spark-history-server:HistoryServer.md[HistoryServer] | [[HistoryServer]] Application UI for active and completed Spark applications (i.e. Spark applications that are still running or have already finished)
| spark-webui-SparkUI.md[SparkUI] | [[SparkUI]] Application UI for an active Spark application (i.e. a Spark application that is still running) |===
UIRootFromServletContext manages the current <> object in a Jetty ContextHandler.
[[attribute]] UIRootFromServletContext uses its canonical name for the context attribute that is used to <> or <> the current spark-api-UIRoot.md[UIRoot] object (in Jetty's ContextHandler).
NOTE: https://www.eclipse.org/jetty/javadoc/current/org/eclipse/jetty/server/handler/ContextHandler.html[ContextHandler] is the environment for multiple Jetty Handlers, e.g. URI context path, class loader, static resource base.
In essence, UIRootFromServletContext is simply a \"bridge\" between two worlds, Spark's spark-api-UIRoot.md[UIRoot] and Jetty's ContextHandler.
NOTE: setUiRoot is used exclusively when ApiRootResource is requested to spark-api-ApiRootResource.md#getServletHandler[register /api/* context handler].
NettyRpcEnv is an RpcEnv that uses Netty (\"an asynchronous event-driven network application framework for rapid development of maintainable high performance protocol servers & clients\").
create creates a JavaSerializerInstance (using a JavaSerializer).
Note
KryoSerializer is not supported.
create creates a rpc:NettyRpcEnv.md[] with the JavaSerializerInstance. create uses the given rpc:RpcEnvConfig.md[] for the rpc:RpcEnvConfig.md#advertiseAddress[advertised address], rpc:RpcEnvConfig.md#securityManager[SecurityManager] and rpc:RpcEnvConfig.md#numUsableCores[number of CPU cores].
create returns the NettyRpcEnv unless the rpc:RpcEnvConfig.md#clientMode[clientMode] is turned off (server mode).
In server mode, create attempts to start the NettyRpcEnv on a given port. create uses the given rpc:RpcEnvConfig.md[] for the rpc:RpcEnvConfig.md#port[port], rpc:RpcEnvConfig.md#bindAddress[bind address], and rpc:RpcEnvConfig.md#name[name]. With the port, the NettyRpcEnv is requested to rpc:NettyRpcEnv.md#startServer[start a server].
create is part of the rpc:RpcEnvFactory.md#create[RpcEnvFactory] abstraction.
RpcEndpointRef is a reference to a rpc:RpcEndpoint.md[RpcEndpoint] in a rpc:index.md[RpcEnv].
RpcEndpointRef is a serializable entity and so you can send it over a network or save it for later use (it can however be deserialized using the owning RpcEnv only).
A RpcEndpointRef has <> (a Spark URL), and a name.
You can send asynchronous one-way messages to the corresponding RpcEndpoint using <> method.
You can send a semi-synchronous message, i.e. \"subscribe\" to be notified when a response arrives, using ask method. You can also block the current calling thread for a response using askWithRetry method.
spark.rpc.numRetries (default: 3) - the number of times to retry connection attempts.
spark.rpc.retry.wait (default: 3s) - the number of milliseconds to wait on each retry.
It also uses rpc:index.md#endpoint-lookup-timeout[lookup timeouts].
RpcEnv uses the default lookup timeout for...FIXME
When a remote endpoint is resolved, a local RPC environment connects to the remote one (endpoint lookup). To configure the time needed for the endpoint lookup you can use the following settings.
It is a prioritized list of lookup timeout properties (the higher on the list, the more important):
[[creating-instance]] RpcEnvConfig is a configuration of an rpc:RpcEnv.md[]:
[[conf]] SparkConf.md[]
[[name]] System Name
[[bindAddress]] Bind Address
[[advertiseAddress]] Advertised Address
[[port]] Port
[[securityManager]] SecurityManager
[[numUsableCores]] Number of CPU cores
<>
RpcEnvConfig is created when RpcEnv utility is used to rpc:RpcEnv.md#create[create an RpcEnv] (using rpc:RpcEnvFactory.md[]).
== [[clientMode]] Client Mode
When an RPC Environment is initialized core:SparkEnv.md#createDriverEnv[as part of the initialization of the driver] or core:SparkEnv.md#createExecutorEnv[executors] (using RpcEnv.create), clientMode is false for the driver and true for executors.
Copied (almost verbatim) from https://issues.apache.org/jira/browse/SPARK-10997[SPARK-10997 Netty-based RPC env should support a \"client-only\" mode] and the https://github.com/apache/spark/commit/71d1c907dec446db566b19f912159fd8f46deb7d[commit]:
\"Client mode\" means the RPC env will not listen for incoming connections.
This allows certain processes in the Spark stack (such as Executors or tha YARN client-mode AM) to act as pure clients when using the netty-based RPC backend, reducing the number of sockets Spark apps need to use and also the number of open ports.
The AM connects to the driver in \"client mode\", and that connection is used for all driver -- AM communication, and so the AM is properly notified when the connection goes down.
In \"general\", non-YARN case, clientMode flag is therefore enabled for executors and disabled for the driver.
In Spark on YARN in client deploy mode, clientMode flag is however enabled explicitly when Spark on YARN's spark-yarn-applicationmaster.md#runExecutorLauncher-sparkYarnAM[ApplicationMaster] creates the sparkYarnAM RPC Environment.
rpc:NettyRpcEnvFactory.md[] is the default and only known RpcEnvFactory in Apache Spark (as of https://github.com/apache/spark/commit/4f5a24d7e73104771f233af041eeba4f41675974[this commit]).
Netty-based RPC Environment is created by NettyRpcEnvFactory when rpc:index.md#settings[spark.rpc] is netty or org.apache.spark.rpc.netty.NettyRpcEnvFactory.
NettyRpcEnv is only started on spark-driver.md[the driver]. See <>.
The default port to listen to is 7077.
When NettyRpcEnv starts, the following INFO message is printed out in the logs:
Successfully started service 'NettyRpcEnv' on port 0.\n
== [[thread-pools]] Thread Pools
=== shuffle-server-ID
EventLoopGroup uses a daemon thread pool called shuffle-server-ID, where ID is a unique integer for NioEventLoopGroup (NIO) or EpollEventLoopGroup (EPOLL) for the Shuffle server.
CAUTION: FIXME Review Netty's NioEventLoopGroup.
CAUTION: FIXME Where are SO_BACKLOG, SO_RCVBUF, SO_SNDBUF channel options used?
=== dispatcher-event-loop-ID
NettyRpcEnv's Dispatcher uses the daemon fixed thread pool with <> threads.
Thread names are formatted as dispatcher-event-loop-ID, where ID is a unique, sequentially assigned integer.
It starts the message processing loop on all of the threads.
=== netty-rpc-env-timeout
NettyRpcEnv uses the daemon single-thread scheduled thread pool netty-rpc-env-timeout.
NettyRpcEnv uses the daemon cached thread pool with up to <> threads.
Thread names are formatted as netty-rpc-connection-ID, where ID is a unique, sequentially assigned integer.
== [[settings]] Settings
The Netty-based implementation uses the following properties:
spark.rpc.io.mode (default: NIO) - NIO or EPOLL for low-level IO. NIO is always available, while EPOLL is only available on Linux. NIO uses io.netty.channel.nio.NioEventLoopGroup while EPOLLio.netty.channel.epoll.EpollEventLoopGroup.
spark.rpc.io.threads (default: 0; maximum: 8) - the number of threads to use for the Netty client and server thread pools. ** spark.shuffle.io.serverThreads (default: the value of spark.rpc.io.threads) ** spark.shuffle.io.clientThreads (default: the value of spark.rpc.io.threads)
spark.rpc.netty.dispatcher.numThreads (default: the number of processors available to JVM)
spark.rpc.connect.threads (default: 64) - used in cluster mode to communicate with a remote RPC endpoint
spark.port.maxRetries (default: 16 or 100 for testing when spark.testing is set) controls the maximum number of binding attempts/retries to a port before giving up.
== [[endpoints]] Endpoints
endpoint-verifier (RpcEndpointVerifier) - a rpc:RpcEndpoint.md[RpcEndpoint] for remote RpcEnvs to query whether an RpcEndpoint exists or not. It uses Dispatcher that keeps track of registered endpoints and responds true/false to CheckExistence message.
endpoint-verifier is used to check out whether a given endpoint exists or not before the endpoint's reference is given back to clients.
One use case is when an spark-standalone.md#AppClient[AppClient connects to standalone Masters] before it registers the application it acts for.
CAUTION: FIXME Who'd like to use endpoint-verifier and how?
== Message Dispatcher
A message dispatcher is responsible for routing RPC messages to the appropriate endpoint(s).
It uses the daemon fixed thread pool dispatcher-event-loop with spark.rpc.netty.dispatcher.numThreads threads for dispatching messages.
Every partition of a Stage is transformed into a Task (ShuffleMapTask or ResultTask for ShuffleMapStage and ResultStage, respectively).
Submitting a stage can therefore trigger execution of a series of dependent parent stages.
When a Spark job is submitted, a new stage is created (they can be created from scratch or linked to, i.e. shared, if other jobs use them already).
DAGScheduler splits up a job into a collection of Stages. A Stage contains a sequence of narrow transformations that can be completed without shuffling data set, separated at shuffle boundaries (where shuffle occurs). Stages are thus a result of breaking the RDD graph at shuffle boundaries.
Shuffle boundaries introduce a barrier where stages/tasks must wait for the previous stage to finish before they fetch map outputs.
ActiveJob (job, action job) is a top-level work item (computation) submitted to DAGScheduler for execution (usually to compute the result of an RDD action).
Executing a job is equivalent to computing the partitions of the RDD an action has been executed upon. The number of partitions (numPartitions) to compute in a job depends on the type of a stage (ResultStage or ShuffleMapStage).
A job starts with a single target RDD, but can ultimately include other RDDs that are all part of RDD lineage.
The parent stages are always ShuffleMapStages.
Note
Not always all partitions have to be computed for ResultStages (e.g. for actions like first() and lookup()).
CoarseGrainedSchedulerBackend is a base SchedulerBackend for coarse-grained schedulers.
CoarseGrainedSchedulerBackend is an ExecutorAllocationClient.
CoarseGrainedSchedulerBackend is responsible for requesting resources from a cluster manager for executors that it in turn uses to launch tasks (on CoarseGrainedExecutorBackend).
CoarseGrainedSchedulerBackend holds executors for the duration of the Spark job rather than relinquishing executors whenever a task is done and asking the scheduler to launch a new executor for each new task.
CoarseGrainedSchedulerBackend registers CoarseGrainedScheduler RPC Endpoint that executors use for RPC communication.
Note
Active executors are executors that are not pending to be removed or lost.
maxNumConcurrentTasks is part of the SchedulerBackend abstraction.
maxNumConcurrentTasks uses the Available Executors registry to find out about available ResourceProfiles, total number of CPU cores and ExecutorResourceInfos of every active executor.
In the end, maxNumConcurrentTasks calculates the available (parallel) slots for the given ResourceProfile (and given the available executor resources).
sufficientResourcesRegistered is true (and is supposed to be overriden by custom CoarseGrainedSchedulerBackends).
","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#minimum-resources-available-ratio","title":"Minimum Resources Available Ratio
minRegisteredRatio: Double\n
minRegisteredRatio is a ratio of the minimum resources available to the total expected resources for the CoarseGrainedSchedulerBackend to be ready for scheduling tasks (for execution).
minRegisteredRatio uses spark.scheduler.minRegisteredResourcesRatio configuration property if defined or defaults to 0.0.
minRegisteredRatio can be between 0.0 and 1.0 (inclusive).
minRegisteredRatio is used when:
CoarseGrainedSchedulerBackend is requested to isReady
StandaloneSchedulerBackend is requested to sufficientResourcesRegistered
KubernetesClusterSchedulerBackend is requested to sufficientResourcesRegistered
MesosCoarseGrainedSchedulerBackend is requested to sufficientResourcesRegistered
YarnSchedulerBackend is requested to sufficientResourcesRegistered
makeOffers takes the active executors (out of the <> internal registry) and creates WorkerOffer resource offers for each (one per executor with the executor's id, host and free cores).
CAUTION: Only free cores are considered in making offers. Memory is not! Why?!
It then requests TaskSchedulerImpl.md#resourceOffers[TaskSchedulerImpl to process the resource offers] to create a collection of TaskDescription collections that it in turn uses to launch tasks.
requestExecutors is a \"decorator\" method that ultimately calls a cluster-specific doRequestTotalExecutors method and returns whether the request was acknowledged or not (it is assumed false by default).
requestExecutors method is part of the ExecutorAllocationClient abstraction.
When called, you should see the following INFO message followed by DEBUG message in the logs:
Requesting [numAdditionalExecutors] additional executor(s) from the cluster manager\nNumber of pending executors is now [numPendingExecutors]\n
<> is increased by the input numAdditionalExecutors.
requestExecutors requests executors from a cluster manager (that reflects the current computation needs). The \"new executor total\" is a sum of the internal <> and <> decreased by the <>.
If numAdditionalExecutors is negative, a IllegalArgumentException is thrown:
Attempted to request a negative number of additional executor(s) [numAdditionalExecutors] from the cluster manager. Please specify a positive number!\n
NOTE: It is a final method that no other scheduler backends could customize further.
NOTE: The method is a synchronized block that makes multiple concurrent requests be handled in a serial fashion, i.e. one by one.
","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#requesting-exact-number-of-executors","title":"Requesting Exact Number of Executors
requestTotalExecutors is a \"decorator\" method that ultimately calls a cluster-specific doRequestTotalExecutors method and returns whether the request was acknowledged or not (it is assumed false by default).
requestTotalExecutors is part of the ExecutorAllocationClient abstraction.
It sets the internal <> and <> registries. It then calculates the exact number of executors which is the input numExecutors and the <> decreased by the number of <>.
If numExecutors is negative, a IllegalArgumentException is thrown:
Attempted to request a negative number of executor(s) [numExecutors] from the cluster manager. Please specify a positive number!\n
NOTE: It is a final method that no other scheduler backends could customize further.
NOTE: The method is a synchronized block that makes multiple concurrent requests be handled in a serial fashion, i.e. one by one.
","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#finding-default-level-of-parallelism","title":"Finding Default Level of Parallelism
defaultParallelism(): Int\n
defaultParallelism is part of the SchedulerBackend abstraction.
defaultParallelism is spark.default.parallelism configuration property if defined.
Otherwise, defaultParallelism is the maximum of totalCoreCount or 2.
killTask is part of the SchedulerBackend abstraction.
killTask simply sends a KillTask message to <>.","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#stopping-all-executors","title":"Stopping All Executors
stopExecutors sends a blocking <> message to <> (if already initialized).
NOTE: It is called exclusively while CoarseGrainedSchedulerBackend is <>.
You should see the following INFO message in the logs:
Shutting down all executors\n
","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#reset-state","title":"Reset State
reset resets the internal state:
Sets <> to 0
Clears executorsPendingToRemove
Sends a blocking <> message to <> for every executor (in the internal executorDataMap) to inform it about SlaveLost with the message: +
Stale executor after cluster manager re-registered.\n
reset is a method that is defined in CoarseGrainedSchedulerBackend, but used and overriden exclusively by yarn/spark-yarn-yarnschedulerbackend.md[YarnSchedulerBackend].
NOTE: It is called by subclasses spark-standalone.md#SparkDeploySchedulerBackend[SparkDeploySchedulerBackend], spark-mesos/spark-mesos.md#CoarseMesosSchedulerBackend[CoarseMesosSchedulerBackend], and yarn/spark-yarn-yarnschedulerbackend.md[YarnSchedulerBackend].
When <>, it registers CoarseGrainedScheduler RPC endpoint to be the driver's communication endpoint.
driverEndpoint is a DriverEndpoint.
Note
CoarseGrainedSchedulerBackend is created while SparkContext is being created that in turn lives inside a Spark driver. That explains the name driverEndpoint (at least partially).
It is called standalone scheduler's driver endpoint internally.
It tracks:
It uses driver-revive-thread daemon single-thread thread pool for ...FIXME
CAUTION: FIXME A potential issue with driverEndpoint.asInstanceOf[NettyRpcEndpointRef].toURI - doubles spark:// prefix.
start is part of the SchedulerBackend abstraction.
start takes all spark.-prefixed properties and registers the <CoarseGrainedScheduler RPC endpoint>> (backed by DriverEndpoint ThreadSafeRpcEndpoint).
NOTE: start uses <> to access the current SparkContext.md[SparkContext] and in turn SparkConf.md[SparkConf].
NOTE: start uses <> that was given when <CoarseGrainedSchedulerBackend was created>>.","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#checking-if-sufficient-compute-resources-available-or-waiting-time-passedmethod","title":"Checking If Sufficient Compute Resources Available Or Waiting Time PassedMethod
isReady(): Boolean\n
isReady is part of the SchedulerBackend abstraction.
isReady allows to delay task launching until <> or <> passes.
Internally, isReady <>.
NOTE: <> by default responds that sufficient resources are available.
If the <>, you should see the following INFO message in the logs and isReady is positive.
SchedulerBackend is ready for scheduling beginning after reached minRegisteredResourcesRatio: [minRegisteredRatio]\n
If there are no sufficient resources available yet (the above requirement does not hold), isReady checks whether the time since <> passed <> to give a way to launch tasks (even when <> not being reached yet).
You should see the following INFO message in the logs and isReady is positive.
SchedulerBackend is ready for scheduling beginning after waiting maxRegisteredResourcesWaitingTime: [maxRegisteredWaitingTimeMs](ms)\n
Otherwise, when <> and <> has not elapsed, isReady is negative.","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#reviving-resource-offers","title":"Reviving Resource Offers
reviveOffers(): Unit\n
reviveOffers is part of the SchedulerBackend abstraction.
reviveOffers simply sends a ReviveOffers message to CoarseGrainedSchedulerBackend RPC endpoint.
createDriverEndpointRef <> and rpc:index.md#setupEndpoint[registers it] as CoarseGrainedScheduler.
createDriverEndpointRef is used when CoarseGrainedSchedulerBackend is requested to <>.","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#checking-whether-executor-is-active","title":"Checking Whether Executor is Active
isExecutorActive(\n id: String): Boolean\n
isExecutorActive is part of the ExecutorAllocationClient abstraction.
isExecutorActive...FIXME
","text":""},{"location":"scheduler/CoarseGrainedSchedulerBackend/#requesting-executors-from-cluster-manager","title":"Requesting Executors from Cluster Manager
The introduction that follows was highly influenced by the scaladoc of org.apache.spark.scheduler.DAGScheduler. As DAGScheduler is a private class it does not appear in the official API documentation. You are strongly encouraged to read the sources and only then read this and the related pages afterwards.
DAGScheduler is the scheduling layer of Apache Spark that implements stage-oriented scheduling using Jobs and Stages.
DAGScheduler transforms a logical execution plan (RDD lineage of dependencies built using RDD transformations) to a physical execution plan (using stages).
After an action has been called on an RDD, SparkContext hands over a logical plan to DAGScheduler that it in turn translates to a set of stages that are submitted as TaskSets for execution.
DAGScheduler works solely on the driver and is created as part of SparkContext's initialization (right after TaskScheduler and SchedulerBackend are ready).
DAGScheduler does three things in Spark:
Computes an execution DAG (DAG of stages) for a job
Determines the preferred locations to run each task on
Handles failures due to shuffle output files being lost
DAGScheduler computes a directed acyclic graph (DAG) of stages for each job, keeps track of which RDDs and stage outputs are materialized, and finds a minimal schedule to run jobs. It then submits stages to TaskScheduler.
In addition to coming up with the execution DAG, DAGScheduler also determines the preferred locations to run each task on, based on the current cache status, and passes the information to TaskScheduler.
DAGScheduler tracks which rdd/spark-rdd-caching.md[RDDs are cached (or persisted)] to avoid \"recomputing\" them, i.e. redoing the map side of a shuffle. DAGScheduler remembers what ShuffleMapStage.md[ShuffleMapStage]s have already produced output files (that are stored in BlockManagers).
DAGScheduler is only interested in cache location coordinates, i.e. host and executor id, per partition of a RDD.
Furthermore, it handles failures due to shuffle output files being lost, in which case old stages may need to be resubmitted. Failures within a stage that are not caused by shuffle file loss are handled by the TaskScheduler itself, which will retry each task a small number of times before cancelling the whole stage.
DAGScheduler uses an event queue architecture in which a thread can post DAGSchedulerEvent events, e.g. a new job or stage being submitted, that DAGScheduler reads and executes sequentially. See the section Event Bus.
DAGScheduler runs stages in topological order.
DAGScheduler uses SparkContext, TaskScheduler, LiveListenerBus.md[], MapOutputTracker.md[MapOutputTracker] and storage:BlockManager.md[BlockManager] for its services. However, at the very minimum, DAGScheduler takes a SparkContext only (and requests SparkContext for the other services).
When DAGScheduler schedules a job as a result of rdd/index.md#actions[executing an action on a RDD] or calling SparkContext.runJob() method directly, it spawns parallel tasks to compute (partial) results per partition.
submitMapStage requests the given ShuffleDependency for the RDD.
submitMapStage gets the job ID and increments it (for future submissions).
submitMapStage creates a JobWaiter to wait for a MapOutputStatistics. The JobWaiter waits for 1 task and, when completed successfully, executes the given callback function with the computed MapOutputStatistics.
In the end, submitMapStage posts a MapStageSubmitted and returns the JobWaiter.
Used when:
SparkContext is requested to submit a MapStage for execution
submitJob increments the nextJobId internal counter.
submitJob creates a JobWaiter for the (number of) partitions and the given resultHandler function.
submitJob requests the DAGSchedulerEventProcessLoop to post a JobSubmitted.
In the end, submitJob returns the JobWaiter.
For empty partitions (no partitions to compute), submitJob requests the LiveListenerBus to post a SparkListenerJobStart and SparkListenerJobEnd (with JobSucceeded result marker) events and returns a JobWaiter with no tasks to wait for.
submitJob throws an IllegalArgumentException when the partitions indices are not among the partitions of the given RDD:
Attempting to access a non-existent partition: [p]. Total number of partitions: [maxPartitions]\n
Adds a new ActiveJob when requested to handle JobSubmitted or MapStageSubmitted events
Removes an ActiveJob when requested to clean up after an ActiveJob and independent stages.
Removes all ActiveJobs when requested to doCancelAllJobs.
DAGScheduler uses ActiveJobs registry when requested to handle JobGroupCancelled or TaskCompletion events, to cleanUpAfterSchedulerStop and to abort a stage.
The number of ActiveJobs is available using job.activeJobs performance metric.
","text":""},{"location":"scheduler/DAGScheduler/#createResultStage","title":"Creating ResultStage for RDD
createResultStage creates a new ResultStage for the ShuffleDependencies and ResourceProfiles of the given RDD.
createResultStage finds the ShuffleDependencies and ResourceProfiles for the given RDD.
createResultStage merges the ResourceProfiles for the Stage (if enabled or reports an exception).
createResultStage does the following checks (that may report violations and break the execution):
checkBarrierStageWithDynamicAllocation
checkBarrierStageWithNumSlots
checkBarrierStageWithRDDChainPattern
createResultStage getOrCreateParentStages (with the ShuffleDependencyies and the given jobId).
createResultStage uses the nextStageId counter for a stage ID.
createResultStage creates a new ResultStage (with the unique id of a ResourceProfile among others).
createResultStage registers the ResultStage with the stage ID in stageIdToStage.
createResultStage updateJobIdStageIdMaps and returns the ResultStage.
createResultStage is used when:
DAGScheduler is requested to handle a JobSubmitted event
","text":""},{"location":"scheduler/DAGScheduler/#creating-shufflemapstage-for-shuffledependency","title":"Creating ShuffleMapStage for ShuffleDependency
cleanupStateForJobAndIndependentStages cleans up the state for job and any stages that are not part of any other job.
cleanupStateForJobAndIndependentStages looks the job up in the internal jobIdToStageIds registry.
If no stages are found, the following ERROR is printed out to the logs:
No stages registered for job [jobId]\n
Oterwise, cleanupStateForJobAndIndependentStages uses stageIdToStage registry to find the stages (the real objects not ids!).
For each stage, cleanupStateForJobAndIndependentStages reads the jobs the stage belongs to.
If the job does not belong to the jobs of the stage, the following ERROR is printed out to the logs:
Job [jobId] not registered for stage [stageId] even though that stage was registered for the job\n
If the job was the only job for the stage, the stage (and the stage id) gets cleaned up from the registries, i.e. runningStages, shuffleIdToMapStage, waitingStages, failedStages and stageIdToStage.
While removing from runningStages, you should see the following DEBUG message in the logs:
Removing running stage [stageId]\n
While removing from waitingStages, you should see the following DEBUG message in the logs:
Removing stage [stageId] from waiting set.\n
While removing from failedStages, you should see the following DEBUG message in the logs:
Removing stage [stageId] from failed set.\n
After all cleaning (using stageIdToStage as the source registry), if the stage belonged to the one and only job, you should see the following DEBUG message in the logs:
After removal of stage [stageId], remaining stages = [stageIdToStage.size]\n
The job is removed from jobIdToStageIds, jobIdToActiveJob, activeJobs registries.
The final stage of the job is removed, i.e. ResultStage or ShuffleMapStage.
cleanupStateForJobAndIndependentStages is used in handleTaskCompletion when a ResultTask has completed successfully, failJobAndIndependentStages and markMapStageJobAsFinished.
markMapStageJobAsFinished marks the given ActiveJob finished and posts a SparkListenerJobEnd.
markMapStageJobAsFinished requests the given ActiveJob to turn on (true) the 0th bit in the finished partitions registry and increase the number of tasks finished.
markMapStageJobAsFinished requests the given ActiveJob for the JobListener that is requested to taskSucceeded (with the 0th index and the given MapOutputStatistics).
In the end, markMapStageJobAsFinished requests the LiveListenerBus to post a SparkListenerJobEnd.
markMapStageJobAsFinished is used when:
DAGScheduler is requested to handleMapStageSubmitted and markMapStageJobsAsFinished
","text":""},{"location":"scheduler/DAGScheduler/#finding-or-creating-missing-direct-parent-shufflemapstages-for-shuffledependencies-of-rdd","title":"Finding Or Creating Missing Direct Parent ShuffleMapStages (For ShuffleDependencies) of RDD
","text":""},{"location":"scheduler/DAGScheduler/#looking-up-shufflemapstage-for-shuffledependency","title":"Looking Up ShuffleMapStage for ShuffleDependency
getOrCreateShuffleMapStage finds a ShuffleMapStage by the shuffleId of the given ShuffleDependency in the shuffleIdToMapStage internal registry and returns it if available.
If not found, getOrCreateShuffleMapStage finds all the missing ancestor shuffle dependencies and creates the missing ShuffleMapStage stages (including one for the input ShuffleDependency).
getOrCreateShuffleMapStage is used when:
DAGScheduler is requested to find or create missing direct parent ShuffleMapStages of an RDD, find missing parent ShuffleMapStages for a stage, handle a MapStageSubmitted event, and check out stage dependency on a stage
","text":""},{"location":"scheduler/DAGScheduler/#missing-shuffledependencies-of-rdd","title":"Missing ShuffleDependencies of RDD
getMissingAncestorShuffleDependencies finds all the missing ShuffleDependencies for the given RDD (traversing its RDD lineage).
Note
A ShuffleDependency (of an RDD) is considered missing when not registered in the shuffleIdToMapStage internal registry.
Internally, getMissingAncestorShuffleDependencies finds direct parent shuffle dependencies\u2009of the input RDD and collects the ones that are not registered in the shuffleIdToMapStage internal registry. It repeats the process for the RDDs of the parent shuffle dependencies.
","text":""},{"location":"scheduler/DAGScheduler/#finding-direct-parent-shuffle-dependencies-of-rdd","title":"Finding Direct Parent Shuffle Dependencies of RDD
getShuffleDependencies finds direct parent shuffle dependencies for the given RDD.
Internally, getShuffleDependencies takes the direct rdd/index.md#dependencies[shuffle dependencies of the input RDD] and direct shuffle dependencies of all the parent non-ShuffleDependencies in the RDD lineage.
getShuffleDependencies is used when DAGScheduler is requested to find or create missing direct parent ShuffleMapStages (for ShuffleDependencies of a RDD) and find all missing shuffle dependencies for a given RDD.
","text":""},{"location":"scheduler/DAGScheduler/#failing-job-and-independent-single-job-stages","title":"Failing Job and Independent Single-Job Stages
failJobAndIndependentStages fails the input job and all the stages that are only used by the job.
Internally, failJobAndIndependentStages uses jobIdToStageIds internal registry to look up the stages registered for the job.
If no stages could be found, you should see the following ERROR message in the logs:
No stages registered for job [id]\n
Otherwise, for every stage, failJobAndIndependentStages finds the job ids the stage belongs to.
If no stages could be found or the job is not referenced by the stages, you should see the following ERROR message in the logs:
Job [id] not registered for stage [id] even though that stage was registered for the job\n
Only when there is exactly one job registered for the stage and the stage is in RUNNING state (in runningStages internal registry), TaskScheduler.md#contract[TaskScheduler is requested to cancel the stage's tasks] and <>.
NOTE: failJobAndIndependentStages uses jobIdToStageIds, stageIdToStage, and runningStages internal registries.
abortStage is an internal method that finds all the active jobs that depend on the failedStage stage and fails them.
Internally, abortStage looks the failedStage stage up in the internal stageIdToStage registry and exits if there the stage was not registered earlier.
If it was, abortStage finds all the active jobs (in the internal activeJobs registry) with the <failedStage stage>>.
At this time, the completionTime property (of the failed stage's StageInfo) is assigned to the current time (millis).
All the active jobs that depend on the failed stage (as calculated above) and the stages that do not belong to other jobs (aka independent stages) are <> (with the failure reason being \"Job aborted due to stage failure: [reason]\" and the input exception).
If there are no jobs depending on the failed stage, you should see the following INFO message in the logs:
Ignoring failure of [failedStage] because all jobs depending on it are done\n
abortStage is used when DAGScheduler is requested to handle a TaskSetFailed event, submit a stage, submit missing tasks of a stage, handle a TaskCompletion event.
","text":""},{"location":"scheduler/DAGScheduler/#checking-out-stage-dependency-on-given-stage","title":"Checking Out Stage Dependency on Given Stage
stageDependsOn compares two stages and returns whether the stage depends on target stage (i.e. true) or not (i.e. false).
NOTE: A stage A depends on stage B if B is among the ancestors of A.
Internally, stageDependsOn walks through the graph of RDDs of the input stage. For every RDD in the RDD's dependencies (using RDD.dependencies) stageDependsOn adds the RDD of a NarrowDependency to a stack of RDDs to visit while for a ShuffleDependency it <ShuffleMapStage stages for a ShuffleDependency>> for the dependency and the stage's first job id that it later adds to a stack of RDDs to visit if the map stage is ready, i.e. all the partitions have shuffle outputs.
After all the RDDs of the input stage are visited, stageDependsOn checks if the target's RDD is among the RDDs of the stage, i.e. whether the stage depends on target stage.
stageDependsOn is used when DAGScheduler is requested to abort a stage.
","text":""},{"location":"scheduler/DAGScheduler/#submitting-waiting-child-stages-for-execution","title":"Submitting Waiting Child Stages for Execution
submitWaitingChildStages submits for execution all waiting stages for which the input parent Stage.md[Stage] is the direct parent.
NOTE: Waiting stages are the stages registered in waitingStages internal registry.
When executed, you should see the following TRACE messages in the logs:
Checking if any dependencies of [parent] are now runnable\nrunning: [runningStages]\nwaiting: [waitingStages]\nfailed: [failedStages]\n
submitWaitingChildStages finds child stages of the input parent stage, removes them from waitingStages internal registry, and <> one by one sorted by their job ids.
submitWaitingChildStages is used when DAGScheduler is requested to submits missing tasks for a stage and handles a successful ShuffleMapTask completion.
","text":""},{"location":"scheduler/DAGScheduler/#submitting-stage-with-missing-parents-for-execution","title":"Submitting Stage (with Missing Parents) for Execution
submitStage(\n stage: Stage): Unit\n
submitStage submits the input stage or its missing parents (if there any stages not computed yet before the input stage could).
NOTE: submitStage is also used to DAGSchedulerEventProcessLoop.md#resubmitFailedStages[resubmit failed stages].
submitStage recursively submits any missing parents of the stage.
Internally, submitStage first finds the earliest-created job id that needs the stage.
NOTE: A stage itself tracks the jobs (their ids) it belongs to (using the internal jobIds registry).
The following steps depend on whether there is a job or not.
If there are no jobs that require the stage, submitStage <> with the reason:
No active job for stage [id]\n
If however there is a job for the stage, you should see the following DEBUG message in the logs:
submitStage([stage])\n
submitStage checks the status of the stage and continues when it was not recorded in waiting, running or failed internal registries. It simply exits otherwise.
With the stage ready for submission, submitStage calculates the <stage>> (sorted by their job ids). You should see the following DEBUG message in the logs:
missing: [missing]\n
When the stage has no parent stages missing, you should see the following INFO message in the logs:
Submitting [stage] ([stage.rdd]), which has no missing parents\n
submitStage <stage>> (with the earliest-created job id) and finishes.
If however there are missing parent stages for the stage, submitStage <>, and the stage is recorded in the internal waitingStages registry.
submitStage is used recursively for missing parents of the given stage and when DAGScheduler is requested for the following:
resubmitFailedStages (ResubmitFailedStages event)
submitWaitingChildStages (CompletionEvent event)
Handle JobSubmitted, MapStageSubmitted and TaskCompletion events
A single stage can be re-executed in multiple attempts due to fault recovery. The number of attempts is configured (FIXME).
If TaskScheduler reports that a task failed because a map output file from a previous stage was lost, the DAGScheduler resubmits the lost stage. This is detected through a DAGSchedulerEventProcessLoop.md#handleTaskCompletion-FetchFailed[CompletionEvent with FetchFailed], or an <> event. DAGScheduler will wait a small amount of time to see whether other nodes or tasks fail, then resubmit TaskSets for any lost stage(s) that compute the missing tasks.
Please note that tasks from the old attempts of a stage could still be running.
A stage object tracks multiple StageInfo objects to pass to Spark listeners or the web UI.
The latest StageInfo for the most recent attempt for a stage is accessible through latestInfo.
DAGScheduler computes where to run each task in a stage based on the rdd/index.md#getPreferredLocations[preferred locations of its underlying RDDs], or <>.","text":""},{"location":"scheduler/DAGScheduler/#adaptive-query-planning-adaptive-scheduling","title":"Adaptive Query Planning / Adaptive Scheduling
See SPARK-9850 Adaptive execution in Spark for the design document. The work is currently in progress.
DAGScheduler.submitMapStage method is used for adaptive query planning, to run map stages and look at statistics about their outputs before submitting downstream stages.
DAGScheduler uses the following ScheduledThreadPoolExecutors (with the policy of removing cancelled tasks from a work queue at time of cancellation):
dag-scheduler-message - a daemon thread pool using j.u.c.ScheduledThreadPoolExecutor with core pool size 1. It is used to post a DAGSchedulerEventProcessLoop.md#ResubmitFailedStages[ResubmitFailedStages] event when DAGSchedulerEventProcessLoop.md#handleTaskCompletion-FetchFailed[FetchFailed is reported].
They are created using ThreadUtils.newDaemonSingleThreadScheduledExecutor method that uses Guava DSL to instantiate a ThreadFactory.
","text":""},{"location":"scheduler/DAGScheduler/#finding-missing-parent-shufflemapstages-for-stage","title":"Finding Missing Parent ShuffleMapStages For Stage
getMissingParentStages finds missing parent ShuffleMapStages in the dependency graph of the input stage (using the breadth-first search algorithm).
Internally, getMissingParentStages starts with the stage's RDD and walks up the tree of all parent RDDs to find <>.
NOTE: A Stage tracks the associated RDD using Stage.md#rdd[rdd property].
NOTE: An uncached partition of a RDD is a partition that has Nil in the <> (which results in no RDD blocks in any of the active storage:BlockManager.md[BlockManager]s on executors).
getMissingParentStages traverses the rdd/index.md#dependencies[parent dependencies of the RDD] and acts according to their type, i.e. ShuffleDependency or NarrowDependency.
NOTE: ShuffleDependency and NarrowDependency are the main top-level Dependencies.
For each NarrowDependency, getMissingParentStages simply marks the corresponding RDD to visit and moves on to a next dependency of a RDD or works on another unvisited parent RDD.
NOTE: NarrowDependency is a RDD dependency that allows for pipelined execution.
getMissingParentStages focuses on ShuffleDependency dependencies.
NOTE: ShuffleDependency is a RDD dependency that represents a dependency on the output of a ShuffleMapStage, i.e. shuffle map stage.
For each ShuffleDependency, getMissingParentStages <ShuffleMapStage stages>>. If the ShuffleMapStage is not available, it is added to the set of missing (map) stages.
NOTE: A ShuffleMapStage is available when all its partitions are computed, i.e. results are available (as blocks).
CAUTION: FIXME...IMAGE with ShuffleDependencies queried
getMissingParentStages is used when DAGScheduler is requested to submit a stage and handle JobSubmitted and MapStageSubmitted events.
","text":""},{"location":"scheduler/DAGScheduler/#submitting-missing-tasks-of-stage","title":"Submitting Missing Tasks of Stage
submitMissingTasks prints out the following DEBUG message to the logs:
submitMissingTasks([stage])\n
submitMissingTasks requests the given Stage for the missing partitions (partitions that need to be computed).
submitMissingTasks adds the stage to the runningStages internal registry.
submitMissingTasks notifies the OutputCommitCoordinator that stage execution started.
submitMissingTasks determines preferred locations (task locality preferences) of the missing partitions.
submitMissingTasks requests the stage for a new stage attempt.
submitMissingTasks requests the LiveListenerBus to post a SparkListenerStageSubmitted event.
submitMissingTasks uses the closure Serializer to serialize the stage and create a so-called task binary. submitMissingTasks serializes the RDD (of the stage) and either the ShuffleDependency or the compute function based on the type of the stage (ShuffleMapStage or ResultStage, respectively).
submitMissingTasks creates a broadcast variable for the task binary.
Note
That shows how important broadcast variables are for Spark itself to distribute data among executors in a Spark application in the most efficient way.
submitMissingTasks creates tasks for every missing partition:
ShuffleMapTasks for a ShuffleMapStage
ResultTasks for a ResultStage
If there are tasks to submit for execution (i.e. there are missing partitions in the stage), submitMissingTasks prints out the following INFO message to the logs:
Submitting [size] missing tasks from [stage] ([rdd]) (first 15 tasks are for partitions [partitionIds])\n
submitMissingTasks requests the <> to TaskScheduler.md#submitTasks[submit the tasks for execution] (as a new TaskSet.md[TaskSet]).
With no tasks to submit for execution, submitMissingTasks <>.
submitMissingTasks prints out the following DEBUG messages based on the type of the stage:
Stage [stage] is actually done; (available: [isAvailable],available outputs: [numAvailableOutputs],partitions: [numPartitions])\n
or
Stage [stage] is actually done; (partitions: [numPartitions])\n
for ShuffleMapStage and ResultStage, respectively.
In the end, with no tasks to submit for execution, submitMissingTasks <> and exits.
submitMissingTasks is used when DAGScheduler is requested to submit a stage for execution.
","text":""},{"location":"scheduler/DAGScheduler/#finding-preferred-locations-for-missing-partitions","title":"Finding Preferred Locations for Missing Partitions
getCacheLocs gives TaskLocations (block locations) for the partitions of the input rdd. getCacheLocs caches lookup results in <> internal registry.
NOTE: The size of the collection from getCacheLocs is exactly the number of partitions in rdd RDD.
NOTE: The size of every TaskLocation collection (i.e. every entry in the result of getCacheLocs) is exactly the number of blocks managed using storage:BlockManager.md[BlockManagers] on executors.
Internally, getCacheLocs finds rdd in the <> internal registry (of partition locations per RDD).
If rdd is not in <> internal registry, getCacheLocs branches per its storage:StorageLevel.md[storage level].
For NONE storage level (i.e. no caching), the result is an empty locations (i.e. no location preference).
For other non-NONE storage levels, getCacheLocs storage:BlockManagerMaster.md#getLocations-block-array[requests BlockManagerMaster for block locations] that are then mapped to TaskLocations with the hostname of the owning BlockManager for a block (of a partition) and the executor id.
NOTE: getCacheLocs uses <> that was defined when <>.
getCacheLocs records the computed block locations per partition (as TaskLocation) in <> internal registry.
NOTE: getCacheLocs requests locations from BlockManagerMaster using storage:BlockId.md#RDDBlockId[RDDBlockId] with the RDD id and the partition indices (which implies that the order of the partitions matters to request proper blocks).
NOTE: DAGScheduler uses TaskLocation.md[TaskLocations] (with host and executor) while storage:BlockManagerMaster.md[BlockManagerMaster] uses storage:BlockManagerId.md[] (to track similar information, i.e. block locations).
getCacheLocs is used when DAGScheduler is requested to find missing parent MapStages and getPreferredLocsInternal.
","text":""},{"location":"scheduler/DAGScheduler/#finding-placement-preferences-for-rdd-partition-recursively","title":"Finding Placement Preferences for RDD Partition (recursively)
getPreferredLocsInternal first <TaskLocations for the partition of the rdd>> (using <> internal cache) and returns them.
Otherwise, if not found, getPreferredLocsInternal rdd/index.md#preferredLocations[requests rdd for the preferred locations of partition] and returns them.
NOTE: Preferred locations of the partitions of a RDD are also called placement preferences or locality preferences.
Otherwise, if not found, getPreferredLocsInternal finds the first parent NarrowDependency and (recursively) finds TaskLocations.
If all the attempts fail to yield any non-empty result, getPreferredLocsInternal returns an empty collection of TaskLocation.md[TaskLocations].
getPreferredLocsInternal is used when DAGScheduler is requested for the preferred locations for missing partitions.
removeExecutorAndUnregisterOutputs is used when DAGScheduler is requested to handle <> (due to a fetch failure) and <> events.","text":""},{"location":"scheduler/DAGScheduler/#markmapstagejobsasfinished","title":"markMapStageJobsAsFinished
executorHeartbeatReceived posts a SparkListenerExecutorMetricsUpdate (to listenerBus) and informs BlockManagerMaster that blockManagerId block manager is alive (by posting BlockManagerHeartbeat).
executorHeartbeatReceived is used when TaskSchedulerImpl is requested to handle an executor heartbeat.
handleTaskCompletion branches off per TaskEndReason (as event.reason).
TaskEndReason Description Success Acts according to the type of the task that completed, i.e. ShuffleMapTask and ResultTask Resubmitted others","text":""},{"location":"scheduler/DAGScheduler/#handling-successful-task-completion","title":"Handling Successful Task Completion
When a task has finished successfully (i.e. Success end reason), handleTaskCompletion marks the partition as no longer pending (i.e. the partition the task worked on is removed from pendingPartitions of the stage).
NOTE: A Stage tracks its own pending partitions using scheduler:Stage.md#pendingPartitions[pendingPartitions property].
handleTaskCompletion branches off given the type of the task that completed, i.e. <> and <>.","text":""},{"location":"scheduler/DAGScheduler/#handling-successful-resulttask-completion","title":"Handling Successful ResultTask Completion
For scheduler:ResultTask.md[ResultTask], the stage is assumed a scheduler:ResultStage.md[ResultStage].
handleTaskCompletion finds the ActiveJob associated with the ResultStage.
NOTE: scheduler:ResultStage.md[ResultStage] tracks the optional ActiveJob as scheduler:ResultStage.md#activeJob[activeJob property]. There could only be one active job for a ResultStage.
If there is no job for the ResultStage, you should see the following INFO message in the logs:
Ignoring result from [task] because its job has finished\n
Otherwise, when the ResultStage has a ActiveJob, handleTaskCompletion checks the status of the partition output for the partition the ResultTask ran for.
NOTE: ActiveJob tracks task completions in finished property with flags for every partition in a stage. When the flag for a partition is enabled (i.e. true), it is assumed that the partition has been computed (and no results from any ResultTask are expected and hence simply ignored).
CAUTION: FIXME Describe why could a partition has more ResultTask running.
handleTaskCompletion ignores the CompletionEvent when the partition has already been marked as completed for the stage and simply exits.
The partition for the ActiveJob (of the ResultStage) is marked as computed and the number of partitions calculated increased.
NOTE: ActiveJob tracks what partitions have already been computed and their number.
If the ActiveJob has finished (when the number of partitions computed is exactly the number of partitions in a stage) handleTaskCompletion does the following (in order):
scheduler:DAGScheduler.md#cleanupStateForJobAndIndependentStages[Cleans up after ActiveJob and independent stages].
Announces the job completion application-wide (by posting a SparkListener.md#SparkListenerJobEnd[SparkListenerJobEnd] to scheduler:LiveListenerBus.md[]).
In the end, handleTaskCompletion notifies JobListener of the ActiveJob that the task succeeded.
NOTE: A task succeeded notification holds the output index and the result.
When the notification throws an exception (because it runs user code), handleTaskCompletion notifies JobListener about the failure (wrapping it inside a SparkDriverExecutionException exception).
The task's result is assumed scheduler:MapStatus.md[MapStatus] that knows the executor where the task has finished.
You should see the following DEBUG message in the logs:
ShuffleMapTask finished on [execId]\n
If the executor is registered in scheduler:DAGScheduler.md#failedEpoch[failedEpoch internal registry] and the epoch of the completed task is not greater than that of the executor (as in failedEpoch registry), you should see the following INFO message in the logs:
Ignoring possibly bogus [task] completion from executor [executorId]\n
Otherwise, handleTaskCompletion scheduler:ShuffleMapStage.md#addOutputLoc[registers the MapStatus result for the partition with the stage] (of the completed task).
handleTaskCompletion does more processing only if the ShuffleMapStage is registered as still running (in scheduler:DAGScheduler.md#runningStages[runningStages internal registry]) and the scheduler:Stage.md#pendingPartitions[ShuffleMapStage stage has no pending partitions to compute].
The ShuffleMapStage is <>.
You should see the following INFO messages in the logs:
looking for newly runnable stages\nrunning: [runningStages]\nwaiting: [waitingStages]\nfailed: [failedStages]\n
handleTaskCompletion scheduler:MapOutputTrackerMaster.md#registerMapOutputs[registers the shuffle map outputs of the ShuffleDependency with MapOutputTrackerMaster] (with the epoch incremented) and scheduler:DAGScheduler.md#clearCacheLocs[clears internal cache of the stage's RDD block locations].
NOTE: scheduler:MapOutputTrackerMaster.md[MapOutputTrackerMaster] is given when scheduler:DAGScheduler.md#creating-instance[DAGScheduler is created].
If the scheduler:ShuffleMapStage.md#isAvailable[ShuffleMapStage stage is ready], all scheduler:ShuffleMapStage.md#mapStageJobs[active jobs of the stage] (aka map-stage jobs) are scheduler:DAGScheduler.md#markMapStageJobAsFinished[marked as finished] (with scheduler:MapOutputTrackerMaster.md#getStatistics[MapOutputStatistics from MapOutputTrackerMaster for the ShuffleDependency]).
NOTE: A ShuffleMapStage stage is ready (aka available) when all partitions have shuffle outputs, i.e. when their tasks have completed.
Eventually, handleTaskCompletion scheduler:DAGScheduler.md#submitWaitingChildStages[submits waiting child stages (of the ready ShuffleMapStage)].
If however the ShuffleMapStage is not ready, you should see the following INFO message in the logs:
Resubmitting [shuffleStage] ([shuffleStage.name]) because some of its tasks had failed: [missingPartitions]\n
In the end, handleTaskCompletion scheduler:DAGScheduler.md#submitStage[submits the ShuffleMapStage for execution].
When FetchFailed happens, stageIdToStage is used to access the failed stage (using task.stageId and the task is available in event in handleTaskCompletion(event: CompletionEvent)). shuffleToMapStage is used to access the map stage (using shuffleId).
If failedStage.latestInfo.attemptId != task.stageAttemptId, you should see the following INFO in the logs:
Ignoring fetch failure from [task] as it's from [failedStage] attempt [task.stageAttemptId] and there is a more recent attempt for that stage (attempt ID [failedStage.latestInfo.attemptId]) running\n
CAUTION: FIXME What does failedStage.latestInfo.attemptId != task.stageAttemptId mean?
And the case finishes. Otherwise, the case continues.
If the failed stage is in runningStages, the following INFO message shows in the logs:
Marking [failedStage] ([failedStage.name]) as failed due to a fetch failure from [mapStage] ([mapStage.name])\n
markStageAsFinished(failedStage, Some(failureMessage)) is called.
CAUTION: FIXME What does markStageAsFinished do?
If the failed stage is not in runningStages, the following DEBUG message shows in the logs:
Received fetch failure from [task], but its from [failedStage] which is no longer running\n
When disallowStageRetryForTest is set, abortStage(failedStage, \"Fetch failure will not retry stage due to testing config\", None) is called.
CAUTION: FIXME Describe disallowStageRetryForTest and abortStage.
If the scheduler:Stage.md#failedOnFetchAndShouldAbort[number of fetch failed attempts for the stage exceeds the allowed number], the scheduler:DAGScheduler.md#abortStage[failed stage is aborted] with the reason:
[failedStage] ([name]) has failed the maximum allowable number of times: 4. Most recent failure reason: [failureMessage]\n
If there are no failed stages reported (scheduler:DAGScheduler.md#failedStages[DAGScheduler.failedStages] is empty), the following INFO shows in the logs:
Resubmitting [mapStage] ([mapStage.name]) and [failedStage] ([failedStage.name]) due to fetch failure\n
And the following code is executed:
messageScheduler.schedule(\n new Runnable {\n override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages)\n }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS)\n
CAUTION: FIXME What does the above code do?
For all the cases, the failed stage and map stages are both added to the internal scheduler:DAGScheduler.md#failedStages[registry of failed stages].
If mapId (in the FetchFailed object for the case) is provided, the map stage output is cleaned up (as it is broken) using mapStage.removeOutputLoc(mapId, bmAddress) and scheduler:MapOutputTracker.md#unregisterMapOutput[MapOutputTrackerMaster.unregisterMapOutput(shuffleId, mapId, bmAddress)] methods.
CAUTION: FIXME What does mapStage.removeOutputLoc do?
If BlockManagerId (as bmAddress in the FetchFailed object) is defined, handleTaskCompletion <> (with filesLost enabled and maybeEpoch from the scheduler:Task.md#epoch[Task] that completed).
handleTaskCompletion is used when:
DAGSchedulerEventProcessLoop is requested to handle a CompletionEvent event.
handleExecutorLost checks whether the input optional maybeEpoch is defined and if not requests the scheduler:MapOutputTracker.md#getEpoch[current epoch from MapOutputTrackerMaster].
NOTE: MapOutputTrackerMaster is passed in (as mapOutputTracker) when scheduler:DAGScheduler.md#creating-instance[DAGScheduler is created].
Recurring ExecutorLost events lead to the following repeating DEBUG message in the logs:
DEBUG Additional executor lost message for [execId] (epoch [currentEpoch])\n
NOTE: handleExecutorLost handler uses DAGScheduler's failedEpoch and FIXME internal registries.
Otherwise, when the executor execId is not in the scheduler:DAGScheduler.md#failedEpoch[list of executor lost] or the executor failure's epoch is smaller than the input maybeEpoch, the executor's lost event is recorded in scheduler:DAGScheduler.md#failedEpoch[failedEpoch internal registry].
CAUTION: FIXME Describe the case above in simpler non-technical words. Perhaps change the order, too.
You should see the following INFO message in the logs:
INFO Executor lost: [execId] (epoch [epoch])\n
storage:BlockManagerMaster.md#removeExecutor[BlockManagerMaster is requested to remove the lost executor execId].
CAUTION: FIXME Review what's filesLost.
handleExecutorLost exits unless the ExecutorLost event was for a map output fetch operation (and the input filesLost is true) or external shuffle service is not used.
In such a case, you should see the following INFO message in the logs:
Shuffle files lost for executor: [execId] (epoch [epoch])\n
handleExecutorLost walks over all scheduler:ShuffleMapStage.md[ShuffleMapStage]s in scheduler:DAGScheduler.md#shuffleToMapStage[DAGScheduler's shuffleToMapStage internal registry] and do the following (in order):
ShuffleMapStage.removeOutputsOnExecutor(execId) is called
scheduler:MapOutputTrackerMaster.md#registerMapOutputs[MapOutputTrackerMaster.registerMapOutputs(shuffleId, stage.outputLocInMapOutputTrackerFormat(), changeEpoch = true)] is called.
In case scheduler:DAGScheduler.md#shuffleToMapStage[DAGScheduler's shuffleToMapStage internal registry] has no shuffles registered, scheduler:MapOutputTrackerMaster.md#incrementEpoch[MapOutputTrackerMaster is requested to increment epoch].
Ultimatelly, DAGScheduler scheduler:DAGScheduler.md#clearCacheLocs[clears the internal cache of RDD partition locations].
handleExecutorLost is used when DAGSchedulerEventProcessLoop is requested to handle an ExecutorLost event.
handleJobCancellation looks up the active job for the input job ID (in jobIdToActiveJob internal registry) and fails it and all associated independent stages with failure reason:
Job [jobId] cancelled [reason]\n
When the input job ID is not found, handleJobCancellation prints out the following DEBUG message to the logs:
Trying to cancel unregistered job [jobId]\n
handleJobCancellation is used when DAGScheduler is requested to handle a JobCancelled event, doCancelAllJobs, handleJobGroupCancelled, handleStageCancellation.
handleJobGroupCancelled finds active jobs in a group and cancels them.
Internally, handleJobGroupCancelled computes all the active jobs (registered in the internal collection of active jobs) that have spark.jobGroup.id scheduling property set to groupId.
handleJobGroupCancelled then cancels every active job in the group one by one and the cancellation reason:
part of cancelled job group [groupId]\n
handleJobGroupCancelled is used when DAGScheduler is requested to handle JobGroupCancelled event.
handleJobSubmitted creates a ResultStage (finalStage) for the given RDD, func, partitions, jobId and callSite.
BarrierJobSlotsNumberCheckFailed Exception
Creating a ResultStage may fail with a BarrierJobSlotsNumberCheckFailed exception.
handleJobSubmitted removes the given jobId from the barrierJobIdToNumTasksCheckFailures.
handleJobSubmitted creates an ActiveJob for the ResultStage (with the given jobId, the callSite, the JobListener and the properties).
handleJobSubmitted clears the internal cache of RDD partition locations.
FIXME Why is this clearing here so important?
handleJobSubmitted prints out the following INFO messages to the logs (with missingParentStages):
Got job [id] ([callSite]) with [number] output partitions\nFinal stage: [finalStage] ([name])\nParents of final stage: [parents]\nMissing parents: [missingParentStages]\n
handleJobSubmitted registers the new ActiveJob in jobIdToActiveJob and activeJobs internal registries.
handleJobSubmitted requests the ResultStage to associate itself with the ActiveJob.
handleJobSubmitted uses the jobIdToStageIds internal registry to find all registered stages for the given jobId. handleJobSubmitted uses the stageIdToStage internal registry to request the Stages for the latestInfo.
In the end, handleJobSubmitted posts a SparkListenerJobStart message to the LiveListenerBus and submits the ResultStage.
handleJobSubmitted is used when:
DAGSchedulerEventProcessLoop is requested to handle a JobSubmitted event
In case of a BarrierJobSlotsNumberCheckFailed exception while creating a ResultStage, handleJobSubmitted increments the number of failures in the barrierJobIdToNumTasksCheckFailures for the given jobId.
handleJobSubmitted prints out the following WARN message to the logs (with spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures):
Barrier stage in job [jobId] requires [requiredConcurrentTasks] slots, but only [maxConcurrentTasks] are available. Will retry up to [maxFailures] more times\n
If the number of failures is below the spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures threshold, handleJobSubmitted requests the messageScheduler to schedule a one-shot task that requests the DAGSchedulerEventProcessLoop to post a JobSubmitted event (after spark.scheduler.barrier.maxConcurrentTasksCheck.interval seconds).
Note
Posting a JobSubmitted event is to request the DAGScheduler to re-consider the request, hoping that there will be enough resources to fulfill the resource requirements of a barrier job.
Otherwise, if the number of failures crossed the spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures threshold, handleJobSubmitted removes the jobId from the barrierJobIdToNumTasksCheckFailures and informs the given JobListener that the jobFailed.
resubmitFailedStages iterates over the internal collection of failed stages and submits them.
Note
resubmitFailedStages does nothing when there are no failed stages reported.
resubmitFailedStages prints out the following INFO message to the logs:
Resubmitting failed stages\n
resubmitFailedStages clears the internal cache of RDD partition locations and makes a copy of the collection of failed stages to track failed stages afresh.
Note
At this point DAGScheduler has no failed stages reported.
The previously-reported failed stages are sorted by the corresponding job ids in incremental order and resubmitted.
resubmitFailedStages is used when DAGSchedulerEventProcessLoop is requested to handle a ResubmitFailedStages event.
Stages that failed due to fetch failures (when a DAGSchedulerEventProcessLoop.md#handleTaskCompletion-FetchFailed[task fails with FetchFailed exception]).
Used when DAGScheduler creates a <> and a <>. It is the key in stageIdToStage.","text":""},{"location":"scheduler/DAGScheduler/#runningstages","title":"runningStages
The set of stages that are currently \"running\".
A stage is added when <> gets executed (without first checking if the stage has not already been added).","text":""},{"location":"scheduler/DAGScheduler/#shuffleidtomapstage","title":"shuffleIdToMapStage
A lookup table of ShuffleMapStages by ShuffleDependency
Used when DAGScheduler creates a shuffle map stage, creates a result stage, cleans up job state and independent stages, is informed that a task is started, a taskset has failed, a job is submitted (to compute a ResultStage), a map stage was submitted, a task has completed or a stage was cancelled, updates accumulators, aborts a stage and fails a job and independent stages.
Used when TaskSchedulerImpl is requested to handle a task status update (and a task gets lost which is used to indicate that the executor got broken and hence should be considered lost) or executorLost
updateAccumulators merges the partial values of accumulators from a completed task (based on the given CompletionEvent) into their \"source\" accumulators on the driver.
For every AccumulatorV2 update (in the given CompletionEvent), updateAccumulators finds the corresponding accumulator on the driver and requests the AccumulatorV2 to merge the updates.
updateAccumulators...FIXME
For named accumulators with the update value being a non-zero value, i.e. not Accumulable.zero:
stage.latestInfo.accumulables for the AccumulableInfo.id is set
CompletionEvent.taskInfo.accumulables has a new AccumulableInfo added.
CAUTION: FIXME Where are Stage.latestInfo.accumulables and CompletionEvent.taskInfo.accumulables used?
updateAccumulators is used when DAGScheduler is requested to handle a task completion.
Unless the given RDD is isBarrier, checkBarrierStageWithNumSlots does nothing (is a noop).
checkBarrierStageWithNumSlots requests the given RDD for the number of partitions.
checkBarrierStageWithNumSlots requests the SparkContext for the maximum number of concurrent tasks for the given ResourceProfile.
If the number of partitions (based on the RDD) is greater than the maximum number of concurrent tasks (based on the ResourceProfile), checkBarrierStageWithNumSlots reports a BarrierJobSlotsNumberCheckFailed exception.
checkBarrierStageWithNumSlots is used when:
DAGScheduler is requested to create a ShuffleMapStage or a ResultStage stage
The section includes (hides) utility methods that do not really contribute to the understanding of how DAGScheduler works internally.
It's very likely they should not even be part of this page.
","text":""},{"location":"scheduler/DAGScheduler/#getShuffleDependenciesAndResourceProfiles","title":"Finding Shuffle Dependencies and ResourceProfiles of RDD
getShuffleDependenciesAndResourceProfiles returns the direct ShuffleDependencies and all the ResourceProfiles of the given RDD and parent non-shuffle RDDs, if available.
getShuffleDependenciesAndResourceProfiles collects ResourceProfiles of the given RDD and any parent RDDs, if available.
getShuffleDependenciesAndResourceProfiles collects direct ShuffleDependencies of the given RDD and any parent RDDs of non-ShuffleDependencyies, if available.
getShuffleDependenciesAndResourceProfiles is used when:
DAGScheduler is requested to create a ShuffleMapStage and a ResultStage, and for the missing ShuffleDependencies of a RDD
DriverEndpoint is a ThreadSafeRpcEndpoint that is a message handler for CoarseGrainedSchedulerBackend to communicate with CoarseGrainedExecutorBackend.
DriverEndpoint is registered under the name CoarseGrainedScheduler by CoarseGrainedSchedulerBackend.
DriverEndpoint uses executorDataMap internal registry of all the executors that registered with the driver. An executor sends a RegisterExecutor message to inform that it wants to register.
onStart requests the Revive Messages Scheduler Service to schedule a periodic action that sends ReviveOffers messages every revive interval (based on spark.scheduler.revive.interval configuration property).
There are two makeOffers methods to launch tasks that differ by the number of active executor (from the executorDataMap registry) they work with:
All Active Executors
Single Executor
","text":""},{"location":"scheduler/DriverEndpoint/#on-all-active-executors","title":"On All Active Executors","text":"
makeOffers(): Unit\n
makeOffers builds WorkerOffers for every active executor (in the executorDataMap registry) and requests the TaskSchedulerImpl to generate tasks for the available worker offers (that creates TaskDescriptions).
With tasks (TaskDescriptions) to be launched, makeOffers launches them.
makeOffers is used when:
DriverEndpoint handles ReviveOffers messages
"},{"location":"scheduler/DriverEndpoint/#on-single-executor","title":"On Single Executor","text":"
makeOffers(\n executorId: String): Unit\n
Note
makeOffers with a single executor is makeOffers for all active executors for just one executor.
makeOffers is used when:
DriverEndpoint handles StatusUpdate and LaunchedExecutor messages
The input tasks collection contains one or more TaskDescriptions per executor (and the \"task partitioning\" per executor is of no use in launchTasks so it simply flattens the input data structure).
For every TaskDescription (in the given tasks collection), launchTasks encodes it and makes sure that the encoded task size is below the allowed message size.
launchTasks looks up the ExecutorData of the executor that has been assigned to execute the task (in executorDataMap internal registry) and decreases the executor's free cores (based on spark.task.cpus configuration property).
Note
Scheduling in Spark relies on cores only (not memory), i.e. the number of tasks Spark can run on an executor is limited by the number of cores available only. When submitting a Spark application for execution both executor resources -- memory and cores -- can however be specified explicitly. It is the job of a cluster manager to monitor the memory and take action when its use exceeds what was assigned.
launchTasks prints out the following DEBUG message to the logs:
Launching task [taskId] on executor id: [executorId] hostname: [executorHost].\n
In the end, launchTasks sends the (serialized) task to the executor (by sending a LaunchTask message to the executor's RPC endpoint with the serialized task insize SerializableBuffer).
Note
This is the moment in a task's lifecycle when the driver sends the serialized task to an assigned executor.
In case the size of a serialized TaskDescription equals or exceeds the maximum allowed RPC message size, launchTasks looks up the TaskSetManager for the TaskDescription (in taskIdToTaskSetManager registry) and aborts it with the following message:
Serialized task [id]:[index] was [limit] bytes, which exceeds max allowed: spark.rpc.message.maxSize ([maxRpcMessageSize] bytes). Consider increasing spark.rpc.message.maxSize or using broadcast variables for large values.\n
RegisterExecutor is sent when CoarseGrainedExecutorBackend RPC Endpoint is requested to start.
When received, DriverEndpoint makes sure that no other executors were registered under the input executorId and that the input hostname is not blacklisted.
If the requirements hold, you should see the following INFO message in the logs:
Registered executor [executorRef] ([address]) with ID [executorId]\n
DriverEndpoint does the bookkeeping:
Registers executorId (in addressToExecutorId)
Adds cores (in totalCoreCount)
Increments totalRegisteredExecutors
Creates and registers ExecutorData for executorId (in executorDataMap)
Updates currentExecutorIdCounter if the input executorId is greater than the current value.
If numPendingExecutors is greater than 0, you should see the following DEBUG message in the logs and DriverEndpoint decrements numPendingExecutors.
Decremented number of pending executors ([numPendingExecutors] left)\n
DriverEndpoint sends RegisteredExecutor message back (that is to confirm that the executor was registered successfully).
DriverEndpoint replies true (to acknowledge the message).
DriverEndpoint then announces the new executor by posting SparkListenerExecutorAdded to LiveListenerBus.
In the end, DriverEndpoint makes executor resource offers (for launching tasks).
If however there was already another executor registered under the input executorId, DriverEndpoint sends RegisterExecutorFailed message back with the reason:
Duplicate executor ID: [executorId]\n
If however the input hostname is blacklisted, you should see the following INFO message in the logs:
Rejecting [executorId] as it has been blacklisted.\n
DriverEndpoint sends RegisterExecutorFailed message back with the reason:
disableExecutor checks whether the executor is active:
If so, disableExecutor adds the executor to the executorsPendingLossReason registry
Otherwise, disableExecutor checks whether added to executorsPendingToRemove registry
disableExecutor determines whether the executor should really be disabled (as active or registered in executorsPendingToRemove registry).
If the executor should be disabled, disableExecutor prints out the following INFO message to the logs and notifies the TaskSchedulerImpl that the executor is lost.
Disabling executor [executorId].\n
disableExecutor returns the indication whether the executor should have been disabled or not.
disableExecutor is used when:
KubernetesDriverEndpoint is requested to handle onDisconnected event
YarnDriverEndpoint is requested to handle onDisconnected event
ExternalClusterManager is an abstraction of pluggable cluster managers that can create a SchedulerBackend and TaskScheduler for a given master URL (when SparkContext is created).
Note
The support for pluggable cluster managers was introduced in SPARK-13904 Add support for pluggable cluster manager.
ExternalClusterManager can be registered using the java.util.ServiceLoader mechanism (with service markers under META-INF/services directory).
"},{"location":"scheduler/ExternalClusterManager/#contract","title":"Contract","text":""},{"location":"scheduler/ExternalClusterManager/#checking-support-for-master-url","title":"Checking Support for Master URL
canCreate(\n masterURL: String): Boolean\n
Checks whether this cluster manager instance can create scheduler components for a given master URL
Used when SparkContext is created (and requested for a cluster manager)
== FIFOSchedulableBuilder - SchedulableBuilder for FIFO Scheduling Mode
FIFOSchedulableBuilder is a <> that holds a single spark-scheduler-Pool.md[Pool] (that is given when <FIFOSchedulableBuilder is created>>).
NOTE: FIFOSchedulableBuilder is the scheduler:TaskSchedulerImpl.md#creating-instance[default SchedulableBuilder for TaskSchedulerImpl].
NOTE: When FIFOSchedulableBuilder is created, the TaskSchedulerImpl passes its own rootPool (a part of scheduler:TaskScheduler.md#contract[TaskScheduler Contract]).
FIFOSchedulableBuilder obeys the <> as follows:
<> does nothing.
addTaskSetManager spark-scheduler-Pool.md#addSchedulable[passes the input Schedulable to the one and only rootPool Pool (using addSchedulable)] and completely disregards the properties of the Schedulable.
FairSchedulableBuilder is a <> that is <> exclusively for scheduler:TaskSchedulerImpl.md[TaskSchedulerImpl] for FAIR scheduling mode (when configuration-properties.md#spark.scheduler.mode[spark.scheduler.mode] configuration property is FAIR).
[[creating-instance]] FairSchedulableBuilder takes the following to be created:
[[rootPool]] <>
[[conf]] SparkConf.md[]
Once <>, TaskSchedulerImpl requests the FairSchedulableBuilder to <>.
[[DEFAULT_SCHEDULER_FILE]] FairSchedulableBuilder uses the pools defined in an <> that is assumed to be the value of the configuration-properties.md#spark.scheduler.allocation.file[spark.scheduler.allocation.file] configuration property or the default fairscheduler.xml (that is <>).
TIP: Use conf/fairscheduler.xml.template as a template for the <>.
[[DEFAULT_POOL_NAME]] FairSchedulableBuilder always has the default pool defined (and <> unless done in the <>).
[[FAIR_SCHEDULER_PROPERTIES]] [[spark.scheduler.pool]] FairSchedulableBuilder uses spark.scheduler.pool local property for the name of the pool to use when requested to <> (default: <>).
Note
SparkContext.setLocalProperty lets you set local properties per thread to group jobs in logical groups, e.g. to allow FairSchedulableBuilder to use spark.scheduler.pool property and to group jobs from different threads to be submitted for execution on a non-<> pool."},{"location":"scheduler/FairSchedulableBuilder/#source-scala","title":"[source, scala]","text":"
"},{"location":"scheduler/FairSchedulableBuilder/#whatever-is-executed-afterwards-is-submitted-to-production-pool","title":"// whatever is executed afterwards is submitted to production pool","text":"
[[logging]] [TIP] ==== Enable ALL logging level for org.apache.spark.scheduler.FairSchedulableBuilder logger to see what happens inside.
NOTE: buildPools is part of the <> to build a tree of <>.
buildPools <> if available and then <>.
buildPools prints out the following INFO message to the logs when the configuration file (per the configuration-properties.md#spark.scheduler.allocation.file[spark.scheduler.allocation.file] configuration property) could be read:
Creating Fair Scheduler pools from [file]\n
buildPools prints out the following INFO message to the logs when the configuration-properties.md#spark.scheduler.allocation.file[spark.scheduler.allocation.file] configuration property was not used to define the configuration file and the <> is used instead:
Creating Fair Scheduler pools from default file: [DEFAULT_SCHEDULER_FILE]\n
When neither configuration-properties.md#spark.scheduler.allocation.file[spark.scheduler.allocation.file] configuration property nor the <> could be used, buildPools prints out the following WARN message to the logs:
Fair Scheduler configuration file not found so jobs will be scheduled in FIFO order. To use fair scheduling, configure pools in [DEFAULT_SCHEDULER_FILE] or set spark.scheduler.allocation.file to a file that contains the configuration.\n
NOTE: addTaskSetManager is part of the <> to register a new <> with the <>
addTaskSetManager finds the pool by name (in the given Properties) under the <> property or defaults to the <> pool if undefined.
addTaskSetManager then requests the <> to <>.
Unless found, addTaskSetManager creates a new <> with the <> (as if the <> pool were used) and requests the <> to <>. In the end, addTaskSetManager prints out the following WARN message to the logs:
A job was submitted with scheduler pool [poolName], which has not been configured. This can happen when the file that pools are read from isn't set, or when that file doesn't contain [poolName]. Created [poolName] with default configuration (schedulingMode: [mode], minShare: [minShare], weight: [weight])\n
addTaskSetManager then requests the pool (found or newly-created) to <> the given <>.
In the end, addTaskSetManager prints out the following INFO message to the logs:
Added task set [name] tasks to pool [poolName]\n
=== [[buildDefaultPool]] Registering Default Pool -- buildDefaultPool Method
buildFairSchedulerPool( is: InputStream, fileName: String): Unit
buildFairSchedulerPool starts by loading the XML file from the given InputStream.
For every pool element, buildFairSchedulerPool creates a <> with the following:
Pool name per name attribute
Scheduling mode per schedulingMode element (case-insensitive with FIFO as the default)
Initial minimum share per minShare element (default: 0)
Initial weight per weight element (default: 1)
In the end, buildFairSchedulerPool requests the <> to <> followed by the INFO message in the logs:
Created pool: [name], schedulingMode: [mode], minShare: [minShare], weight: [weight]\n
NOTE: buildFairSchedulerPool is used exclusively when FairSchedulableBuilder is requested to <>."},{"location":"scheduler/HighlyCompressedMapStatus/","title":"HighlyCompressedMapStatus","text":"
Used when DAGScheduler is requested to cleanUpAfterSchedulerStop, handleJobSubmitted, handleMapStageSubmitted, handleTaskCompletion or failJobAndIndependentStages
start starts AsyncEventQueues (from the queues internal registry).
In the end, start requests the given MetricsSystem to register the LiveListenerBusMetrics.
start is used when:
SparkContext is created
","text":""},{"location":"scheduler/LiveListenerBus/#posting-event-to-all-queues","title":"Posting Event to All Queues
post(\n event: SparkListenerEvent): Unit\n
post puts the input event onto the internal eventQueue queue and releases the internal eventLock semaphore. If the event placement was not successful (and it could happen since it is tapped at 10000 events) onDropEvent method is called.
The event publishing is only possible when stopped flag has been enabled.
onDropEvent is called when no further events can be added to the internal eventQueue queue (while posting a SparkListenerEvent event).
It simply prints out the following ERROR message to the logs and ensures that it happens only once.
Dropping SparkListenerEvent because no remaining room in event queue. This likely means one of the SparkListeners is too slow and cannot keep up with the rate at which tasks are being started by the scheduler.\n
stop releases the internal eventLock semaphore and waits until listenerThread dies. It can only happen after all events were posted (and polling eventQueue gives nothing).
stopped flag is enabled.
","text":""},{"location":"scheduler/LiveListenerBus/#listenerthread-for-event-polling","title":"listenerThread for Event Polling
LiveListenerBus uses a SparkListenerBus single-daemon thread that ensures that the polling events from the event queue is only after the listener was started and only one event at a time.
","text":""},{"location":"scheduler/LiveListenerBus/#registering-listener-with-status-queue","title":"Registering Listener with Status Queue
addToSharedQueue adds the given SparkListenerInterface to shared queue.
addToSharedQueue is used when:
SparkContext is requested to register a SparkListener and register extra SparkListeners
ExecutionListenerBus (Spark Structured Streaming) is created
","text":""},{"location":"scheduler/LiveListenerBus/#registering-listener-with-executormanagement-queue","title":"Registering Listener with executorManagement Queue
addToQueue finds the queue in the queues internal registry.
If found, addToQueue requests it to add the given listener
If not found, addToQueue creates a AsyncEventQueue (with the given name, the LiveListenerBusMetrics, and this LiveListenerBus) and requests it to add the given listener. The AsyncEventQueue is started and added to the queues internal registry.
addToQueue is used when:
LiveListenerBus is requested to addToSharedQueue, addToManagementQueue, addToStatusQueue, addToEventLogQueue
StreamingQueryListenerBus (Spark Structured Streaming) is created
trackerEndpoint is a RpcEndpointRef of the MapOutputTracker RPC endpoint.
trackerEndpoint is initialized (registered or looked up) when SparkEnv is created for the driver and executors.
trackerEndpoint is used to communicate (synchronously).
trackerEndpoint is cleared (null) when MapOutputTrackerMaster is requested to stop.
","text":""},{"location":"scheduler/MapOutputTracker/#deregistering-map-output-status-information-of-shuffle-stage","title":"Deregistering Map Output Status Information of Shuffle Stage
unregisterShuffle(\n shuffleId: Int): Unit\n
Deregisters map output status information for the given shuffle stage
Used when:
ContextCleaner is requested for shuffle cleanup
BlockManagerSlaveEndpoint is requested to remove a shuffle
stop is used when SparkEnv is requested to stop (and stops all the services, incl. MapOutputTracker).
","text":""},{"location":"scheduler/MapOutputTracker/#converting-mapstatuses-to-blockmanagerids-with-shuffleblockids-and-their-sizes","title":"Converting MapStatuses To BlockManagerIds with ShuffleBlockIds and Their Sizes
convertMapStatuses iterates over the input statuses array (of MapStatus entries indexed by map id) and creates a collection of BlockManagerIds (for each MapStatus entry) with a ShuffleBlockId (with the input shuffleId, a mapId, and partition ranging from the input startPartition and endPartition) and estimated size for the reduce block for every status and partitions.
For any empty MapStatus, convertMapStatuses prints out the following ERROR message to the logs:
Missing an output location for shuffle [id]\n
And convertMapStatuses throws a MetadataFetchFailedException (with shuffleId, startPartition, and the above error message).
convertMapStatuses is used when:
MapOutputTrackerMaster is requested for the sizes of shuffle map outputs by executor and range
MapOutputTrackerWorker is requested to sizes of shuffle map outputs by executor and range
","text":""},{"location":"scheduler/MapOutputTracker/#sending-blocking-messages-to-trackerendpoint-rpcendpointref","title":"Sending Blocking Messages To trackerEndpoint RpcEndpointRef
askTracker[T](message: Any): T\n
askTracker sends the input message to trackerEndpoint RpcEndpointRef and waits for a result.
When an exception happens, askTracker prints out the following ERROR message to the logs and throws a SparkException.
Error communicating with MapOutputTracker\n
askTracker is used when MapOutputTracker is requested to fetches map outputs for ShuffleDependency remotely and sends a one-way message.
serializeMapStatuses serializes the given array of map output locations into an efficient byte format (to send to reduce tasks). serializeMapStatuses compresses the serialized bytes using GZIP. They are supposed to be pretty compressible because many map outputs will be on the same hostname.
Internally, serializeMapStatuses creates a Java ByteArrayOutputStream.
serializeMapStatuses writes out 0 (direct) first.
serializeMapStatuses creates a Java GZIPOutputStream (with the ByteArrayOutputStream created) and writes out the given statuses array.
serializeMapStatuses decides whether to return the output array (of the output stream) or use a broadcast variable based on the size of the byte array.
If the size of the result byte array is the given minBroadcastSize threshold or bigger, serializeMapStatuses requests the input BroadcastManager to create a broadcast variable.
serializeMapStatuses resets the ByteArrayOutputStream and starts over.
serializeMapStatuses writes out 1 (broadcast) first.
serializeMapStatuses creates a new Java GZIPOutputStream (with the ByteArrayOutputStream created) and writes out the broadcast variable.
serializeMapStatuses prints out the following INFO message to the logs:
Broadcast mapstatuses size = [length], actual size = [length]\n
serializeMapStatuses is used when ShuffleStatus is requested to serialize shuffle map output statuses.
MapOutputTrackerMaster is given a BroadcastManager to be created.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#shuffle-map-output-status-registry","title":"Shuffle Map Output Status Registry
MapOutputTrackerMaster uses an internal registry of ShuffleStatuses by shuffle stages.
MapOutputTrackerMaster adds a new shuffle when requested to register one (when DAGScheduler is requested to create a ShuffleMapStage for a ShuffleDependency).
MapOutputTrackerMaster uses the registry when requested for the following:
MapOutputTrackerMaster uses the following configuration properties:
spark.shuffle.mapOutput.minSizeForBroadcast
spark.shuffle.mapOutput.dispatcher.numThreads
spark.shuffle.reduceLocality.enabled","text":""},{"location":"scheduler/MapOutputTrackerMaster/#map-and-reduce-task-thresholds-for-preferred-locations","title":"Map and Reduce Task Thresholds for Preferred Locations
MapOutputTrackerMaster defines 1000 (tasks) as the hardcoded threshold of the number of map and reduce tasks when requested to compute preferred locations with spark.shuffle.reduceLocality.enabled.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#map-output-threshold-for-preferred-location-of-reduce-tasks","title":"Map Output Threshold for Preferred Location of Reduce Tasks
MapOutputTrackerMaster defines 0.2 as the fraction of total map output that must be at a location for it to considered as a preferred location for a reduce task.
Making this larger will focus on fewer locations where most data can be read locally, but may lead to more delay in scheduling if those locations are busy.
MapOutputTrackerMaster uses the fraction when requested for the preferred locations of shuffle RDDs.
MessageLoop is a thread of execution to handle GetMapOutputMessages until a PoisonPill marker message arrives (when MapOutputTrackerMaster is requested to stop).
MessageLoop takes a GetMapOutputMessage and prints out the following DEBUG message to the logs:
Handling request to send map output locations for shuffle [shuffleId] to [hostPort]\n
MessageLoop then finds the ShuffleStatus by the shuffle ID in the shuffleStatuses internal registry and replies back (to the RPC client) with a serialized map output status (with the BroadcastManager and spark.shuffle.mapOutput.minSizeForBroadcast configuration property).
MessageLoop threads run on the map-output-dispatcher Thread Pool.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#map-output-dispatcher-thread-pool","title":"map-output-dispatcher Thread Pool
threadpool: ThreadPoolExecutor\n
threadpool is a daemon fixed thread pool registered with map-output-dispatcher thread name prefix.
threadpool uses spark.shuffle.mapOutput.dispatcher.numThreads configuration property for the number of MessageLoop dispatcher threads to process received GetMapOutputMessage messages.
The dispatcher threads are started immediately when MapOutputTrackerMaster is created.
The thread pool is shut down when MapOutputTrackerMaster is requested to stop.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#epoch-number","title":"Epoch Number
MapOutputTrackerMaster uses an epoch number to...FIXME
getEpoch is used when:
DAGScheduler is requested to removeExecutorAndUnregisterOutputs
TaskSetManager is created (and sets the epoch to tasks)
getPreferredLocationsForShuffle computes the locations (BlockManagers) with the most shuffle map outputs for the input ShuffleDependency and Partition.
getPreferredLocationsForShuffle computes the locations when all of the following are met:
spark.shuffle.reduceLocality.enabled configuration property is enabled
The number of \"map\" partitions (of the RDD of the input ShuffleDependency) is below SHUFFLE_PREF_MAP_THRESHOLD
The number of \"reduce\" partitions (of the Partitioner of the input ShuffleDependency) is below SHUFFLE_PREF_REDUCE_THRESHOLD
Note
getPreferredLocationsForShuffle is simply getLocationsWithLargestOutputs with a guard condition.
Internally, getPreferredLocationsForShuffle checks whether spark.shuffle.reduceLocality.enabled configuration property is enabled with the number of partitions of the RDD of the input ShuffleDependency and partitions in the partitioner of the input ShuffleDependency both being less than 1000.
Note
The thresholds for the number of partitions in the RDD and of the partitioner when computing the preferred locations are 1000 and are not configurable.
If the condition holds, getPreferredLocationsForShuffle finds locations with the largest number of shuffle map outputs for the input ShuffleDependency and partitionId (with the number of partitions in the partitioner of the input ShuffleDependency and 0.2) and returns the hosts of the preferred BlockManagers.
Note
0.2 is the fraction of total map output that must be at a location to be considered as a preferred location for a reduce task. It is not configurable.
getPreferredLocationsForShuffle is used when ShuffledRDD and Spark SQL's ShuffledRowRDD are requested for preferred locations of a partition.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#finding-locations-with-largest-number-of-shuffle-map-outputs","title":"Finding Locations with Largest Number of Shuffle Map Outputs
getLocationsWithLargestOutputs returns BlockManagerIds with the largest size (of all the shuffle blocks they manage) above the input fractionThreshold (given the total size of all the shuffle blocks for the shuffle across all BlockManagers).
Note
getLocationsWithLargestOutputs may return no BlockManagerId if their shuffle blocks do not total up above the input fractionThreshold.
Note
The input numReducers is not used.
Internally, getLocationsWithLargestOutputs queries the mapStatuses internal cache for the input shuffleId.
Note
One entry in mapStatuses internal cache is a MapStatus array indexed by partition id.
MapStatus includes information about the BlockManager (as BlockManagerId) and estimated size of the reduce blocks.
getLocationsWithLargestOutputs iterates over the MapStatus array and builds an interim mapping between BlockManagerId and the cumulative sum of shuffle blocks across BlockManagers.
incrementEpoch prints out the following DEBUG message to the logs:
Increasing epoch to [epoch]\n
incrementEpoch is used when:
MapOutputTrackerMaster is requested to unregisterMapOutput, unregisterAllMapOutput, removeOutputsOnHost and removeOutputsOnExecutor
DAGScheduler is requested to handle a ShuffleMapTask completion (of a ShuffleMapStage)
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#checking-availability-of-shuffle-map-output-status","title":"Checking Availability of Shuffle Map Output Status
containsShuffle(\n shuffleId: Int): Boolean\n
containsShuffle checks if the input shuffleId is registered in the cachedSerializedStatuses or mapStatuses internal caches.
containsShuffle is used when DAGScheduler is requested to create a createShuffleMapStage (for a ShuffleDependency).
registerShuffle registers a new ShuffleStatus (for the given shuffle ID and the number of partitions) to the shuffleStatuses internal registry.
registerShuffle throws an IllegalArgumentException when the shuffle ID has already been registered:
Shuffle ID [shuffleId] registered twice\n
registerShuffle is used when:
DAGScheduler is requested to create a ShuffleMapStage (for a ShuffleDependency)
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#registering-map-outputs-for-shuffle-possibly-with-epoch-change","title":"Registering Map Outputs for Shuffle (Possibly with Epoch Change)
getSerializedMapOutputStatuses finds cached serialized map statuses for the input shuffleId.
If found, getSerializedMapOutputStatuses returns the cached serialized map statuses.
Otherwise, getSerializedMapOutputStatuses acquires the shuffle lock for shuffleId and finds cached serialized map statuses again since some other thread could not update the cachedSerializedStatuses internal cache.
getSerializedMapOutputStatuses returns the serialized map statuses if found.
If not, getSerializedMapOutputStatuses serializes the local array of MapStatuses (from checkCachedStatuses).
getSerializedMapOutputStatuses prints out the following INFO message to the logs:
Size of output statuses for shuffle [shuffleId] is [bytes] bytes\n
getSerializedMapOutputStatuses saves the serialized map output statuses in cachedSerializedStatuses internal cache if the epoch has not changed in the meantime. getSerializedMapOutputStatuses also saves its broadcast version in cachedSerializedBroadcast internal cache.
If the epoch has changed in the meantime, the serialized map output statuses and their broadcast version are not saved, and getSerializedMapOutputStatuses prints out the following INFO message to the logs:
Epoch changed, not caching!\n
getSerializedMapOutputStatuses removes the broadcast.
getSerializedMapOutputStatuses returns the serialized map statuses.
getSerializedMapOutputStatuses is used when MapOutputTrackerMaster responds to GetMapOutputMessage requests and DAGScheduler creates ShuffleMapStage for ShuffleDependency (copying the shuffle map output locations from previous jobs to avoid unnecessarily regenerating data).
checkCachedStatuses is an internal helper method that <> uses to do some bookkeeping (when the <> and <> differ) and set local statuses, retBytes and epochGotten (that getSerializedMapOutputStatuses uses).
Internally, checkCachedStatuses acquires the MapOutputTracker.md#epochLock[epochLock lock] and checks the status of <> to <cacheEpoch>>.
If epoch is younger (i.e. greater), checkCachedStatuses clears <> internal cache, <> and sets cacheEpoch to be epoch.
checkCachedStatuses gets the serialized map output statuses for the shuffleId (of the owning <>).
When the serialized map output status is found, checkCachedStatuses saves it in a local retBytes and returns true.
When not found, you should see the following DEBUG message in the logs:
cached status not found for : [shuffleId]\n
checkCachedStatuses uses MapOutputTracker.md#mapStatuses[mapStatuses] internal cache to get map output statuses for the shuffleId (of the owning <>) or falls back to an empty array and sets it to a local statuses. checkCachedStatuses sets the local epochGotten to the current <> and returns false.","text":""},{"location":"scheduler/MapOutputTrackerMaster/#registering-shuffle-map-output","title":"Registering Shuffle Map Output
registerMapOutput finds the ShuffleStatus by the given shuffle ID and adds the given MapStatus:
The given mapId is the partitionId of the ShuffleMapTask that finished.
The given shuffleId is the shuffleId of the ShuffleDependency of the ShuffleMapStage (for which the ShuffleMapTask completed)
registerMapOutput is used when DAGScheduler is requested to handle a ShuffleMapTask completion.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#map-output-statistics-for-shuffledependency","title":"Map Output Statistics for ShuffleDependency
getStatistics requests the input ShuffleDependency for the shuffle ID and looks up the corresponding ShuffleStatus (in the shuffleStatuses registry).
getStatistics assumes that the ShuffleStatus is in shuffleStatuses registry.
getStatistics requests the ShuffleStatus for the MapStatuses (of the ShuffleDependency).
getStatistics uses the spark.shuffle.mapOutput.parallelAggregationThreshold configuration property to decide on parallelism to calculate the statistics.
With no parallelism, getStatistics simply traverses over the MapStatuses and requests them (one by one) for the size of every shuffle block.
Note
getStatistics requests the given ShuffleDependency for the Partitioner that in turn is requested for the number of partitions.
The number of blocks is the number of MapStatuses multiplied by the number of partitions.
And hence the need for parallelism based on the spark.shuffle.mapOutput.parallelAggregationThreshold configuration property.
In the end, getStatistics creates a MapOutputStatistics with the shuffle ID (of the given ShuffleDependency) and the total sizes (sumed up for every partition).
getStatistics is used when:
DAGScheduler is requested to handle a successful ShuffleMapStage submission and markMapStageJobsAsFinished
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#deregistering-all-map-outputs-of-shuffle-stage","title":"Deregistering All Map Outputs of Shuffle Stage
unregisterAllMapOutput(\n shuffleId: Int): Unit\n
unregisterAllMapOutput...FIXME
unregisterAllMapOutput is used when DAGScheduler is requested to handle a task completion (due to a fetch failure).
removeOutputsOnExecutor is used when DAGScheduler is requested to removeExecutorAndUnregisterOutputs.
","text":""},{"location":"scheduler/MapOutputTrackerMaster/#number-of-partitions-with-shuffle-map-outputs-available","title":"Number of Partitions with Shuffle Map Outputs Available
getNumAvailableOutputs(\n shuffleId: Int): Int\n
getNumAvailableOutputs...FIXME
getNumAvailableOutputs is used when ShuffleMapStage is requested for the number of partitions with shuffle outputs available.
Posted when MapOutputTrackerWorker is requested for shuffle map outputs for a given shuffle ID
When received, MapOutputTrackerMasterEndpoint prints out the following INFO message to the logs:
Asked to send map output locations for shuffle [shuffleId] to [hostPort]\n
In the end, MapOutputTrackerMasterEndpoint requests the MapOutputTrackerMaster to post a GetMapOutputMessage (with the input shuffleId). Whatever is returned from MapOutputTrackerMaster becomes the response.
MapOutputTrackerWorker is the MapOutputTracker for executors.
MapOutputTrackerWorker uses Java's thread-safe java.util.concurrent.ConcurrentHashMap for mapStatuses internal cache and any lookup cache miss triggers a fetch from the driver's MapOutputTrackerMaster.
getStatuses finds MapStatus.md[MapStatuses] for the input shuffleId in the <> internal cache and, when not available, fetches them from a remote MapOutputTrackerMaster.md[MapOutputTrackerMaster] (using RPC).
Internally, getStatuses first queries the <mapStatuses internal cache>> and returns the map outputs if found.
If not found (in the mapStatuses internal cache), you should see the following INFO message in the logs:
Don't have map outputs for shuffle [id], fetching them\n
If some other process fetches the map outputs for the shuffleId (as recorded in fetching internal registry), getStatuses waits until it is done.
When no other process fetches the map outputs, getStatuses registers the input shuffleId in fetching internal registry (of shuffle map outputs being fetched).
You should see the following INFO message in the logs:
Doing the fetch; tracker endpoint = [trackerEndpoint]\n
getStatuses sends a GetMapOutputStatuses RPC remote message for the input shuffleId to the trackerEndpoint expecting a Array[Byte].
NOTE: getStatuses requests shuffle map outputs remotely within a timeout and with retries. Refer to rpc:RpcEndpointRef.md[RpcEndpointRef].
getStatuses <> and records the result in the <mapStatuses internal cache>>.
You should see the following INFO message in the logs:
Got the output locations\n
getStatuses removes the input shuffleId from fetching internal registry.
You should see the following DEBUG message in the logs:
Fetching map output statuses for shuffle [id] took [time] ms\n
If getStatuses could not find the map output locations for the input shuffleId (locally and remotely), you should see the following ERROR message in the logs and throws a MetadataFetchFailedException.
Missing all output locations for shuffle [id]\n
NOTE: getStatuses is used when MapOutputTracker <> and <ShuffleDependency>>.
== [[logging]] Logging
Enable ALL logging level for org.apache.spark.MapOutputTrackerWorker logger to see what happens inside.
MapStatus is an abstraction of shuffle map output statuses with an estimated size, location and map Id.
MapStatus is a result of executing a ShuffleMapTask.
After a ShuffleMapTask has finished execution successfully, DAGScheduler is requested to handle a ShuffleMapTask completion that in turn requests the MapOutputTrackerMaster to register the MapStatus.
MapStatus utility uses spark.shuffle.minNumPartitionsToHighlyCompress internal configuration property for the minimum number of partitions to prefer a HighlyCompressedMapStatus.
apply creates a HighlyCompressedMapStatus when the number of uncompressedSizes is above minPartitionsToUseHighlyCompressMapStatus threshold. Otherwise, apply creates a CompressedMapStatus.
apply is used when:
SortShuffleWriter is requested to write records
BypassMergeSortShuffleWriter is requested to write records
UnsafeShuffleWriter is requested to close resources and write out merged spill files
Pool is a scheduler:spark-scheduler-Schedulable.md[Schedulable] entity that represents a tree of scheduler:TaskSetManager.md[TaskSetManagers], i.e. it contains a collection of TaskSetManagers or the Pools thereof.
A Pool has a mandatory name, a spark-scheduler-SchedulingMode.md[scheduling mode], initial minShare and weight that are defined when it is created.
NOTE: An instance of Pool is created when scheduler:TaskSchedulerImpl.md#initialize[TaskSchedulerImpl is initialized].
NOTE: The scheduler:TaskScheduler.md#contract[TaskScheduler Contract] and spark-scheduler-Schedulable.md#contract[Schedulable Contract] both require that their entities have rootPool of type Pool.
Using the spark-scheduler-SchedulingMode.md[scheduling mode] (given when a Pool object is created), Pool selects <> and sets taskSetSchedulingAlgorithm:
<> for FIFO scheduling mode.
<> for FAIR scheduling mode.
It throws an IllegalArgumentException when unsupported scheduling mode is passed on:
NOTE: getSortedTaskSetQueue is part of the spark-scheduler-Schedulable.md#contract[Schedulable Contract].
getSortedTaskSetQueue sorts all the spark-scheduler-Schedulable.md[Schedulables] in spark-scheduler-Schedulable.md#contract[schedulableQueue] queue by a <> (from the internal <>).
NOTE: It is called when scheduler:TaskSchedulerImpl.md#resourceOffers[TaskSchedulerImpl processes executor resource offers].
=== [[schedulableNameToSchedulable]] Schedulables by Name -- schedulableNameToSchedulable Registry
"},{"location":"scheduler/Pool/#source-scala","title":"[source, scala]","text":""},{"location":"scheduler/Pool/#schedulablenametoschedulable-new-concurrenthashmapstring-schedulable","title":"schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable]","text":"
schedulableNameToSchedulable is a lookup table of spark-scheduler-Schedulable.md[Schedulable] objects by their names.
Beside the obvious usage in the housekeeping methods like addSchedulable, removeSchedulable, getSchedulableByName from the spark-scheduler-Schedulable.md#contract[Schedulable Contract], it is exclusively used in SparkContext.md#getPoolForName[SparkContext.getPoolForName].
=== [[addSchedulable]] addSchedulable Method
NOTE: addSchedulable is part of the spark-scheduler-Schedulable.md#contract[Schedulable Contract].
addSchedulable adds a Schedulable to the spark-scheduler-Schedulable.md#contract[schedulableQueue] and <>.
More importantly, it sets the Schedulable entity's spark-scheduler-Schedulable.md#contract[parent] to itself.
ResultTask[T, U] is a Task that executes a partition processing function on a partition with records (of type T) to produce a result (of type U) that is sent back to the driver.
== [[SchedulableBuilder]] SchedulableBuilder Contract -- Builders of Schedulable Pools
SchedulableBuilder is the <> of <> that manage a <>, which is to <> and <>.
SchedulableBuilder is a private[spark] Scala trait that is used exclusively by scheduler:TaskSchedulerImpl.md[TaskSchedulerImpl] (the default Spark scheduler). When requested to scheduler:TaskSchedulerImpl.md#initialize[initialize], TaskSchedulerImpl uses the configuration-properties.md#spark.scheduler.mode[spark.scheduler.mode] configuration property (default: FIFO) to select one of the <>.
Used exclusively when TaskSchedulerImpl is requested to scheduler:TaskSchedulerImpl.md#submitTasks[submit tasks (of TaskSet) for execution] (and registers a new scheduler:TaskSetManager.md[TaskSetManager] for the TaskSet)
Used exclusively when TaskSchedulerImpl is requested to scheduler:TaskSchedulerImpl.md#initialize[initialize] (and creates a scheduler:TaskSchedulerImpl.md#schedulableBuilder[SchedulableBuilder] per configuration-properties.md#spark.scheduler.mode[spark.scheduler.mode] configuration property)
| <> | [[FairSchedulableBuilder]] Used when the configuration-properties.md#spark.scheduler.mode[spark.scheduler.mode] configuration property is FAIR
| <> | [[FIFOSchedulableBuilder]] Default SchedulableBuilder that is used when the configuration-properties.md#spark.scheduler.mode[spark.scheduler.mode] configuration property is FIFO (default)
SchedulerBackend is an abstraction of task scheduling backends that can revive resource offers from cluster managers.
SchedulerBackend abstraction allows TaskSchedulerImpl to use variety of cluster managers (with their own resource offers and task scheduling modes).
Note
Being a scheduler backend system assumes a Apache Mesos-like scheduling model in which \"an application\" gets resource offers as machines become available so it is possible to launch tasks on them. Once required resource allocation is obtained, the scheduler backend can start executors.
"},{"location":"scheduler/SchedulerBackendUtils/","title":"SchedulerBackendUtils Utility","text":""},{"location":"scheduler/SchedulerBackendUtils/#default-number-of-executors","title":"Default Number of Executors
SchedulerBackendUtils defaults to 2 as the default number of executors.
getInitialTargetExecutorNumber(\n conf: SparkConf,\n numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int\n
getInitialTargetExecutorNumber branches off based on whether Dynamic Allocation of Executors is enabled or not.
With no Dynamic Allocation of Executors, getInitialTargetExecutorNumber uses the spark.executor.instances configuration property (if defined) or uses the given numExecutors (and the DEFAULT_NUMBER_EXECUTORS).
With Dynamic Allocation of Executors enabled, getInitialTargetExecutorNumber getDynamicAllocationInitialExecutors and makes sure that the value is between the following configuration properties:
spark.dynamicAllocation.minExecutors
spark.dynamicAllocation.maxExecutors
getInitialTargetExecutorNumber is used when:
KubernetesClusterSchedulerBackend (Spark on Kubernetes) is created
Spark on YARN's YarnAllocator, YarnClientSchedulerBackend and YarnClusterSchedulerBackend are used
Scheduling Mode (aka order task policy or scheduling policy or scheduling order) defines a policy to sort tasks in order for execution.
The scheduling mode schedulingMode attribute is part of the scheduler:TaskScheduler.md#schedulingMode[TaskScheduler Contract].
The only implementation of the TaskScheduler contract in Spark -- scheduler:TaskSchedulerImpl.md[TaskSchedulerImpl] -- uses configuration-properties.md#spark.scheduler.mode[spark.scheduler.mode] setting to configure schedulingMode that is merely used to set up the scheduler:TaskScheduler.md#rootPool[rootPool] attribute (with FIFO being the default). It happens when scheduler:TaskSchedulerImpl.md#initialize[TaskSchedulerImpl is initialized].
There are three acceptable scheduling modes:
[[FIFO]] FIFO with no pools but a single top-level unnamed pool with elements being scheduler:TaskSetManager.md[TaskSetManager] objects; lower priority gets scheduler:spark-scheduler-Schedulable.md[Schedulable] sooner or earlier stage wins.
[[FAIR]] FAIR with a scheduler:spark-scheduler-FairSchedulableBuilder.md#buildPools[hierarchy of Schedulable (sub)pools] with the scheduler:TaskScheduler.md#rootPool[rootPool] at the top.
[[NONE]] NONE (not used)
NOTE: Out of three possible SchedulingMode policies only FIFO and FAIR modes are supported by scheduler:TaskSchedulerImpl.md[TaskSchedulerImpl].
After the root pool is initialized, the scheduling mode is no longer relevant (since the spark-scheduler-Schedulable.md[Schedulable] that represents the root pool is fully set up).
"},{"location":"scheduler/SchedulingMode/#the-root-pool-is-later-used-when-schedulertaskschedulerimplmdsubmittaskstaskschedulerimpl-submits-tasks-as-tasksets-for-execution","title":"The root pool is later used when scheduler:TaskSchedulerImpl.md#submitTasks[TaskSchedulerImpl submits tasks (as TaskSets) for execution].","text":"
NOTE: The scheduler:TaskScheduler.md#rootPool[root pool] is a Schedulable. Refer to spark-scheduler-Schedulable.md[Schedulable].
=== [[fair-scheduling-sparkui]] Monitoring FAIR Scheduling Mode using Spark UI
ShuffleMapStage (shuffle map stage or simply map stage) is a Stage.
ShuffleMapStage corresponds to (and is associated with) a ShuffleDependency.
ShuffleMapStage can be submitted independently but it is usually an intermediate step in a physical execution plan (with the final step being a ResultStage).
findMissingPartitions requests the MapOutputTrackerMaster for the missing partitions (of the ShuffleDependency) and returns them.
If not available (MapOutputTrackerMaster does not track the ShuffleDependency), findMissingPartitions simply assumes that all the partitions are missing.
findMissingPartitions is part of the Stage abstraction.
isAvailable is true when the ShuffleMapStage is ready and all partitions have shuffle outputs (i.e. the numAvailableOutputs is exactly the numPartitions).
isAvailable is used when:
DAGScheduler is requested to getMissingParentStages, handleMapStageSubmitted, submitMissingTasks, processShuffleMapStageCompletion, markMapStageJobsAsFinished and stageDependsOn
A ShuffleMapStage can be shared across multiple jobs (if these jobs reuse the same RDDs).
val keyValuePairs = sc.parallelize(0 to 5).map((_, 1))\nval rdd = keyValuePairs.sortByKey() // (1)\n\nscala> println(rdd.toDebugString)\n(6) ShuffledRDD[4] at sortByKey at <console>:39 []\n +-(16) MapPartitionsRDD[1] at map at <console>:39 []\n | ParallelCollectionRDD[0] at parallelize at <console>:39 []\n\nrdd.count // (2)\nrdd.count // (3)\n
Shuffle at sortByKey()
Submits a job with two stages (and two to be executed)
Intentionally repeat the last action that submits a new job with two stages with one being shared as already-computed
ShuffleMapTask is a Task to produce a MapStatus (Task[MapStatus]).
ShuffleMapTask is one of the two types of Tasks. When executed, ShuffleMapTask writes the result of executing a serialized task code over the records (of a RDD partition) to the shuffle system and returns a MapStatus (with the BlockManager and estimated size of the result shuffle blocks).
ShuffleMapTask tracks TaskLocations as unique entries in the given locs (with the only rule that when locs is not defined, it is empty, and no task location preferences are defined).
ShuffleMapTask initializes the preferredLocs internal property when created
runTask writes the result (records) of executing the serialized task code over the records (in the RDD partition) to the shuffle system and returns a MapStatus (with the BlockManager and an estimated size of the result shuffle blocks).
Internally, runTask requests the SparkEnv for the new instance of closure serializer and requests it to deserialize the serialized task code (into a tuple of a RDD and a ShuffleDependency).
runTask measures the thread and CPU deserialization times.
runTask requests the SparkEnv for the ShuffleManager and requests it for a ShuffleWriter (for the ShuffleHandle and the partition).
runTask then requests the RDD for the records (of the partition) that the ShuffleWriter is requested to write out (to the shuffle system).
In the end, runTask requests the ShuffleWriter to stop (with the success flag on) and returns the shuffle map output status.
Note
This is the moment in Task's lifecycle (and its corresponding RDD) when a RDD partition is computed and in turn becomes a sequence of records (i.e. real data) on an executor.
In case of any exceptions, runTask requests the ShuffleWriter to stop (with the success flag off) and (re)throws the exception.
runTask may also print out the following DEBUG message to the logs when the ShuffleWriter could not be stopped.
Number of Partitions (of the RDD of the ShuffleDependency of a ShuffleMapStage)
ShuffleStatus is created\u00a0when:
MapOutputTrackerMaster is requested to register a shuffle (when DAGScheduler is requested to create a ShuffleMapStage)
"},{"location":"scheduler/ShuffleStatus/#mapstatuses-per-partition","title":"MapStatuses per Partition
ShuffleStatus creates a mapStatuses internal registry of MapStatuses per partition (using the numPartitions) when created.
A missing partition is when there is no MapStatus for a partition (null at the index of the partition ID) and can be requested using findMissingPartitions.
mapStatuses is all null (for every partition) initially (and so all partitions are missing / uncomputed).
A new MapStatus is added in addMapOutput and updateMapOutput.
A MapStatus is removed (nulled) in removeMapOutput and removeOutputsByFilter.
The number of available MapStatuses is tracked by _numAvailableMapOutputs internal counter.
addMapOutput adds the MapStatus to the mapStatuses internal registry.
In case the mapStatuses internal registry had no MapStatus for the mapIndex already available, addMapOutput increments the _numAvailableMapOutputs internal counter and invalidateSerializedMapOutputStatusCache.
addMapOutput\u00a0is used when:
MapOutputTrackerMaster is requested to registerMapOutput
Stage is an abstraction of steps in a physical execution plan.
Note
The logical DAG or logical execution plan is the RDD lineage.
Indirectly, a Stage is a set of parallel tasks - one task per partition (of an RDD that computes partial results of a function executed as part of a Spark job).
In other words, a Spark job is a computation \"sliced\" (not to use the reserved term partitioned) into stages.
DAGScheduler is requested to submit missing tasks of a stage
Abstract Class
Task\u00a0is an abstract class and cannot be created directly. It is created indirectly for the concrete Tasks.
"},{"location":"scheduler/Task/#isBarrier","title":"isBarrier Flag
Task can be given isBarrier flag when created. Unless given, isBarrier is assumed disabled (false).
isBarrier flag indicates whether this Task belongs to a Barrier Stage in Barrier Execution Mode.
isBarrier flag is used when:
DAGScheduler is requested to handleTaskCompletion (of a FetchFailed task) to fail the parent stage (and retry a barrier stage when one of the barrier tasks fails)
Task is requested to run (to create a BarrierTaskContext)
TaskSetManager is requested to isBarrier and handleFailedTask
run registers the task (attempt) with the BlockManager.
run creates a TaskContextImpl (and perhaps a BarrierTaskContext too when the given isBarrier flag is enabled) that in turn becomes the task's TaskContext.
run checks _killed flag and, if enabled, kills the task (with interruptThread flag disabled).
run creates a Hadoop CallerContext and sets it.
run informs the given PluginContainer that the task is started.
run runs the task.
Note
This is the moment when the custom Task's runTask is executed.
In the end, run notifies TaskContextImpl that the task has completed (regardless of the final outcome -- a success or a failure).
In case of any exceptions, run notifies TaskContextImpl that the task has failed. run requests MemoryStore to release unroll memory for this task (for both ON_HEAP and OFF_HEAP memory modes).
Note
run uses SparkEnv to access the current BlockManager that it uses to access MemoryStore.
run requests MemoryManager to notify any tasks waiting for execution memory to be freed to wake up and try to acquire memory again.
run unsets the task's TaskContext.
Note
run uses SparkEnv to access the current MemoryManager.
run is used when:
TaskRunner is requested to run (when Executor is requested to launch a task (on \"Executor task launch worker\" thread pool sometime in the future))
","text":""},{"location":"scheduler/Task/#task-states","title":"Task States
Task can be in one of the following states (as described by TaskState enumeration):
LAUNCHING
RUNNING when the task is being started.
FINISHED when the task finished with the serialized result.
FAILED when the task fails, e.g. when FetchFailedException, CommitDeniedException or any Throwable occurs
KILLED when an executor kills a task.
LOST
States are the values of org.apache.spark.TaskState.
Note
Task status updates are sent from executors to the driver through ExecutorBackend.
Task is finished when it is in one of FINISHED, FAILED, KILLED, LOST.
LOST and FAILED states are considered failures.
","text":""},{"location":"scheduler/Task/#collecting-latest-values-of-accumulators","title":"Collecting Latest Values of Accumulators
collectAccumulatorUpdates collects the latest values of internal and external accumulators from a task (and returns the values as a collection of AccumulableInfo).
TaskInfo is information about a running task attempt inside a scheduler:TaskSet.md[TaskSet].
TaskInfo is created when:
scheduler:TaskSetManager.md#resourceOffer[TaskSetManager dequeues a task for execution (given resource offer)] (and records the task as running)
TaskUIData does dropInternalAndSQLAccumulables
JsonProtocol utility is used to spark-history-server:JsonProtocol.md#taskInfoFromJson[re-create a task details from JSON]
NOTE: Back then, at the commit 63051dd2bcc4bf09d413ff7cf89a37967edc33ba, when TaskInfo was first merged to Apache Spark on 07/06/12, TaskInfo was part of spark.scheduler.mesos package -- note \"Mesos\" in the name of the package that shows how much Spark and Mesos influenced each other at that time.
[[internal-registries]] .TaskInfo's Internal Registries and Counters [cols=\"1,2\",options=\"header\",width=\"100%\"] |=== | Name | Description
| [[finishTime]] finishTime | Time when TaskInfo was <>.
[[index]] Index of the task within its scheduler:TaskSet.md[TaskSet] that may not necessarily be the same as the ID of the RDD partition that the task is computing.
[[attemptNumber]] Task attempt ID
[[launchTime]] Time when the task was dequeued for execution
[[executorId]] Executor that has been offered (as a resource) to run the task
[[host]] Host of the <>
[[taskLocality]] scheduler:TaskSchedulerImpl.md#TaskLocality[TaskLocality], i.e. locality preference of the task
[[speculative]] Flag whether a task is speculative or not
TaskInfo initializes the <>.
=== [[markFinished]] Marking Task As Finished (Successfully or Not) -- markFinished Method
"},{"location":"scheduler/TaskInfo/#source-scala","title":"[source, scala]","text":""},{"location":"scheduler/TaskInfo/#markfinishedstate-taskstate-time-long-systemcurrenttimemillis-unit","title":"markFinished(state: TaskState, time: Long = System.currentTimeMillis): Unit","text":"
markFinished records the input time as <>.
markFinished marks TaskInfo as <> when the input state is FAILED or <> for state being KILLED.
NOTE: markFinished is used when TaskSetManager is notified that a task has finished scheduler:TaskSetManager.md#handleSuccessfulTask[successfully] or scheduler:TaskSetManager.md#handleFailedTask[failed].
TaskLocation represents a placement preference of an RDD partition, i.e. a hint of the location to submit scheduler:Task.md[tasks] for execution.
TaskLocations are tracked by scheduler:DAGScheduler.md#cacheLocs[DAGScheduler] for scheduler:DAGScheduler.md#submitMissingTasks[submitting missing tasks of a stage].
TaskLocation is available as scheduler:Task.md#preferredLocations[preferredLocations] of a task.
[[host]] Every TaskLocation describes the location by host name, but could also use other location-related metadata.
TaskLocations of an RDD and a partition is available using SparkContext.md#getPreferredLocs[SparkContext.getPreferredLocs] method.
Sealed
TaskLocation is a Scala private[spark] sealed trait so all the available implementations of TaskLocation trait are in a single Scala file.
ExecutorCacheTaskLocation describes a <> and an executor.
ExecutorCacheTaskLocation informs the Scheduler to prefer a given executor, but the next level of preference is any executor on the same host if this is not possible.
HDFSCacheTaskLocation describes a <> that is cached by HDFS.
Used exclusively when rdd:HadoopRDD.md#getPreferredLocations[HadoopRDD] and rdd:NewHadoopRDD.md#getPreferredLocations[NewHadoopRDD] are requested for their placement preferences (aka preferred locations).
== [[HostTaskLocation]] HostTaskLocation
HostTaskLocation describes a <> only."},{"location":"scheduler/TaskResult/","title":"TaskResult","text":"
TaskResult is an abstraction of task results (of type T).
The decision what TaskResult type to use is made when TaskRunner finishes running a task.
Sealed Trait
TaskResult is a Scala sealed trait which means that all of the implementations are in the same compilation unit (a single file).
TaskResultGetter is a helper class of scheduler:TaskSchedulerImpl.md#statusUpdate[TaskSchedulerImpl] for asynchronous deserialization of <> (possibly fetching remote blocks) or <>.
CAUTION: FIXME Image with the dependencies
TIP: Consult scheduler:Task.md#states[Task States] in Tasks to learn about the different task states.
NOTE: The only instance of TaskResultGetter is created while scheduler:TaskSchedulerImpl.md#creating-instance[TaskSchedulerImpl is created].
TaskResultGetter requires a core:SparkEnv.md[SparkEnv] and scheduler:TaskSchedulerImpl.md[TaskSchedulerImpl] to be created and is stopped when scheduler:TaskSchedulerImpl.md#stop[TaskSchedulerImpl stops].
TaskResultGetter uses <task-result-getter asynchronous task executor>> for operation."},{"location":"scheduler/TaskResultGetter/#tip","title":"[TIP]","text":"
Enable DEBUG logging level for org.apache.spark.scheduler.TaskResultGetter logger to see what happens inside.
getTaskResultExecutor creates a daemon thread pool with <> threads and task-result-getter prefix.
TIP: Read up on https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ThreadPoolExecutor.html[java.util.concurrent.ThreadPoolExecutor] that getTaskResultExecutor uses under the covers.
serializer is a thread-local serializer:SerializerInstance.md[SerializerInstance] that TaskResultGetter uses to deserialize byte buffers (with TaskResults or a TaskEndReason).
When created for a new thread, serializer is initialized with a new instance of Serializer (using core:SparkEnv.md#closureSerializer[SparkEnv.closureSerializer]).
NOTE: TaskResultGetter uses https://docs.oracle.com/javase/8/docs/api/java/lang/ThreadLocal.html[java.lang.ThreadLocal] for the thread-local SerializerInstance variable.
taskResultSerializer is a thread-local serializer:SerializerInstance.md[SerializerInstance] that TaskResultGetter uses to...
When created for a new thread, taskResultSerializer is initialized with a new instance of Serializer (using core:SparkEnv.md#serializer[SparkEnv.serializer]).
NOTE: TaskResultGetter uses https://docs.oracle.com/javase/8/docs/api/java/lang/ThreadLocal.html[java.lang.ThreadLocal] for the thread-local SerializerInstance variable.
enqueueSuccessfulTask submits an asynchronous task (to <> asynchronous task executor) that first deserializes serializedData to a DirectTaskResult, then updates the internal accumulator (with the size of the DirectTaskResult) and ultimately notifies the TaskSchedulerImpl that the tid task was completed and scheduler:TaskSchedulerImpl.md#handleSuccessfulTask[the task result was received successfully] or scheduler:TaskSchedulerImpl.md#handleFailedTask[not].
NOTE: enqueueSuccessfulTask is just the asynchronous task enqueued for execution by <> asynchronous task executor at some point in the future.
Internally, the enqueued task first deserializes serializedData to a TaskResult (using the internal thread-local <>).
For a DirectTaskResult, the task scheduler:TaskSetManager.md#canFetchMoreResults[checks the available memory for the task result] and, when the size overflows configuration-properties.md#spark.driver.maxResultSize[spark.driver.maxResultSize], it simply returns.
Note
enqueueSuccessfulTask is a mere thread so returning from a thread is to do nothing else. That is why the check for quota does abort when there is not enough memory.
Otherwise, when there is enough memory to hold the task result, it deserializes the DirectTaskResult (using the internal thread-local <>).
For an IndirectTaskResult, the task checks the available memory for the task result and, when the size could overflow the maximum result size, it storage:BlockManagerMaster.md#removeBlock[removes the block] and simply returns.
Otherwise, when there is enough memory to hold the task result, you should see the following DEBUG message in the logs:
Fetching indirect task result for TID [tid]\n
The task scheduler:TaskSchedulerImpl.md#handleTaskGettingResult[notifies TaskSchedulerImpl that it is about to fetch a remote block for a task result]. It then storage:BlockManager.md#getRemoteBytes[gets the block from remote block managers (as serialized bytes)].
When the block could not be fetched, scheduler:TaskSchedulerImpl.md#handleFailedTask[TaskSchedulerImpl is informed] (with TaskResultLost task failure reason) and the task simply returns.
NOTE: enqueueSuccessfulTask is a mere thread so returning from a thread is to do nothing else and so the real handling is when scheduler:TaskSchedulerImpl.md#handleFailedTask[TaskSchedulerImpl is informed].
The task result (as a serialized byte buffer) is then deserialized to a DirectTaskResult (using the internal thread-local <>) and deserialized again using the internal thread-local <> (just like for the DirectTaskResult case). The storage:BlockManagerMaster.md#removeBlock[block is removed from BlockManagerMaster] and simply returns.
Note
A IndirectTaskResult is deserialized twice to become the final deserialized task result (using <> for a DirectTaskResult). Compare it to a DirectTaskResult task result that is deserialized once only.
With no exceptions thrown, enqueueSuccessfulTask scheduler:TaskSchedulerImpl.md#handleSuccessfulTask[informs the TaskSchedulerImpl that the tid task was completed and the task result was received].
A ClassNotFoundException leads to scheduler:TaskSetManager.md#abort[aborting the TaskSet] (with ClassNotFound with classloader: [loader] error message) while any non-fatal exception shows the following ERROR message in the logs followed by scheduler:TaskSetManager.md#abort[aborting the TaskSet].
Exception while getting task result\n
enqueueSuccessfulTask is used when TaskSchedulerImpl is requested to handle task status update (and the task has finished successfully).
=== [[enqueueFailedTask]] Deserializing TaskFailedReason and Notifying TaskSchedulerImpl -- enqueueFailedTask Method
enqueueFailedTask( taskSetManager: TaskSetManager, tid: Long, taskState: TaskState.TaskState, serializedData: ByteBuffer): Unit
enqueueFailedTask submits an asynchronous task (to <task-result-getter asynchronous task executor>>) that first attempts to deserialize a TaskFailedReason from serializedData (using the internal thread-local <>) and then scheduler:TaskSchedulerImpl.md#handleFailedTask[notifies TaskSchedulerImpl that the task has failed].
Any ClassNotFoundException leads to the following ERROR message in the logs (without breaking the flow of enqueueFailedTask):
ERROR Could not deserialize TaskEndReason: ClassNotFound with classloader [loader]\n
NOTE: enqueueFailedTask is called when scheduler:TaskSchedulerImpl.md#statusUpdate[TaskSchedulerImpl is notified about a task that has failed (and is in FAILED, KILLED or LOST state)].
=== [[settings]] Settings
.Spark Properties [cols=\"1,1,2\",options=\"header\",width=\"100%\"] |=== | Spark Property | Default Value | Description | [[spark_resultGetter_threads]] spark.resultGetter.threads | 4 | The number of threads for TaskResultGetter. |===
TaskScheduler is an abstraction of <> that can <> in a Spark application (per <>).
NOTE: TaskScheduler works closely with scheduler:DAGScheduler.md[DAGScheduler] that <> (for every stage in a Spark job).
TaskScheduler can track the executors available in a Spark application using <> and <> interceptors (that inform about active and lost executors, respectively).
Returns true when the execId executor is managed by the TaskScheduler. false indicates that the executor:Executor.md#reportHeartBeat[block manager (on the executor) should re-register].
Used when HeartbeatReceiver RPC endpoint is requested to handle a Heartbeat (with task metrics) from an executor
TaskSchedulerImpl is requested to scheduler:TaskSchedulerImpl.md#initialize[initialize]
SparkContext is requested to SparkContext.md#getAllPools[getAllPools] and SparkContext.md#getPoolForName[getPoolForName]
TaskSchedulerImpl is requested to scheduler:TaskSchedulerImpl.md#resourceOffers[resourceOffers], scheduler:TaskSchedulerImpl.md#checkSpeculatableTasks[checkSpeculatableTasks], and scheduler:TaskSchedulerImpl.md#removeExecutor[removeExecutor]
A TaskScheduler is created while SparkContext is being created (by calling SparkContext.createTaskScheduler for a given master URL and deploy mode).
At this point in SparkContext's lifecycle, the internal _taskScheduler points at the TaskScheduler (and it is \"announced\" by sending a blocking TaskSchedulerIsSet message to HeartbeatReceiver RPC endpoint).
The <> right after the blocking TaskSchedulerIsSet message receives a response.
The <> and the <> are set at this point (and SparkContext uses the application id to set SparkConf.md#spark.app.id[spark.app.id] Spark property, and configure webui:spark-webui-SparkUI.md[SparkUI], and storage:BlockManager.md[BlockManager]).
CAUTION: FIXME The application id is described as \"associated with the job.\" in TaskScheduler, but I think it is \"associated with the application\" and you can have many jobs per application.
Right before SparkContext is fully initialized, <> is called.
The internal _taskScheduler is cleared (i.e. set to null) while SparkContext.md#stop[SparkContext is being stopped].
<> while scheduler:DAGScheduler.md#stop[DAGScheduler is being stopped].
WARNING: FIXME If it is SparkContext to start a TaskScheduler, shouldn't SparkContext stop it too? Why is this the way it is now?
TaskSchedulerImpl is a TaskScheduler that uses a SchedulerBackend to schedule tasks (for execution on a cluster manager).
When a Spark application starts (and so an instance of SparkContext is created) TaskSchedulerImpl with a SchedulerBackend and DAGScheduler are created and soon started.
TaskSchedulerImpl generates tasks based on executor resource offers.
TaskSchedulerImpl can track racks per host and port (that however is only used with Hadoop YARN cluster manager).
Using spark.scheduler.mode configuration property you can select the scheduling policy.
TaskSchedulerImpl submits tasks using SchedulableBuilders.
TaskSchedulerImpl is given a SchedulerBackend when requested to initialize.
The lifecycle of the SchedulerBackend is tightly coupled to the lifecycle of the TaskSchedulerImpl:
It is started when TaskSchedulerImpl is
It is stopped when TaskSchedulerImpl is
TaskSchedulerImpl waits until the SchedulerBackend is ready before requesting it for the following:
Reviving resource offers when requested to submitTasks, statusUpdate, handleFailedTask, checkSpeculatableTasks, and executorLost
Killing tasks when requested to killTaskAttempt and killAllTaskAttempts
Default parallelism, applicationId and applicationAttemptId when requested for the defaultParallelism, applicationId and applicationAttemptId, respectively
","text":""},{"location":"scheduler/TaskSchedulerImpl/#unique-identifier-of-spark-application","title":"Unique Identifier of Spark Application
applicationId(): String\n
applicationId is part of the TaskScheduler abstraction.
applicationId simply request the SchedulerBackend for the applicationId.
","text":""},{"location":"scheduler/TaskSchedulerImpl/#cancelling-all-tasks-of-stage","title":"Cancelling All Tasks of Stage
statusUpdate finds TaskSetManager for the input tid task (in <>).
When state is LOST, statusUpdate...FIXME
NOTE: TaskState.LOST is only used by the deprecated Mesos fine-grained scheduling mode.
When state is one of the scheduler:Task.md#states[finished states], i.e. FINISHED, FAILED, KILLED or LOST, statusUpdate <> for the input tid.
statusUpdate scheduler:TaskSetManager.md#removeRunningTask[requests TaskSetManager to unregister tid from running tasks].
statusUpdate requests <> to scheduler:TaskResultGetter.md#enqueueSuccessfulTask[schedule an asynchrounous task to deserialize the task result (and notify TaskSchedulerImpl back)] for tid in FINISHED state and scheduler:TaskResultGetter.md#enqueueFailedTask[schedule an asynchrounous task to deserialize TaskFailedReason (and notify TaskSchedulerImpl back)] for tid in the other finished states (i.e. FAILED, KILLED, LOST).
If a task is in LOST state, statusUpdate scheduler:DAGScheduler.md#executorLost[notifies DAGScheduler that the executor was lost] (with SlaveLost and the reason Task [tid] was lost, so marking the executor as lost as well.) and scheduler:SchedulerBackend.md#reviveOffers[requests SchedulerBackend to revive offers].
In case the TaskSetManager for tid could not be found (in <> registry), you should see the following ERROR message in the logs:
Ignoring update with state [state] for TID [tid] because its task set is gone (this is likely the result of receiving duplicate task finished status updates)\n
Any exception is caught and reported as ERROR message in the logs:
Exception in statusUpdate\n
CAUTION: FIXME image with scheduler backends calling TaskSchedulerImpl.statusUpdate.
statusUpdate is used when:
DriverEndpoint (of CoarseGrainedSchedulerBackend) is requested to handle a StatusUpdate message
LocalEndpoint is requested to handle a StatusUpdate message
","text":""},{"location":"scheduler/TaskSchedulerImpl/#task-scheduler-speculation-scheduled-executor-service","title":"task-scheduler-speculation Scheduled Executor Service
speculationScheduler is a java.util.concurrent.ScheduledExecutorService with the name task-scheduler-speculation for Speculative Execution of Tasks.
When TaskSchedulerImpl is requested to start (in non-local run mode) with spark.speculation enabled, speculationScheduler is used to schedule checkSpeculatableTasks to execute periodically every spark.speculation.interval.
speculationScheduler is shut down when TaskSchedulerImpl is requested to stop.
","text":""},{"location":"scheduler/TaskSchedulerImpl/#checking-for-speculatable-tasks","title":"Checking for Speculatable Tasks
checkSpeculatableTasks(): Unit\n
checkSpeculatableTasks requests rootPool to check for speculatable tasks (if they ran for more than 100 ms) and, if there any, requests scheduler:SchedulerBackend.md#reviveOffers[SchedulerBackend to revive offers].
NOTE: checkSpeculatableTasks is executed periodically as part of speculative-execution-of-tasks.md[].
","text":""},{"location":"scheduler/TaskSchedulerImpl/#cleaning-up-after-removing-executor","title":"Cleaning up After Removing Executor
removeExecutor removes the executorId executor from the following <>: <>, executorIdToHost, executorsByHost, and hostsByRack. If the affected hosts and racks are the last entries in executorsByHost and hostsByRack, appropriately, they are removed from the registries.
Unless reason is LossReasonPending, the executor is removed from executorIdToHost registry and Schedulable.md#executorLost[TaskSetManagers get notified].
NOTE: The internal removeExecutor is called as part of <> and scheduler:TaskScheduler.md#executorLost[executorLost].","text":""},{"location":"scheduler/TaskSchedulerImpl/#handling-nearly-completed-sparkcontext-initialization","title":"Handling Nearly-Completed SparkContext Initialization
postStartHook(): Unit\n
postStartHook is part of the TaskScheduler abstraction.
postStartHook waits until a scheduler backend is ready.
","text":""},{"location":"scheduler/TaskSchedulerImpl/#waiting-until-schedulerbackend-is-ready","title":"Waiting Until SchedulerBackend is Ready
waitBackendReady(): Unit\n
waitBackendReady waits until the SchedulerBackend is ready. If it is, waitBackendReady returns immediately. Otherwise, waitBackendReady keeps checking every 100 milliseconds (hardcoded) or the <> is SparkContext.md#stopped[stopped].
Note
A SchedulerBackend is ready by default.
If the SparkContext happens to be stopped while waiting, waitBackendReady throws an IllegalStateException:
stop stops all the internal services, i.e. <task-scheduler-speculation executor service>>, scheduler:SchedulerBackend.md[SchedulerBackend], scheduler:TaskResultGetter.md[TaskResultGetter], and <> timer.","text":""},{"location":"scheduler/TaskSchedulerImpl/#default-level-of-parallelism","title":"Default Level of Parallelism
defaultParallelism(): Int\n
defaultParallelism is part of the TaskScheduler abstraction.
defaultParallelism requests the SchedulerBackend for the default level of parallelism.
Note
Default level of parallelism is a hint for sizing jobs that SparkContext uses to create RDDs with the right number of partitions unless specified explicitly.
","text":""},{"location":"scheduler/TaskSchedulerImpl/#submitting-tasks-of-taskset-for-execution","title":"Submitting Tasks (of TaskSet) for Execution
submitTasks(\n taskSet: TaskSet): Unit\n
submitTasks is part of the TaskScheduler abstraction.
In essence, submitTasks registers a new TaskSetManager (for the given TaskSet) and requests the SchedulerBackend to handle resource allocation offers (from the scheduling system).
Internally, submitTasks prints out the following INFO message to the logs:
Adding task set [id] with [length] tasks\n
submitTasks then <> (for the given TaskSet.md[TaskSet] and the <>).
submitTasks registers (adds) the TaskSetManager per TaskSet.md#stageId[stage] and TaskSet.md#stageAttemptId[stage attempt] IDs (of the TaskSet.md[TaskSet]) in the <> internal registry.
NOTE: <> internal registry tracks the TaskSetManager.md[TaskSetManagers] (that represent TaskSet.md[TaskSets]) per stage and stage attempts. In other words, there could be many TaskSetManagers for a single stage, each representing a unique stage attempt.
NOTE: Not only could a task be retried (cf. <>), but also a single stage.
submitTasks makes sure that there is exactly one active TaskSetManager (with different TaskSet) across all the managers (for the stage). Otherwise, submitTasks throws an IllegalStateException:
more than one active taskSet for stage [stage]: [TaskSet ids]\n
NOTE: TaskSetManager is considered active when it is not a zombie.
submitTasks requests the <> to SchedulableBuilder.md#addTaskSetManager[add the TaskSetManager to the schedulable pool].
NOTE: The TaskScheduler.md#rootPool[schedulable pool] can be a single flat linked queue (in FIFOSchedulableBuilder.md[FIFO scheduling mode]) or a hierarchy of pools of Schedulables (in FairSchedulableBuilder.md[FAIR scheduling mode]).
submitTasks <> to make sure that the requested resources (i.e. CPU and memory) are assigned to the Spark application for a <> (the very first time the Spark application is started per <> flag).
NOTE: The very first time (<> flag is false) in cluster mode only (i.e. isLocal of the TaskSchedulerImpl is false), starvationTimer is scheduled to execute after configuration-properties.md#spark.starvation.timeout[spark.starvation.timeout] to ensure that the requested resources, i.e. CPUs and memory, were assigned by a cluster manager.
NOTE: After the first configuration-properties.md#spark.starvation.timeout[spark.starvation.timeout] passes, the <> internal flag is true.
In the end, submitTasks requests the <> to scheduler:SchedulerBackend.md#reviveOffers[reviveOffers].
TIP: Use dag-scheduler-event-loop thread to step through the code in a debugger.
handleFailedTask scheduler:TaskSetManager.md#handleFailedTask[notifies taskSetManager that tid task has failed] and, only when scheduler:TaskSetManager.md#zombie-state[taskSetManager is not in zombie state] and tid is not in KILLED state, scheduler:SchedulerBackend.md#reviveOffers[requests SchedulerBackend to revive offers].
NOTE: handleFailedTask is called when scheduler:TaskResultGetter.md#enqueueSuccessfulTask[TaskResultGetter deserializes a TaskFailedReason] for a failed task.
taskSetFinished looks all scheduler:TaskSet.md[TaskSet]s up by the stage id (in <> registry) and removes the stage attempt from them, possibly with removing the entire stage record from taskSetsByStageIdAndAttempt registry completely (if there are no other attempts registered).
taskSetFinished then removes manager from the parent's schedulable pool.
You should see the following INFO message in the logs:
Removed TaskSet [id], whose tasks have all completed, from pool [name]\n
taskSetFinished is used when:
TaskSetManager is requested to maybeFinishTaskSet
","text":""},{"location":"scheduler/TaskSchedulerImpl/#notifying-dagscheduler-about-new-executor","title":"Notifying DAGScheduler About New Executor
executorAdded just DAGScheduler.md#executorAdded[notifies DAGScheduler that an executor was added].
NOTE: executorAdded uses <> that was given when <>.","text":""},{"location":"scheduler/TaskSchedulerImpl/#resourceOffers","title":"Creating TaskDescriptions For Available Executor Resource Offers
resourceOffers takes the resources offers and generates a collection of tasks (as TaskDescriptions) to launch (given the resources available).
Note
A WorkerOffer represents a resource offer with CPU cores free to use on an executor.
Internally, resourceOffers first updates <> and <> lookup tables to record new hosts and executors (given the input offers).
For new executors (not in <>) resourceOffers <DAGScheduler that an executor was added>>.
NOTE: TaskSchedulerImpl uses resourceOffers to track active executors.
CAUTION: FIXME a picture with executorAdded call from TaskSchedulerImpl to DAGScheduler.
resourceOffers requests BlacklistTracker to applyBlacklistTimeout and filters out offers on blacklisted nodes and executors.
NOTE: resourceOffers uses the optional <> that was given when <>.
CAUTION: FIXME Expand on blacklisting
resourceOffers then randomly shuffles offers (to evenly distribute tasks across executors and avoid over-utilizing some executors) and initializes the local data structures tasks and availableCpus (as shown in the figure below).
resourceOffers Pool.md#getSortedTaskSetQueue[takes TaskSets in scheduling order] from scheduler:TaskScheduler.md#rootPool[top-level Schedulable Pool].
Note
rootPool is configured when <>.
rootPool is part of the scheduler:TaskScheduler.md#rootPool[TaskScheduler Contract] and exclusively managed by scheduler:SchedulableBuilder.md[SchedulableBuilders], i.e. scheduler:FIFOSchedulableBuilder.md[FIFOSchedulableBuilder] and scheduler:FairSchedulableBuilder.md[FairSchedulableBuilder] (that scheduler:SchedulableBuilder.md#addTaskSetManager[manage registering TaskSetManagers with the root pool]).
scheduler:TaskSetManager.md[TaskSetManager] manages execution of the tasks in a single scheduler:TaskSet.md[TaskSet] that represents a single scheduler:Stage.md[Stage].
For every TaskSetManager (in scheduling order), you should see the following DEBUG message in the logs:
Only if a new executor was added, resourceOffers scheduler:TaskSetManager.md#executorAdded[notifies every TaskSetManager about the change] (to recompute locality preferences).
resourceOffers then takes every TaskSetManager (in scheduling order) and offers them each node in increasing order of locality levels (per scheduler:TaskSetManager.md#computeValidLocalityLevels[TaskSetManager's valid locality levels]).
NOTE: A TaskSetManager scheduler:TaskSetManager.md#computeValidLocalityLevels[computes locality levels of the tasks] it manages.
For every TaskSetManager and the TaskSetManager's valid locality level, resourceOffers tries to <> as long as the TaskSetManager manages to launch a task (given the locality level).
If resourceOffers did not manage to offer resources to a TaskSetManager so it could launch any task, resourceOffers scheduler:TaskSetManager.md#abortIfCompletelyBlacklisted[requests the TaskSetManager to abort the TaskSet if completely blacklisted].
When resourceOffers managed to launch a task, the internal <> flag gets enabled (that effectively means what the name says \"there were executors and I managed to launch a task\").
resourceOffers is used when:
CoarseGrainedSchedulerBackend (via DriverEndpoint RPC endpoint) is requested to make executor resource offers
LocalEndpoint is requested to revive resource offers
Unless a BarrierCoordinator has already been registered, maybeInitBarrierCoordinator creates a BarrierCoordinator and registers it to be known as barrierSync.
In the end, maybeInitBarrierCoordinator prints out the following INFO message to the logs:
Registered BarrierCoordinator endpoint\n
","text":""},{"location":"scheduler/TaskSchedulerImpl/#resourceOfferSingleTaskSet","title":"Finding Tasks from TaskSetManager to Schedule on Executors
resourceOfferSingleTaskSet takes every WorkerOffer (from the input shuffledOffers) and (only if the number of available CPU cores (using the input availableCpus) is at least configuration-properties.md#spark.task.cpus[spark.task.cpus]) scheduler:TaskSetManager.md#resourceOffer[requests TaskSetManager (as the input taskSet) to find a Task to execute (given the resource offer)] (as an executor, a host, and the input maxLocality).
resourceOfferSingleTaskSet adds the task to the input tasks collection.
resourceOfferSingleTaskSet records the task id and TaskSetManager in some registries.
resourceOfferSingleTaskSet decreases configuration-properties.md#spark.task.cpus[spark.task.cpus] from the input availableCpus (for the WorkerOffer).
resourceOfferSingleTaskSet returns whether a task was launched or not.
Note
resourceOfferSingleTaskSet asserts that the number of available CPU cores (in the input availableCpus per WorkerOffer) is at least 0.
If there is a TaskNotSerializableException, resourceOfferSingleTaskSet prints out the following ERROR in the logs:
Resource offer failed, task set [name] was not serializable\n
TaskSet is a collection of independent tasks of a stage (and a stage execution attempt) that are missing (uncomputed), i.e. for which computation results are unavailable (as RDD blocks on BlockManagers on executors).
In other words, a TaskSet represents the missing partitions of a stage that (as tasks) can be run right away based on the data that is already on the cluster, e.g. map output files from previous stages, though they may fail if this data becomes unavailable.
Since the tasks are only the missing tasks, their number does not necessarily have to be the number of all the tasks of a stage. For a brand new stage (that has never been attempted to compute) their numbers are exactly the same.
Once DAGScheduler submits the missing tasks for execution (to the TaskScheduler), the execution of the TaskSet is managed by a TaskSetManager that allows for spark.task.maxFailures.
The priority is the ID of the earliest-created active job that needs the stage (that is given when DAGScheduler is requested to submit the missing tasks of a stage).
Once submitted for execution, the priority is the priority of the TaskSetManager (which is a Schedulable) that is used for task prioritization (prioritizing scheduling of tasks) in the FIFO scheduling mode.
TaskSchedulerImpl is requested to create a TaskSetManager
While being created, TaskSetManager requests the current epoch from MapOutputTracker and sets it on all tasks in the taskset.
Note
TaskSetManager uses TaskSchedulerImpl to access the current MapOutputTracker.
TaskSetManager prints out the following DEBUG to the logs:
Epoch for [taskSet]: [epoch]\n
TaskSetManager adds the tasks as pending execution (in reverse order from the highest partition to the lowest).
"},{"location":"scheduler/TaskSetManager/#number-of-task-failures","title":"Number of Task Failures
TaskSetManager is given maxTaskFailures value that is how many times a single task can fail before the whole TaskSet is aborted.
Master URL Number of Task Failures local 1 local-with-retries maxFailureslocal-cluster spark.task.maxFailures Cluster Manager spark.task.maxFailures","text":""},{"location":"scheduler/TaskSetManager/#isBarrier","title":"isBarrier","text":"
isBarrier: Boolean\n
isBarrier is enabled (true) when this TaskSetManager is created for a TaskSet with barrier tasks.
isBarrier is used when:
TaskSchedulerImpl is requested to resourceOfferSingleTaskSet, resourceOffers
TaskSetManager is requested to resourceOffer, checkSpeculatableTasks, getLocalityWait
resourceOffer determines allowed locality level for the given TaskLocality being anything but NO_PREF.
resourceOffer dequeueTask for the given execId and host, and the allowed locality level. This may or may not give a TaskDescription.
In the end, resourceOffer returns the TaskDescription, hasScheduleDelayReject, and the index of the dequeued task (if any).
resourceOffer returns a (None, false, -1) tuple when this TaskSetManager is isZombie or the offer (by the given host or execId) should be ignored (excluded).
resourceOffer is used when:
TaskSchedulerImpl is requested to resourceOfferSingleTaskSet
recomputeLocality recomputes myLocalityLevels, localityWaits and currentLocalityIndex internal registries.
recomputeLocality computes locality levels (for scheduled tasks) and saves the result in myLocalityLevels internal registry.
recomputeLocality computes localityWaits by determining the locality wait for every locality level in myLocalityLevels.
recomputeLocality computes currentLocalityIndex by getLocalityIndex with the previous locality level. If the current locality index is higher than the previous, recomputeLocality recalculates currentLocalityIndex.
recomputeLocality is used when:
TaskSetManager is notified about status change in executors (i.e., lost, decommissioned, added)
A TaskSetManager is a zombie when all tasks in a taskset have completed successfully (regardless of the number of task attempts), or if the taskset has been aborted.
While in zombie state, a TaskSetManager can launch no new tasks and responds with no TaskDescriptions to resourceOffers.
A TaskSetManager remains in the zombie state until all tasks have finished running, i.e. to continue to track and account for the running tasks.
Enable DEBUG logging level for org.apache.spark.scheduler.TaskSchedulerImpl (or org.apache.spark.scheduler.cluster.YarnScheduler for YARN) and org.apache.spark.scheduler.TaskSetManager and execute the following two-stage job to see their low-level innerworkings.
A cluster manager is recommended since it gives more task localization choices (with YARN additionally supporting rack localization).
$ ./bin/spark-shell \\\n --master yarn \\\n --conf spark.ui.showConsoleProgress=false\n\n// Keep # partitions low to keep # messages low\n\nscala> sc.parallelize(0 to 9, 3).groupBy(_ % 3).count\nINFO YarnScheduler: Adding task set 0.0 with 3 tasks\nDEBUG TaskSetManager: Epoch for TaskSet 0.0: 0\nDEBUG TaskSetManager: Valid locality levels for TaskSet 0.0: NO_PREF, ANY\nDEBUG YarnScheduler: parentName: , name: TaskSet_0.0, runningTasks: 0\nINFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, 10.0.2.87, executor 1, partition 0, PROCESS_LOCAL, 7541 bytes)\nINFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, 10.0.2.87, executor 2, partition 1, PROCESS_LOCAL, 7541 bytes)\nDEBUG YarnScheduler: parentName: , name: TaskSet_0.0, runningTasks: 1\nINFO TaskSetManager: Starting task 2.0 in stage 0.0 (TID 2, 10.0.2.87, executor 1, partition 2, PROCESS_LOCAL, 7598 bytes)\nDEBUG YarnScheduler: parentName: , name: TaskSet_0.0, runningTasks: 1\nDEBUG TaskSetManager: No tasks for locality level NO_PREF, so moving to locality level ANY\nINFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 518 ms on 10.0.2.87 (executor 1) (1/3)\nINFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 512 ms on 10.0.2.87 (executor 2) (2/3)\nDEBUG YarnScheduler: parentName: , name: TaskSet_0.0, runningTasks: 0\nINFO TaskSetManager: Finished task 2.0 in stage 0.0 (TID 2) in 51 ms on 10.0.2.87 (executor 1) (3/3)\nINFO YarnScheduler: Removed TaskSet 0.0, whose tasks have all completed, from pool\nINFO YarnScheduler: Adding task set 1.0 with 3 tasks\nDEBUG TaskSetManager: Epoch for TaskSet 1.0: 1\nDEBUG TaskSetManager: Valid locality levels for TaskSet 1.0: NODE_LOCAL, RACK_LOCAL, ANY\nDEBUG YarnScheduler: parentName: , name: TaskSet_1.0, runningTasks: 0\nINFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 3, 10.0.2.87, executor 2, partition 0, NODE_LOCAL, 7348 bytes)\nINFO TaskSetManager: Starting task 1.0 in stage 1.0 (TID 4, 10.0.2.87, executor 1, partition 1, NODE_LOCAL, 7348 bytes)\nDEBUG YarnScheduler: parentName: , name: TaskSet_1.0, runningTasks: 1\nINFO TaskSetManager: Starting task 2.0 in stage 1.0 (TID 5, 10.0.2.87, executor 1, partition 2, NODE_LOCAL, 7348 bytes)\nINFO TaskSetManager: Finished task 1.0 in stage 1.0 (TID 4) in 130 ms on 10.0.2.87 (executor 1) (1/3)\nDEBUG YarnScheduler: parentName: , name: TaskSet_1.0, runningTasks: 1\nDEBUG TaskSetManager: No tasks for locality level NODE_LOCAL, so moving to locality level RACK_LOCAL\nDEBUG TaskSetManager: No tasks for locality level RACK_LOCAL, so moving to locality level ANY\nINFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 3) in 133 ms on 10.0.2.87 (executor 2) (2/3)\nDEBUG YarnScheduler: parentName: , name: TaskSet_1.0, runningTasks: 0\nINFO TaskSetManager: Finished task 2.0 in stage 1.0 (TID 5) in 21 ms on 10.0.2.87 (executor 1) (3/3)\nINFO YarnScheduler: Removed TaskSet 1.0, whose tasks have all completed, from pool\nres0: Long = 3\n
supportsRelocationOfSerializedObjects\u00a0is part of the Serializer abstraction.
supportsRelocationOfSerializedObjects creates a new SerializerInstance (that is assumed to be a KryoSerializerInstance) and requests it to get the value of the autoReset field.
Serializer is an abstraction of serializers for serialization and deserialization of tasks (closures) and data blocks in a Spark application.
"},{"location":"serializer/Serializer/#contract","title":"Contract","text":""},{"location":"serializer/Serializer/#creating-new-serializerinstance","title":"Creating New SerializerInstance
newInstance(): SerializerInstance\n
Creates a new SerializerInstance
Used when:
Task is created (only used in tests)
SerializerSupport (Spark SQL) utility is used to newSerializer
RangePartitioner is requested to writeObject and readObject
TorrentBroadcast utility is used to blockifyObject and unBlockifyObject
TaskRunner is requested to run
NettyBlockRpcServer is requested to deserializeMetadata
NettyBlockTransferService is requested to uploadBlock
PairRDDFunctions is requested to...FIXME
ParallelCollectionPartition is requested to...FIXME
RDD is requested to...FIXME
ReliableCheckpointRDD utility is used to...FIXME
NettyRpcEnvFactory is requested to create a RpcEnv
getSerializer returns the KryoSerializer when the given ClassTags are Kryo-compatible and the autoPick flag is true. Otherwise, getSerializer returns the default Serializer.
autoPick flag is true for all BlockIds but Spark Streaming's StreamBlockIds.
getSerializer (with autoPick flag) is used when:
SerializerManager is requested to dataSerializeStream, dataSerializeWithExplicitClassTag and dataDeserializeStream
SerializedValuesHolder (of MemoryStore) is requested for a SerializationStream
getSerializer (with key and value ClassTags only) is used when:
BaseShuffleHandle is a ShuffleHandle that is used to capture the parameters when SortShuffleManager is requested for a ShuffleHandle (and the other specialized ShuffleHandles could not be selected):
// Start a Spark application, e.g. spark-shell, with the Spark properties to trigger selection of BaseShuffleHandle:\n// 1. spark.shuffle.spill.numElementsForceSpillThreshold=1\n// 2. spark.shuffle.sort.bypassMergeThreshold=1\n\n// numSlices > spark.shuffle.sort.bypassMergeThreshold\nscala> val rdd = sc.parallelize(0 to 4, numSlices = 2).groupBy(_ % 2)\nrdd: org.apache.spark.rdd.RDD[(Int, Iterable[Int])] = ShuffledRDD[2] at groupBy at <console>:24\n\nscala> rdd.dependencies\nDEBUG SortShuffleManager: Can't use serialized shuffle for shuffle 0 because an aggregator is defined\nres0: Seq[org.apache.spark.Dependency[_]] = List(org.apache.spark.ShuffleDependency@1160c54b)\n\nscala> rdd.getNumPartitions\nres1: Int = 2\n\nscala> import org.apache.spark.ShuffleDependency\nimport org.apache.spark.ShuffleDependency\n\nscala> val shuffleDep = rdd.dependencies(0).asInstanceOf[ShuffleDependency[Int, Int, Int]]\nshuffleDep: org.apache.spark.ShuffleDependency[Int,Int,Int] = org.apache.spark.ShuffleDependency@1160c54b\n\n// mapSideCombine is disabled\nscala> shuffleDep.mapSideCombine\nres2: Boolean = false\n\n// aggregator defined\nscala> shuffleDep.aggregator\nres3: Option[org.apache.spark.Aggregator[Int,Int,Int]] = Some(Aggregator(<function1>,<function2>,<function2>))\n\n// the number of reduce partitions < spark.shuffle.sort.bypassMergeThreshold\nscala> shuffleDep.partitioner.numPartitions\nres4: Int = 2\n\nscala> shuffleDep.shuffleHandle\nres5: org.apache.spark.shuffle.ShuffleHandle = org.apache.spark.shuffle.BaseShuffleHandle@22b0fe7e\n
","text":""},{"location":"shuffle/BlockStoreShuffleReader/#review-me","title":"Review Me
=== [[read]] Reading Combined Records For Reduce Task
Internally, read first storage:ShuffleBlockFetcherIterator.md#creating-instance[creates a ShuffleBlockFetcherIterator] (passing in the values of <>, <> and <> Spark properties).
NOTE: read uses scheduler:MapOutputTracker.md#getMapSizesByExecutorId[MapOutputTracker to find the BlockManagers with the shuffle blocks and sizes] to create ShuffleBlockFetcherIterator.
read creates a new serializer:SerializerInstance.md[SerializerInstance] (using Serializer from ShuffleDependency).
read creates a key/value iterator by deserializeStream every shuffle block stream.
read updates the context task metrics for each record read.
NOTE: read uses CompletionIterator (to count the records read) and spark-InterruptibleIterator.md[InterruptibleIterator] (to support task cancellation).
If the ShuffleDependency has an Aggregator defined, read wraps the current iterator inside an iterator defined by Aggregator.combineCombinersByKey (for mapSideCombine enabled) or Aggregator.combineValuesByKey otherwise.
NOTE: run reports an exception when ShuffleDependency has no Aggregator defined with mapSideCombine flag enabled.
For keyOrdering defined in the ShuffleDependency, run does the following:
shuffle:ExternalSorter.md#creating-instance[Creates an ExternalSorter]
shuffle:ExternalSorter.md#insertAll[Inserts all the records] into the ExternalSorter
Updates context TaskMetrics
Returns a CompletionIterator for the ExternalSorter
BypassMergeSortShuffleHandle is a BaseShuffleHandle that SortShuffleManager uses when can avoid merge-sorting data (when requested to register a shuffle).
SerializedShuffleHandle tells SortShuffleManager to use BypassMergeSortShuffleWriter when requested for a ShuffleWriter.