├── .gitattributes ├── .github ├── FUNDING.yml └── workflows │ └── ci.yml ├── .gitignore ├── LICENSE ├── README.md ├── docs ├── BytesToBytesMap.md ├── ConsoleProgressBar.md ├── DriverLogger.md ├── ExecutorDeadException.md ├── FileCommitProtocol.md ├── HadoopMapRedCommitProtocol.md ├── HadoopMapRedWriteConfigUtil.md ├── HadoopMapReduceCommitProtocol.md ├── HadoopMapReduceWriteConfigUtil.md ├── HadoopWriteConfigUtil.md ├── HeartbeatReceiver.md ├── InterruptibleIterator.md ├── ListenerBus.md ├── OutputCommitCoordinator.md ├── PartitionEvaluator.md ├── PartitionEvaluatorFactory.md ├── SparkConf.md ├── SparkContext-creating-instance-internals.md ├── SparkContext.md ├── SparkCoreErrors.md ├── SparkEnv.md ├── SparkFiles.md ├── SparkHadoopWriter.md ├── SparkListener.md ├── SparkListenerBus.md ├── SparkListenerEvent.md ├── SparkListenerInterface.md ├── SparkListenerTaskEnd.md ├── SparkStatusTracker.md ├── SpillListener.md ├── StatsReportListener.md ├── TaskCompletionListener.md ├── TaskFailureListener.md ├── Utils.md ├── accumulators │ ├── AccumulableInfo.md │ ├── AccumulatorContext.md │ ├── AccumulatorSource.md │ ├── AccumulatorV2.md │ ├── InternalAccumulator.md │ └── index.md ├── architecture.md ├── barrier-execution-mode │ ├── .pages │ ├── BarrierCoordinator.md │ ├── BarrierCoordinatorMessage.md │ ├── BarrierJobAllocationFailed.md │ ├── BarrierJobSlotsNumberCheckFailed.md │ ├── BarrierTaskContext.md │ ├── ContextBarrierState.md │ ├── RDDBarrier.md │ ├── RequestMethod.md │ ├── RequestToSync.md │ └── index.md ├── broadcast-variables │ ├── Broadcast.md │ ├── BroadcastFactory.md │ ├── BroadcastManager.md │ ├── TorrentBroadcast.md │ ├── TorrentBroadcastFactory.md │ └── index.md ├── configuration-properties.md ├── core │ ├── BlockFetchStarter.md │ ├── BlockFetchingListener.md │ ├── CleanerListener.md │ ├── ContextCleaner.md │ ├── InMemoryStore.md │ ├── KVStore.md │ ├── LevelDB.md │ └── RetryingBlockFetcher.md ├── demo │ ├── diskblockmanager-and-block-data.md │ └── index.md ├── developer-api.md ├── driver.md ├── dynamic-allocation │ ├── ExecutorAllocationClient.md │ ├── ExecutorAllocationListener.md │ ├── ExecutorAllocationManager.md │ ├── ExecutorAllocationManagerSource.md │ ├── ExecutorMonitor.md │ ├── Tracker.md │ ├── configuration-properties.md │ └── index.md ├── executor │ ├── CoarseGrainedExecutorBackend.md │ ├── Executor.md │ ├── ExecutorBackend.md │ ├── ExecutorLogUrlHandler.md │ ├── ExecutorMetricType.md │ ├── ExecutorMetrics.md │ ├── ExecutorMetricsPoller.md │ ├── ExecutorMetricsSource.md │ ├── ExecutorSource.md │ ├── ShuffleReadMetrics.md │ ├── ShuffleWriteMetrics.md │ ├── TaskMetrics.md │ ├── TaskRunner.md │ └── index.md ├── exercises │ ├── spark-examples-wordcount-spark-shell.md │ ├── spark-exercise-custom-scheduler-listener.md │ ├── spark-exercise-dataframe-jdbc-postgresql.md │ ├── spark-exercise-failing-stage.md │ ├── spark-exercise-pairrddfunctions-oneliners.md │ ├── spark-exercise-standalone-master-ha.md │ ├── spark-exercise-take-multiple-jobs.md │ ├── spark-first-app.md │ ├── spark-hello-world-using-spark-shell.md │ └── spark-sql-hive-orc-example.md ├── external-shuffle-service │ ├── ExecutorShuffleInfo.md │ ├── ExternalBlockHandler.md │ ├── ExternalShuffleBlockResolver.md │ ├── ExternalShuffleService.md │ ├── configuration-properties.md │ └── index.md ├── features │ └── index.md ├── history-server │ ├── ApplicationCache.md │ ├── ApplicationCacheOperations.md │ ├── ApplicationHistoryProvider.md │ ├── EventLogFileWriter.md │ ├── EventLoggingListener.md │ ├── FsHistoryProvider.md │ ├── HistoryAppStatusStore.md │ ├── HistoryServer.md │ ├── HistoryServerArguments.md │ ├── HistoryServerDiskManager.md │ ├── JsonProtocol.md │ ├── ReplayListenerBus.md │ ├── SQLHistoryListener.md │ ├── configuration-properties.md │ └── index.md ├── images │ ├── CoarseGrainedExecutorBackend-reviveOffers.png │ ├── CoarseGrainedScheduler-rpc-endpoint.png │ ├── CoarseGrainedSchedulerBackend-DriverEndpoint-CoarseGrainedExecutorBackend.png │ ├── CoarseGrainedSchedulerBackend-RegisterExecutor-event.png │ ├── LocalSchedulerBackend-LocalEndpoint-Executor-task-status-updates.png │ ├── ShuffleMapTask-runTask.png │ ├── SparkDeploySchedulerBackend-AppClient-start.png │ ├── TaskSetManager-TaskSchedulerImpl-TaskSet.png │ ├── TaskSetManager-handleFailedTask.png │ ├── core │ │ ├── AppStatusStore-createLiveStore.png │ │ ├── BroadcastManager.png │ │ ├── ContextCleaner.png │ │ ├── sparkenv-driver-blockmanager.png │ │ └── sparkenv-executor-blockmanager.png │ ├── diagrams │ │ ├── rdd-saveas.graphml │ │ ├── rdd-saveas.png │ │ ├── rdd-sparkcontext-transformations-action.graphml │ │ ├── rdd-sparkcontext-transformations-action.png │ │ ├── spark-local-architecture.graphml │ │ ├── spark-local-architecture.png │ │ ├── spark-platform.graphml │ │ ├── spark-platform.png │ │ ├── spark-rdd-partitioned-distributed.graphml │ │ ├── spark-rdds.graphml │ │ ├── spark-submit-master-workers.graphml │ │ ├── spark-submit-master-workers.png │ │ ├── sparkcontext-createtaskscheduler.graphml │ │ ├── sparkcontext-createtaskscheduler.png │ │ ├── sparkcontext-rdds.graphml │ │ ├── sparkcontext-rdds.png │ │ ├── sparkcontext-services.graphml │ │ ├── sparkcontext-services.png │ │ ├── taskscheduler-single-sparkcontext.graphml │ │ └── taskscheduler-single-sparkcontext.png │ ├── driver-sparkcontext-clustermanager-workers-executors.png │ ├── executor │ │ ├── CoarseGrainedExecutorBackend-statusUpdate.png │ │ ├── CoarseGrainedExecutorBackend.png │ │ ├── ExecutorBackend.png │ │ ├── HeartbeatReceiver-Heartbeat.png │ │ ├── TaskRunner.png │ │ ├── executor-heartbeatReceiver-endpoint.png │ │ ├── executor-taskrunner-executorbackend.png │ │ ├── spark-executorsource-jconsole.png │ │ └── spark-standalone-webui-memory-per-node.png │ ├── history-server │ │ └── spark-history-server-webui.png │ ├── memory │ │ ├── MemoryManager.png │ │ └── TaskMemoryManager.png │ ├── mesos-console-slaves.png │ ├── mesos-console.png │ ├── rdd-lineage.png │ ├── rpc │ │ └── rpcenv-endpoints.png │ ├── scheduler │ │ ├── DAGScheduler-MapOutputTrackerMaster-containsShuffle.png │ │ ├── ShuffleMapTask.png │ │ ├── TaskSchedulerImpl-initialize.png │ │ ├── TaskSchedulerImpl-resourceOffers-internal-structures.png │ │ ├── TaskSchedulerImpl-resourceOffers-rootPool-getSortedTaskSetQueue.png │ │ ├── action-job.png │ │ ├── dagscheduler-handleExecutorLost.png │ │ ├── dagscheduler-handleJobSubmitted.png │ │ ├── dagscheduler-job-resultstage.png │ │ ├── dagscheduler-new-instance.png │ │ ├── dagscheduler-rdd-lineage-stage-dag.png │ │ ├── dagscheduler-rdd-partitions-job-resultstage.png │ │ ├── dagscheduler-resultstage-partitions.png │ │ ├── dagscheduler-stages.png │ │ ├── dagscheduler-submitjob.png │ │ ├── dagscheduler-tasksetmanager.png │ │ ├── dagscheduler-webui-skipped-stages.png │ │ ├── job-stage.png │ │ ├── rdd-job-partitions.png │ │ ├── resultstage-findMissingPartitions.png │ │ ├── scheduler-handlemapstagesubmitted.png │ │ ├── scheduler-job-shuffles-result-stages.png │ │ ├── scheduler-job-splits-into-stages.png │ │ ├── spark-DAGScheduler-getShuffleDependencies.png │ │ ├── spark-rdd-partitions-job-stage-tasks.png │ │ ├── spark-sparklistener-event-senders.png │ │ ├── sparkstandalone-sparkcontext-taskscheduler-schedulerbackend.png │ │ ├── stage-tasks.png │ │ ├── taskscheduler-resourceOffers.png │ │ ├── taskscheduler-uses-schedulerbackend.png │ │ ├── taskschedulerImpl-submitTasks.png │ │ ├── taskschedulerimpl-sparkcontext-schedulerbackend-dagscheduler.png │ │ ├── taskschedulerimpl-start-standalone.png │ │ └── taskschedulerimpl-tasksetmanager-tasksetfinished.png │ ├── shuffle │ │ ├── BypassMergeSortShuffleWriter-write.png │ │ ├── IndexShuffleBlockResolver-SortShuffleManager.png │ │ ├── IndexShuffleBlockResolver-writeIndexFileAndCommit.png │ │ ├── ShuffleExternalSorter.png │ │ ├── ShuffleInMemorySorter.png │ │ ├── SortShuffleManager.png │ │ ├── UnsafeShuffleWriter-ShuffleExternalSorter.png │ │ └── UnsafeShuffleWriter.png │ ├── spark-SQLTab-creating-instance.png │ ├── spark-YarnSchedulerBackend-doRequestTotalExecutors.png │ ├── spark-YarnSchedulerEndpoint-RequestExecutors.png │ ├── spark-broadcast-webui-executors-rdd-blocks.png │ ├── spark-driver.png │ ├── spark-mesos-completed-tasks.png │ ├── spark-mesos-frameworks-one-active.png │ ├── spark-mesos.png │ ├── spark-metrics-MetricsSystem-driver.png │ ├── spark-metrics-MetricsSystem.png │ ├── spark-metrics-jconsole.png │ ├── spark-mllib-pipeline.png │ ├── spark-partitions-ui-stages-2-partitions.png │ ├── spark-partitions-ui-stages.png │ ├── spark-platform.png │ ├── spark-pool-FairSchedulingAlgorithm.png │ ├── spark-pull-request-tasks-progress.png │ ├── spark-rdd-partitioned-distributed.png │ ├── spark-rdds.png │ ├── spark-runjob.png │ ├── spark-shuffle-join-webui.png │ ├── spark-standalone-console-master-only.png │ ├── spark-standalone-console-one-worker.png │ ├── spark-standalone-console-two-workers-alive.png │ ├── spark-standalone-console-worker-dead.png │ ├── spark-standalone-console-workers-alive-and-dead.png │ ├── spark-standalone-master-worker-LaunchDriver.png │ ├── spark-standalone-webui-appid-notfound.png │ ├── spark-standalone-webui-executor-summary.png │ ├── spark-thriftserver-squirrel-addalias.png │ ├── spark-thriftserver-squirrel-adddriver.png │ ├── spark-thriftserver-squirrel-metadata.png │ ├── spark-thriftserver-squirrel-show-tables.png │ ├── spark-thriftserver-squirrel.png │ ├── spark-thriftserver-webui.png │ ├── spark-transformations-zipWithIndex-webui.png │ ├── spark-ui-rdd-name.png │ ├── spark-ui-repartition-2.png │ ├── spark-ui-storage-hundred-ints.png │ ├── sparkapp-sparkcontext-master-slaves.png │ ├── sparkcontext-broadcast-bittorrent-newBroadcast.png │ ├── sparkcontext-broadcast-bittorrent.png │ ├── sparkcontext-broadcast-executors.png │ ├── sparkcontext-broadcastmanager-contextcleaner.png │ ├── sparkenv-driver.png │ ├── sparkenv-executor.png │ ├── sparkui-stages-locality-level.png │ ├── sparkui.png │ ├── storage │ │ ├── BlockInfoManager-BlockManager.png │ │ ├── BlockManager-SparkEnv.png │ │ ├── BlockManager.png │ │ ├── BlockManagerMaster-RegisterBlockManager.png │ │ ├── DiskBlockManager-BlockManager.png │ │ ├── DiskStore-BlockManager.png │ │ ├── MemoryStore-BlockManager.png │ │ ├── NettyBlockRpcServer.png │ │ ├── NettyBlockTransferService.png │ │ ├── ShuffleMetricsSource.png │ │ ├── demo-DiskBlockManager-and-Block-Data-webui-storage.png │ │ └── spark-MemoryStore.png │ ├── taskscheduler-submitTasks-local-mode.png │ ├── webui │ │ ├── spark-webui-SparkUI.png │ │ ├── spark-webui-accumulators.png │ │ ├── spark-webui-active-stages.png │ │ ├── spark-webui-environment.png │ │ ├── spark-webui-executors.png │ │ ├── spark-webui-jobs-details-for-job-active-pending-stages.png │ │ ├── spark-webui-jobs-details-for-job-four-stages.png │ │ ├── spark-webui-jobs-details-for-job-no-job.png │ │ ├── spark-webui-jobs-details-for-job.png │ │ ├── spark-webui-jobs-event-timeline.png │ │ ├── spark-webui-jobs-status-section.png │ │ ├── spark-webui-jobs-summary-section.png │ │ ├── spark-webui-jobs-timeline-executors.png │ │ ├── spark-webui-jobs-timeline-popup.png │ │ ├── spark-webui-jobs.png │ │ ├── spark-webui-pool-details.png │ │ ├── spark-webui-pool-summary.png │ │ ├── spark-webui-sql-execution-graph.png │ │ ├── spark-webui-sql-no-details-for-query.png │ │ ├── spark-webui-sql.png │ │ ├── spark-webui-stage-accumulators.png │ │ ├── spark-webui-stage-aggregated-metrics-by-executor.png │ │ ├── spark-webui-stage-dagvisualization.png │ │ ├── spark-webui-stage-details.png │ │ ├── spark-webui-stage-eventtimeline.png │ │ ├── spark-webui-stage-header.png │ │ ├── spark-webui-stage-summary-metrics-tasks.png │ │ ├── spark-webui-stage-tasks.png │ │ ├── spark-webui-stages-alljobs.png │ │ ├── spark-webui-stages-completed.png │ │ ├── spark-webui-stages-empty.png │ │ ├── spark-webui-stages-fairschedulerpools.png │ │ ├── spark-webui-stages.png │ │ └── spark-webui-storage.png │ └── yarn-YarnSchedulerBackend.png ├── index.md ├── local-properties.md ├── local │ ├── LauncherBackend.md │ ├── LocalEndpoint.md │ ├── LocalSchedulerBackend.md │ └── index.md ├── master.md ├── memory │ ├── ExecutionMemoryPool.md │ ├── MemoryAllocator.md │ ├── MemoryConsumer.md │ ├── MemoryManager.md │ ├── MemoryPool.md │ ├── StorageMemoryPool.md │ ├── TaskMemoryManager.md │ ├── UnifiedMemoryManager.md │ ├── UnsafeExternalSorter.md │ ├── UnsafeInMemorySorter.md │ ├── UnsafeSorterSpillReader.md │ ├── UnsafeSorterSpillWriter.md │ └── index.md ├── metrics │ ├── JvmSource.md │ ├── MetricsConfig.md │ ├── MetricsServlet.md │ ├── MetricsSystem.md │ ├── PrometheusServlet.md │ ├── Sink.md │ ├── Source.md │ ├── configuration-properties.md │ └── index.md ├── network │ ├── SparkTransportConf.md │ ├── TransportClientFactory.md │ ├── TransportConf.md │ ├── TransportContext.md │ └── index.md ├── overview.md ├── plugins │ ├── DriverPlugin.md │ ├── DriverPluginContainer.md │ ├── ExecutorPlugin.md │ ├── ExecutorPluginContainer.md │ ├── PluginContainer.md │ ├── PluginContextImpl.md │ ├── SparkPlugin.md │ └── index.md ├── push-based-shuffle.md ├── rdd │ ├── Aggregator.md │ ├── AsyncRDDActions.md │ ├── CheckpointRDD.md │ ├── CoGroupedRDD.md │ ├── CoalescedRDD.md │ ├── Dependency.md │ ├── HadoopRDD.md │ ├── HashPartitioner.md │ ├── LocalCheckpointRDD.md │ ├── LocalRDDCheckpointData.md │ ├── MapPartitionsRDD.md │ ├── MapPartitionsWithEvaluatorRDD.md │ ├── NarrowDependency.md │ ├── NewHadoopRDD.md │ ├── OrderedRDDFunctions.md │ ├── PairRDDFunctions.md │ ├── ParallelCollectionRDD.md │ ├── Partition.md │ ├── Partitioner.md │ ├── RDD.md │ ├── RDDCheckpointData.md │ ├── RDDOperationScope.md │ ├── RangePartitioner.md │ ├── ReliableCheckpointRDD.md │ ├── ReliableRDDCheckpointData.md │ ├── ShuffleDependency.md │ ├── ShuffledRDD.md │ ├── ZippedPartitionsWithEvaluatorRDD.md │ ├── checkpointing.md │ ├── index.md │ ├── lineage.md │ ├── spark-rdd-actions.md │ ├── spark-rdd-caching.md │ ├── spark-rdd-operations.md │ ├── spark-rdd-partitions.md │ └── spark-rdd-transformations.md ├── rest │ ├── AbstractApplicationResource.md │ ├── ApiRequestContext.md │ ├── ApiRootResource.md │ ├── ApplicationListResource.md │ ├── BaseAppResource.md │ ├── OneApplicationAttemptResource.md │ ├── OneApplicationResource.md │ ├── StagesResource.md │ ├── UIRoot.md │ ├── UIRootFromServletContext.md │ └── index.md ├── rpc │ ├── NettyRpcEnv.md │ ├── NettyRpcEnvFactory.md │ ├── RpcAddress.md │ ├── RpcEndpoint.md │ ├── RpcEndpointAddress.md │ ├── RpcEndpointRef.md │ ├── RpcEnv.md │ ├── RpcEnvConfig.md │ ├── RpcEnvFactory.md │ ├── RpcEnvFileServer.md │ ├── RpcUtils.md │ ├── index.md │ └── spark-rpc-netty.md ├── scheduler │ ├── ActiveJob.md │ ├── BlacklistTracker.md │ ├── CoarseGrainedSchedulerBackend.md │ ├── CompressedMapStatus.md │ ├── DAGScheduler.md │ ├── DAGSchedulerEvent.md │ ├── DAGSchedulerEventProcessLoop.md │ ├── DAGSchedulerSource.md │ ├── DriverEndpoint.md │ ├── ExecutorData.md │ ├── ExternalClusterManager.md │ ├── FIFOSchedulableBuilder.md │ ├── FairSchedulableBuilder.md │ ├── HighlyCompressedMapStatus.md │ ├── JobListener.md │ ├── JobWaiter.md │ ├── LiveListenerBus.md │ ├── MapOutputStatistics.md │ ├── MapOutputTracker.md │ ├── MapOutputTrackerMaster.md │ ├── MapOutputTrackerMasterEndpoint.md │ ├── MapOutputTrackerWorker.md │ ├── MapStatus.md │ ├── Pool.md │ ├── ResultStage.md │ ├── ResultTask.md │ ├── Schedulable.md │ ├── SchedulableBuilder.md │ ├── SchedulerBackend.md │ ├── SchedulerBackendUtils.md │ ├── SchedulingMode.md │ ├── ShuffleMapStage.md │ ├── ShuffleMapTask.md │ ├── ShuffleStatus.md │ ├── Stage.md │ ├── StageInfo.md │ ├── Task.md │ ├── TaskContext.md │ ├── TaskContextImpl.md │ ├── TaskDescription.md │ ├── TaskInfo.md │ ├── TaskLocation.md │ ├── TaskResult.md │ ├── TaskResultGetter.md │ ├── TaskScheduler.md │ ├── TaskSchedulerImpl.md │ ├── TaskSet.md │ ├── TaskSetBlacklist.md │ ├── TaskSetManager.md │ └── index.md ├── serializer │ ├── DeserializationStream.md │ ├── JavaSerializerInstance.md │ ├── KryoSerializer.md │ ├── KryoSerializerInstance.md │ ├── SerializationStream.md │ ├── Serializer.md │ ├── SerializerInstance.md │ ├── SerializerManager.md │ └── index.md ├── shuffle │ ├── BaseShuffleHandle.md │ ├── BlockStoreShuffleReader.md │ ├── BypassMergeSortShuffleHandle.md │ ├── BypassMergeSortShuffleWriter.md │ ├── DownloadFileManager.md │ ├── ExecutorDiskUtils.md │ ├── ExternalAppendOnlyMap.md │ ├── ExternalSorter.md │ ├── FetchFailedException.md │ ├── IndexShuffleBlockResolver.md │ ├── LocalDiskShuffleDataIO.md │ ├── LocalDiskShuffleExecutorComponents.md │ ├── LocalDiskShuffleMapOutputWriter.md │ ├── LocalDiskSingleSpillMapOutputWriter.md │ ├── MigratableResolver.md │ ├── SerializedShuffleHandle.md │ ├── ShuffleBlockPusher.md │ ├── ShuffleBlockResolver.md │ ├── ShuffleDataIO.md │ ├── ShuffleDataIOUtils.md │ ├── ShuffleDriverComponents.md │ ├── ShuffleExecutorComponents.md │ ├── ShuffleExternalSorter.md │ ├── ShuffleHandle.md │ ├── ShuffleInMemorySorter.md │ ├── ShuffleManager.md │ ├── ShuffleMapOutputWriter.md │ ├── ShuffleReader.md │ ├── ShuffleWriteMetricsReporter.md │ ├── ShuffleWriteProcessor.md │ ├── ShuffleWriter.md │ ├── SingleSpillShuffleMapOutputWriter.md │ ├── SortShuffleManager.md │ ├── SortShuffleWriter.md │ ├── Spillable.md │ ├── UnsafeShuffleWriter.md │ └── index.md ├── spark-debugging.md ├── spark-logging.md ├── spark-properties.md ├── speculative-execution-of-tasks.md ├── stage-level-scheduling │ ├── .pages │ ├── ExecutorResourceInfo.md │ ├── ExecutorResourceRequest.md │ ├── ExecutorResourceRequests.md │ ├── ResourceAllocator.md │ ├── ResourceID.md │ ├── ResourceProfile.md │ ├── ResourceProfileBuilder.md │ ├── ResourceProfileManager.md │ ├── ResourceUtils.md │ ├── SparkListenerResourceProfileAdded.md │ ├── TaskResourceProfile.md │ ├── TaskResourceRequest.md │ ├── TaskResourceRequests.md │ └── index.md ├── status │ ├── AppStatusListener.md │ ├── AppStatusSource.md │ ├── AppStatusStore.md │ ├── ElementTrackingStore.md │ ├── LiveEntity.md │ └── index.md ├── storage │ ├── BlockData.md │ ├── BlockDataManager.md │ ├── BlockEvictionHandler.md │ ├── BlockId.md │ ├── BlockInfo.md │ ├── BlockInfoManager.md │ ├── BlockManager.md │ ├── BlockManagerDecommissioner.md │ ├── BlockManagerId.md │ ├── BlockManagerInfo.md │ ├── BlockManagerMaster.md │ ├── BlockManagerMasterEndpoint.md │ ├── BlockManagerMasterHeartbeatEndpoint.md │ ├── BlockManagerSlaveEndpoint.md │ ├── BlockManagerSource.md │ ├── BlockManagerStorageEndpoint.md │ ├── BlockReplicationPolicy.md │ ├── BlockStoreClient.md │ ├── BlockStoreUpdater.md │ ├── BlockTransferService.md │ ├── ByteBufferBlockStoreUpdater.md │ ├── DiskBlockManager.md │ ├── DiskBlockObjectWriter.md │ ├── DiskStore.md │ ├── ExternalBlockStoreClient.md │ ├── FallbackStorage.md │ ├── MemoryStore.md │ ├── NettyBlockRpcServer.md │ ├── NettyBlockTransferService.md │ ├── OneForOneBlockFetcher.md │ ├── RDDInfo.md │ ├── RandomBlockReplicationPolicy.md │ ├── ShuffleBlockFetcherIterator.md │ ├── ShuffleFetchCompletionListener.md │ ├── ShuffleMetricsSource.md │ ├── ShuffleMigrationRunnable.md │ ├── StorageLevel.md │ ├── StorageStatus.md │ ├── StorageUtils.md │ ├── TempFileBasedBlockStoreUpdater.md │ └── index.md ├── tips-and-tricks │ ├── .pages │ ├── access-private-members-spark-shell.md │ ├── index.md │ ├── running-spark-windows.md │ └── sparkexception-task-not-serializable.md ├── tools │ ├── AbstractCommandBuilder.md │ ├── AbstractLauncher.md │ ├── DependencyUtils.md │ ├── JavaMainApplication.md │ ├── Main.md │ ├── SparkApplication.md │ ├── SparkClassCommandBuilder.md │ ├── SparkLauncher.md │ ├── index.md │ ├── pyspark.md │ ├── spark-class.md │ ├── spark-shell.md │ └── spark-submit │ │ ├── .pages │ │ ├── SparkSubmit.md │ │ ├── SparkSubmitArguments.md │ │ ├── SparkSubmitCommandBuilder.OptionParser.md │ │ ├── SparkSubmitCommandBuilder.md │ │ ├── SparkSubmitOperation.md │ │ ├── SparkSubmitOptionParser.md │ │ ├── SparkSubmitUtils.md │ │ └── index.md ├── webui │ ├── AllJobsPage.md │ ├── AllStagesPage.md │ ├── EnvironmentPage.md │ ├── EnvironmentTab.md │ ├── ExecutorThreadDumpPage.md │ ├── ExecutorsPage.md │ ├── ExecutorsTab.md │ ├── JettyUtils.md │ ├── JobPage.md │ ├── JobsTab.md │ ├── PoolPage.md │ ├── PrometheusResource.md │ ├── RDDPage.md │ ├── SparkUI.md │ ├── SparkUITab.md │ ├── StagePage.md │ ├── StagesTab.md │ ├── StoragePage.md │ ├── StorageTab.md │ ├── UIUtils.md │ ├── WebUI.md │ ├── WebUIPage.md │ ├── WebUITab.md │ ├── configuration-properties.md │ └── index.md └── workers.md ├── graffles ├── CoarseGrainedExecutorBackend-reviveOffers.graffle ├── CoarseGrainedScheduler-rpc-endpoint.graffle ├── CoarseGrainedSchedulerBackend-DriverEndpoint-CoarseGrainedExecutorBackend.graffle ├── CoarseGrainedSchedulerBackend-RegisterExecutor-event.graffle ├── LocalSchedulerBackend-LocalEndpoint-Executor-task-status-updates.graffle ├── SparkDeploySchedulerBackend-AppClient-start.graffle ├── TaskSetManager-TaskSchedulerImpl-TaskSet.graffle ├── TaskSetManager-handleFailedTask.graffle ├── core │ ├── AppStatusStore-createLiveStore.graffle │ ├── BroadcastManager.graffle │ ├── ContextCleaner.graffle │ ├── sparkcontext-broadcast-bittorrent-newBroadcast.graffle │ ├── sparkcontext-broadcast-bittorrent.graffle │ ├── sparkenv-driver-blockmanager.graffle │ ├── sparkenv-driver.graffle │ ├── sparkenv-executor-blockmanager.graffle │ └── sparkenv-executor.graffle ├── driver-sparkcontext-clustermanager-workers-executors.graffle ├── executor │ ├── CoarseGrainedExecutorBackend-statusUpdate.graffle │ ├── CoarseGrainedExecutorBackend.graffle │ ├── ExecutorBackend.graffle │ ├── HeartbeatReceiver-Heartbeat.graffle │ ├── TaskRunner.graffle │ ├── executor-heartbeatReceiver-endpoint.graffle │ └── executor-taskrunner-executorbackend.graffle ├── memory │ ├── MemoryManager.graffle │ └── TaskMemoryManager.graffle ├── rdd-lineage.graffle ├── scheduler │ ├── DAGScheduler-MapOutputTrackerMaster-containsShuffle.graffle │ ├── ShuffleMapTask-runTask.graffle │ ├── ShuffleMapTask.graffle │ ├── TaskSchedulerImpl-initialize.graffle │ ├── TaskSchedulerImpl-resourceOffers-internal-structures.graffle │ ├── TaskSchedulerImpl-resourceOffers-rootPool-getSortedTaskSetQueue.graffle │ ├── dagscheduler-handleExecutorLost.graffle │ ├── dagscheduler-handleJobSubmitted.graffle │ ├── dagscheduler-new-instance.graffle │ ├── dagscheduler-rdd-lineage-stage-dag.graffle │ ├── dagscheduler-rdd-partitions-job-resultstage.graffle │ ├── dagscheduler-resultstage-partitions.graffle │ ├── dagscheduler-stages.graffle │ ├── dagscheduler-submitjob.graffle │ ├── dagscheduler-tasksetmanager.graffle │ ├── resultstage-findMissingPartitions.graffle │ ├── spark-DAGScheduler-getShuffleDependencies.graffle │ ├── spark-rdd-partitions-job-stage-tasks.graffle │ ├── spark-sparklistener-event-senders.graffle │ ├── sparkstandalone-sparkcontext-taskscheduler-schedulerbackend.graffle │ ├── taskscheduler-resourceOffers.graffle │ ├── taskschedulerImpl-submitTasks.graffle │ ├── taskschedulerimpl-sparkcontext-schedulerbackend-dagscheduler.graffle │ ├── taskschedulerimpl-start-standalone.graffle │ └── taskschedulerimpl-tasksetmanager-tasksetfinished.graffle ├── shuffle │ ├── BypassMergeSortShuffleWriter-write.graffle │ ├── IndexShuffleBlockResolver-SortShuffleManager.graffle │ ├── IndexShuffleBlockResolver-writeIndexFileAndCommit.graffle │ ├── ShuffleExternalSorter.graffle │ ├── ShuffleInMemorySorter.graffle │ ├── SortShuffleManager.graffle │ ├── UnsafeShuffleWriter-ShuffleExternalSorter.graffle │ └── UnsafeShuffleWriter.graffle ├── spark-SQLTab-creating-instance.graffle ├── spark-YarnSchedulerBackend-doRequestTotalExecutors.graffle ├── spark-YarnSchedulerEndpoint-RequestExecutors.graffle ├── spark-driver.graffle ├── spark-mesos.graffle ├── spark-metrics-MetricsSystem-driver.graffle ├── spark-metrics-MetricsSystem.graffle ├── spark-mllib-pipeline.graffle ├── spark-pool-FairSchedulingAlgorithm.graffle ├── spark-runjob.graffle ├── spark-standalone-master-worker-LaunchDriver.graffle ├── spark-webui-SparkUI.graffle ├── spark-yarn-ApplicationMaster-client-submitApplication.graffle ├── spark-yarn-ApplicationMaster-main.graffle ├── spark-yarn-ApplicationMaster-registerAM.graffle ├── spark-yarn-ApplicationMaster-runAMEndpoint.graffle ├── spark-yarn-ApplicationMaster.graffle ├── spark-yarn-Client-YarnClient.graffle ├── spark-yarn-ExecutorRunnable.graffle ├── spark-yarn-YarnAllocator-amClient-ResourceManager.graffle ├── spark-yarn-YarnAllocator-runAllocatedContainers.graffle ├── spark-yarn-YarnAllocator.graffle ├── spark-yarn-YarnClientSchedulerBackend-start.graffle ├── spark-yarn-YarnRMClient-register.graffle ├── sparkapp-sparkcontext-master-slaves.graffle ├── sparkcontext-broadcast-executors.graffle ├── sparkcontext-broadcastmanager-contextcleaner.graffle ├── storage │ ├── BlockInfoManager-BlockManager.graffle │ ├── BlockManager-SparkEnv.graffle │ ├── BlockManager.graffle │ ├── BlockManagerMaster-RegisterBlockManager.graffle │ ├── DiskBlockManager-BlockManager.graffle │ ├── DiskStore-BlockManager.graffle │ ├── MemoryStore-BlockManager.graffle │ ├── NettyBlockRpcServer.graffle │ ├── NettyBlockTransferService.graffle │ ├── ShuffleMetricsSource.graffle │ └── spark-MemoryStore.graffle ├── taskscheduler-submitTasks-local-mode.graffle └── yarn-YarnSchedulerBackend.graffle ├── graphs ├── rpc │ └── rpcenv-endpoints.graphml └── scheduler │ ├── action-job.graphml │ ├── dagscheduler-job-resultstage.graphml │ ├── job-stage.graphml │ ├── rdd-job-partitions.graphml │ ├── scheduler-handlemapstagesubmitted.graphml │ ├── scheduler-job-shuffles-result-stages.graphml │ ├── scheduler-job-splits-into-stages.graphml │ ├── stage-tasks.graphml │ └── taskscheduler-uses-schedulerbackend.graphml ├── mkdocs.yml ├── modules ├── spark-mllib │ ├── nav.adoc │ └── pages │ │ ├── index.adoc │ │ ├── spark-mllib-ALS.adoc │ │ ├── spark-mllib-ALSModel.adoc │ │ ├── spark-mllib-ALSModelReader.adoc │ │ ├── spark-mllib-BinaryClassificationEvaluator.adoc │ │ ├── spark-mllib-Classifier.adoc │ │ ├── spark-mllib-ClusteringEvaluator.adoc │ │ ├── spark-mllib-CrossValidator-example.adoc │ │ ├── spark-mllib-CrossValidator.adoc │ │ ├── spark-mllib-CrossValidatorModel.adoc │ │ ├── spark-mllib-DecisionTreeClassifier.adoc │ │ ├── spark-mllib-Estimator.adoc │ │ ├── spark-mllib-Evaluator.adoc │ │ ├── spark-mllib-GeneralizedLinearRegression.adoc │ │ ├── spark-mllib-HasParallelism.adoc │ │ ├── spark-mllib-Instrumentation.adoc │ │ ├── spark-mllib-KMeans.adoc │ │ ├── spark-mllib-LinearRegression.adoc │ │ ├── spark-mllib-LogisticRegression.adoc │ │ ├── spark-mllib-MLReader.adoc │ │ ├── spark-mllib-MLUtils.adoc │ │ ├── spark-mllib-MLWritable.adoc │ │ ├── spark-mllib-Model.adoc │ │ ├── spark-mllib-MulticlassClassificationEvaluator.adoc │ │ ├── spark-mllib-ParamGridBuilder.adoc │ │ ├── spark-mllib-Params.adoc │ │ ├── spark-mllib-Pipeline.adoc │ │ ├── spark-mllib-PipelineStage.adoc │ │ ├── spark-mllib-Predictor.adoc │ │ ├── spark-mllib-RandomForestClassifier.adoc │ │ ├── spark-mllib-RandomForestRegressor.adoc │ │ ├── spark-mllib-RegressionEvaluator.adoc │ │ ├── spark-mllib-Regressor.adoc │ │ ├── spark-mllib-StringIndexer.adoc │ │ ├── spark-mllib-TrainValidationSplit.adoc │ │ ├── spark-mllib-Transformer.adoc │ │ ├── spark-mllib-ValidatorParams.adoc │ │ ├── spark-mllib-alternating-least-squares.adoc │ │ ├── spark-mllib-estimators.adoc │ │ ├── spark-mllib-labeledpoint.adoc │ │ ├── spark-mllib-latent-dirichlet-allocation.adoc │ │ ├── spark-mllib-logistic-regression.adoc │ │ ├── spark-mllib-models.adoc │ │ ├── spark-mllib-pipelines-example-classification.adoc │ │ ├── spark-mllib-pipelines-example-regression.adoc │ │ ├── spark-mllib-pipelines-persistence.adoc │ │ ├── spark-mllib-pipelines.adoc │ │ ├── spark-mllib-streaming.adoc │ │ ├── spark-mllib-transformers-Tokenizer.adoc │ │ ├── spark-mllib-transformers.adoc │ │ └── spark-mllib-vector.adoc ├── spark-on-yarn │ ├── assets │ │ └── images │ │ │ ├── spark-yarn-ApplicationMaster-client-submitApplication.png │ │ │ ├── spark-yarn-ApplicationMaster-main.png │ │ │ ├── spark-yarn-ApplicationMaster-registerAM.png │ │ │ ├── spark-yarn-ApplicationMaster-runAMEndpoint.png │ │ │ ├── spark-yarn-ApplicationMaster.png │ │ │ ├── spark-yarn-Client-YarnClient.png │ │ │ ├── spark-yarn-ExecutorRunnable.png │ │ │ ├── spark-yarn-YarnAllocator-amClient-ResourceManager.png │ │ │ ├── spark-yarn-YarnAllocator-runAllocatedContainers.png │ │ │ ├── spark-yarn-YarnAllocator.png │ │ │ ├── spark-yarn-YarnClientSchedulerBackend-start.png │ │ │ ├── spark-yarn-YarnRMClient-register.png │ │ │ └── spark-yarn-console-progress-10.png │ ├── nav.adoc │ └── pages │ │ ├── index.adoc │ │ ├── spark-yarn-AMEndpoint.adoc │ │ ├── spark-yarn-ClientDistributedCacheManager.adoc │ │ ├── spark-yarn-ConfigurableCredentialManager.adoc │ │ ├── spark-yarn-ExecutorRunnable.adoc │ │ ├── spark-yarn-YarnAllocator.adoc │ │ ├── spark-yarn-YarnClusterManager.adoc │ │ ├── spark-yarn-YarnShuffleService.adoc │ │ ├── spark-yarn-YarnSparkHadoopUtil.adoc │ │ ├── spark-yarn-applicationmaster.adoc │ │ ├── spark-yarn-client-yarnclientschedulerbackend.adoc │ │ ├── spark-yarn-client.adoc │ │ ├── spark-yarn-cluster-YarnSchedulerEndpoint.adoc │ │ ├── spark-yarn-cluster-setup.adoc │ │ ├── spark-yarn-cluster-yarnclusterschedulerbackend.adoc │ │ ├── spark-yarn-introduction.adoc │ │ ├── spark-yarn-kerberos.adoc │ │ ├── spark-yarn-schedulerbackends.adoc │ │ ├── spark-yarn-settings.adoc │ │ ├── spark-yarn-taskschedulers.adoc │ │ ├── spark-yarn-yarnclusterscheduler.adoc │ │ ├── spark-yarn-yarnrmclient.adoc │ │ ├── spark-yarn-yarnscheduler.adoc │ │ └── spark-yarn-yarnschedulerbackend.adoc └── spark-standalone │ └── pages │ ├── spark-standalone-example-2-workers-on-1-node-cluster.adoc │ ├── spark-standalone-master-scripts.adoc │ ├── spark-standalone-status.adoc │ ├── spark-standalone-submission-gateways.adoc │ ├── spark-standalone-webui-ApplicationPage.adoc │ ├── spark-standalone-webui.adoc │ └── spark-standalone-worker-scripts.adoc └── requirements.txt /.gitattributes: -------------------------------------------------------------------------------- 1 | *.md diff 2 | -------------------------------------------------------------------------------- /.github/FUNDING.yml: -------------------------------------------------------------------------------- 1 | github: jaceklaskowski 2 | ko_fi: jaceklaskowski 3 | custom: "https://paypal.me/JacekLaskowski" 4 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | # Based on https://github.com/squidfunk/mkdocs-material/blob/master/.github/workflows/ci.yml 2 | 3 | name: CI 4 | on: 5 | push: 6 | branches: 7 | - main 8 | 9 | jobs: 10 | deploy: 11 | if: github.event.pull_request.head.repo.fork == false 12 | runs-on: ubuntu-latest 13 | steps: 14 | - uses: actions/checkout@v2 15 | with: 16 | fetch-depth: 0 17 | - uses: actions/setup-python@v2 18 | with: 19 | python-version: 3.x 20 | - name: Install dependencies 21 | env: 22 | GH_TOKEN: ${{ secrets.GH_TOKEN }} 23 | run: | 24 | pip install git+https://${GH_TOKEN}@github.com/squidfunk/mkdocs-material-insiders.git 25 | pip install -r requirements.txt 26 | - name: Build documentation 27 | env: 28 | GOOGLE_ANALYTICS_KEY: ${{ secrets.GOOGLE_ANALYTICS_KEY }} 29 | run: | 30 | mkdocs gh-deploy --force 31 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | site/ 2 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # The Internals of Spark Core Online Book 2 | 3 | [![CI](https://github.com/japila-books/apache-spark-internals/workflows/CI/badge.svg)](https://github.com/japila-books/apache-spark-internals/actions) 4 | 5 | The project contains the sources of [The Internals of Spark Core](https://books.japila.pl/apache-spark-internals) online book. 6 | -------------------------------------------------------------------------------- /docs/ExecutorDeadException.md: -------------------------------------------------------------------------------- 1 | # ExecutorDeadException 2 | 3 | `ExecutorDeadException` is a `SparkException`. 4 | 5 | ## Creating Instance 6 | 7 | `ExecutorDeadException` takes the following to be created: 8 | 9 | * Error message 10 | 11 | `ExecutorDeadException` is created when: 12 | 13 | * `NettyBlockTransferService` is requested to [fetch blocks](storage/NettyBlockTransferService.md#fetchBlocks) 14 | -------------------------------------------------------------------------------- /docs/HadoopMapRedCommitProtocol.md: -------------------------------------------------------------------------------- 1 | # HadoopMapRedCommitProtocol 2 | 3 | `HadoopMapRedCommitProtocol` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/HadoopMapRedWriteConfigUtil.md: -------------------------------------------------------------------------------- 1 | # HadoopMapRedWriteConfigUtil 2 | 3 | `HadoopMapRedWriteConfigUtil` is a [HadoopWriteConfigUtil](HadoopWriteConfigUtil.md) for [RDD.saveAsHadoopDataset](rdd/PairRDDFunctions.md#saveAsHadoopDataset) operator. 4 | 5 | ## Creating Instance 6 | 7 | `HadoopMapRedWriteConfigUtil` takes the following to be created: 8 | 9 | * `SerializableJobConf` 10 | 11 | `HadoopMapRedWriteConfigUtil` is created when: 12 | 13 | * `PairRDDFunctions` is requested to [saveAsHadoopDataset](rdd/PairRDDFunctions.md#saveAsHadoopDataset) 14 | 15 | ## Logging 16 | 17 | Enable `ALL` logging level for `org.apache.spark.internal.io.HadoopMapRedWriteConfigUtil` logger to see what happens inside. 18 | 19 | Add the following line to `conf/log4j2.properties`: 20 | 21 | ```text 22 | logger.HadoopMapRedWriteConfigUtil.name = org.apache.spark.internal.io.HadoopMapRedWriteConfigUtil 23 | logger.HadoopMapRedWriteConfigUtil.level = all 24 | ``` 25 | 26 | Refer to [Logging](spark-logging.md). 27 | -------------------------------------------------------------------------------- /docs/HadoopMapReduceWriteConfigUtil.md: -------------------------------------------------------------------------------- 1 | # HadoopMapReduceWriteConfigUtil 2 | 3 | `HadoopMapReduceWriteConfigUtil` is a [HadoopWriteConfigUtil](HadoopWriteConfigUtil.md) for [RDD.saveAsNewAPIHadoopDataset](rdd/PairRDDFunctions.md#saveAsNewAPIHadoopDataset) operator. 4 | 5 | ## Creating Instance 6 | 7 | `HadoopMapReduceWriteConfigUtil` takes the following to be created: 8 | 9 | * `SerializableConfiguration` 10 | 11 | `HadoopMapReduceWriteConfigUtil` is created when: 12 | 13 | * `PairRDDFunctions` is requested to [saveAsNewAPIHadoopDataset](rdd/PairRDDFunctions.md#saveAsNewAPIHadoopDataset) 14 | 15 | ## Logging 16 | 17 | Enable `ALL` logging level for `org.apache.spark.internal.io.HadoopMapReduceWriteConfigUtil` logger to see what happens inside. 18 | 19 | Add the following line to `conf/log4j2.properties`: 20 | 21 | ```text 22 | logger.HadoopMapReduceWriteConfigUtil.name = org.apache.spark.internal.io.HadoopMapReduceWriteConfigUtil 23 | logger.HadoopMapReduceWriteConfigUtil.level = all 24 | ``` 25 | 26 | Refer to [Logging](spark-logging.md). 27 | -------------------------------------------------------------------------------- /docs/PartitionEvaluator.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # PartitionEvaluator 7 | 8 | `PartitionEvaluator[T, U]` is an [abstraction](#contract) of [partition evaluators](#implementations) that can [compute (_evaluate_) one or more RDD partitions](#eval). 9 | 10 | ## Contract 11 | 12 | ### Evaluate Partitions { #eval } 13 | 14 | ```scala 15 | eval( 16 | partitionIndex: Int, 17 | inputs: Iterator[T]*): Iterator[U] 18 | ``` 19 | 20 | Used when: 21 | 22 | * `MapPartitionsWithEvaluatorRDD` is requested to [compute a partition](rdd/MapPartitionsWithEvaluatorRDD.md#compute) 23 | * `ZippedPartitionsWithEvaluatorRDD` is requested to [compute a partition](rdd/ZippedPartitionsWithEvaluatorRDD.md#compute) 24 | 25 | ## Implementations 26 | 27 | !!! note 28 | No built-in implementations available in Spark Core (but [Spark SQL]({{ book.spark_sql }})). 29 | -------------------------------------------------------------------------------- /docs/PartitionEvaluatorFactory.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # PartitionEvaluatorFactory 7 | 8 | `PartitionEvaluatorFactory[T, U]` is an [abstraction](#contract) of [PartitionEvaluator factories](#implementations). 9 | 10 | `PartitionEvaluatorFactory` is a `Serializable` ([Java]({{ java.api }}/java/io/Serializable.html)). 11 | 12 | ## Contract 13 | 14 | ### Creating PartitionEvaluator { #createEvaluator } 15 | 16 | ```scala 17 | createEvaluator(): PartitionEvaluator[T, U] 18 | ``` 19 | 20 | Creates a [PartitionEvaluator](PartitionEvaluator.md) 21 | 22 | Used when: 23 | 24 | * `MapPartitionsWithEvaluatorRDD` is requested to [compute a partition](rdd/MapPartitionsWithEvaluatorRDD.md#compute) 25 | * `ZippedPartitionsWithEvaluatorRDD` is requested to [compute a partition](rdd/ZippedPartitionsWithEvaluatorRDD.md#compute) 26 | 27 | ## Implementations 28 | 29 | !!! note 30 | No built-in implementations available in Spark Core (but [Spark SQL]({{ book.spark_sql }})). 31 | -------------------------------------------------------------------------------- /docs/SparkCoreErrors.md: -------------------------------------------------------------------------------- 1 | # SparkCoreErrors 2 | 3 | ## numPartitionsGreaterThanMaxNumConcurrentTasksError { #numPartitionsGreaterThanMaxNumConcurrentTasksError } 4 | 5 | ```scala 6 | numPartitionsGreaterThanMaxNumConcurrentTasksError( 7 | numPartitions: Int, 8 | maxNumConcurrentTasks: Int): Throwable 9 | ``` 10 | 11 | `numPartitionsGreaterThanMaxNumConcurrentTasksError` creates a [BarrierJobSlotsNumberCheckFailed](barrier-execution-mode/BarrierJobSlotsNumberCheckFailed.md) with the given input arguments. 12 | 13 | --- 14 | 15 | `numPartitionsGreaterThanMaxNumConcurrentTasksError` is used when: 16 | 17 | * `DAGScheduler` is requested to [checkBarrierStageWithNumSlots](scheduler/DAGScheduler.md#checkBarrierStageWithNumSlots) 18 | -------------------------------------------------------------------------------- /docs/SparkFiles.md: -------------------------------------------------------------------------------- 1 | # SparkFiles 2 | 3 | `SparkFiles` is an utility to work with files added using [SparkContext.addFile](SparkContext.md#addFile). 4 | 5 | ## Absolute Path of Added File 6 | 7 | ```scala 8 | get( 9 | filename: String): String 10 | ``` 11 | 12 | `get` gets the absolute path of the given file in the [root directory](#getRootDirectory). 13 | 14 | ## Root Directory 15 | 16 | ```scala 17 | getRootDirectory(): String 18 | ``` 19 | 20 | `getRootDirectory` requests the current `SparkEnv` for [driverTmpDir](SparkEnv.md#driverTmpDir) (if defined) or defaults to the current directory (`.`). 21 | 22 | `getRootDirectory` is used when: 23 | 24 | * `SparkContext` is requested to [addFile](SparkContext.md#addFile) 25 | * `Executor` is requested to [updateDependencies](executor/Executor.md#updateDependencies) 26 | * `SparkFiles` utility is requested to [get the absolute path of a file](#get) 27 | -------------------------------------------------------------------------------- /docs/SparkListener.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # SparkListener 7 | 8 | `SparkListener` is an extension of the [SparkListenerInterface](SparkListenerInterface.md) abstraction for [event listeners](#implementations) with a no-op implementation for callback methods. 9 | 10 | ## Implementations 11 | 12 | * BarrierCoordinator 13 | * SparkSession ([Spark SQL]({{ book.spark_sql }}/SparkSession/#registerContextListener)) 14 | * AppListingListener (Spark History Server) 15 | * [AppStatusListener](status/AppStatusListener.md) 16 | * BasicEventFilterBuilder (Spark History Server) 17 | * [EventLoggingListener](history-server/EventLoggingListener.md) (Spark History Server) 18 | * ExecutionListenerBus 19 | * [ExecutorAllocationListener](dynamic-allocation/ExecutorAllocationListener.md) 20 | * ExecutorMonitor 21 | * [HeartbeatReceiver](HeartbeatReceiver.md) 22 | * HiveThriftServer2Listener (Spark Thrift Server) 23 | * [SpillListener](SpillListener.md) 24 | * SQLAppStatusListener ([Spark SQL]({{ book.spark_sql }}/SQLAppStatusListener/)) 25 | * SQLEventFilterBuilder 26 | * [StatsReportListener](StatsReportListener.md) 27 | * StreamingQueryListenerBus ([Spark Structured Streaming]({{ book.structured_streaming }}/StreamingQueryListenerBus/)) 28 | -------------------------------------------------------------------------------- /docs/SparkListenerBus.md: -------------------------------------------------------------------------------- 1 | # SparkListenerBus 2 | 3 | `SparkListenerBus` is an extension of the [ListenerBus](ListenerBus.md) abstraction for [event buses](#implementations) for [SparkListenerInterface](SparkListenerInterface.md)s to be notified about [SparkListenerEvent](SparkListenerEvent.md)s. 4 | 5 | # Posting Event to SparkListener 6 | 7 | ```scala 8 | doPostEvent( 9 | listener: SparkListenerInterface, 10 | event: SparkListenerEvent): Unit 11 | ``` 12 | 13 | `doPostEvent` is part of the [ListenerBus](ListenerBus.md#doPostEvent) abstraction. 14 | 15 | `doPostEvent` notifies the given [SparkListenerInterface](SparkListenerInterface.md) about the [SparkListenerEvent](SparkListenerEvent.md). 16 | 17 | `doPostEvent` calls an event-specific method of [SparkListenerInterface](SparkListenerInterface.md) or falls back to [onOtherEvent](SparkListenerInterface.md#onOtherEvent). 18 | 19 | ## Implementations 20 | 21 | * `AsyncEventQueue` 22 | * [ReplayListenerBus](history-server/ReplayListenerBus.md) 23 | -------------------------------------------------------------------------------- /docs/SparkListenerTaskEnd.md: -------------------------------------------------------------------------------- 1 | # SparkListenerTaskEnd 2 | 3 | `SparkListenerTaskEnd` is a [SparkListenerEvent](SparkListenerEvent.md). 4 | 5 | `SparkListenerTaskEnd` is posted (and [created](#creating-instance)) when: 6 | 7 | * `DAGScheduler` is requested to [postTaskEnd](scheduler/DAGScheduler.md#postTaskEnd) 8 | 9 | `SparkListenerTaskEnd` is intercepted using [SparkListenerInterface.onTaskEnd](SparkListenerInterface.md#onTaskEnd) 10 | 11 | ## Creating Instance 12 | 13 | `SparkListenerTaskEnd` takes the following to be created: 14 | 15 | * Stage ID 16 | * Stage Attempt ID 17 | * Task Type 18 | * `TaskEndReason` 19 | * [TaskInfo](scheduler/TaskInfo.md) 20 | * `ExecutorMetrics` 21 | * [TaskMetrics](executor/TaskMetrics.md) 22 | -------------------------------------------------------------------------------- /docs/SparkStatusTracker.md: -------------------------------------------------------------------------------- 1 | # SparkStatusTracker 2 | 3 | `SparkStatusTracker` is created for [SparkContext](SparkContext.md#statusTracker) for Spark developers to access the [AppStatusStore](#store) and the following: 4 | 5 | * All active job IDs 6 | * All active stage IDs 7 | * All known job IDs (and possibly limited to a particular job group) 8 | * `SparkExecutorInfo`s of all known executors 9 | * `SparkJobInfo` of a job ID 10 | * `SparkStageInfo` of a stage ID 11 | 12 | ## Creating Instance 13 | 14 | `SparkStatusTracker` takes the following to be created: 15 | 16 | * [SparkContext](SparkContext.md) (_unused_) 17 | * [AppStatusStore](status/AppStatusStore.md) 18 | 19 | `SparkStatusTracker` is created when: 20 | 21 | * `SparkContext` is [created](SparkContext.md#_statusTracker) 22 | -------------------------------------------------------------------------------- /docs/SpillListener.md: -------------------------------------------------------------------------------- 1 | # SpillListener 2 | 3 | `SpillListener` is a [SparkListener](SparkListener.md) that intercepts (_listens to_) the following events for detecting spills in jobs: 4 | 5 | * [onTaskEnd](#onTaskEnd) 6 | * [onStageCompleted](#onStageCompleted) 7 | 8 | `SpillListener` is used for testing only. 9 | 10 | ## Creating Instance 11 | 12 | `SpillListener` takes no input arguments to be created. 13 | 14 | `SpillListener` is created when `TestUtils` is requested to `assertSpilled` and `assertNotSpilled`. 15 | 16 | ## onTaskEnd Callback 17 | 18 | ```scala 19 | onTaskEnd( 20 | taskEnd: SparkListenerTaskEnd): Unit 21 | ``` 22 | 23 | `onTaskEnd`...FIXME 24 | 25 | `onTaskEnd` is part of the [SparkListener](SparkListener.md#onTaskEnd) abstraction. 26 | 27 | ## onStageCompleted Callback 28 | 29 | ```scala 30 | onStageCompleted( 31 | stageComplete: SparkListenerStageCompleted): Unit 32 | ``` 33 | 34 | `onStageCompleted`...FIXME 35 | 36 | `onStageCompleted` is part of the [SparkListener](SparkListener.md#onStageCompleted) abstraction. 37 | -------------------------------------------------------------------------------- /docs/TaskCompletionListener.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # TaskCompletionListener 7 | 8 | `TaskCompletionListener` is an [extension](#contract) of the `EventListener` ([Java]({{ java.api }}/java/util/EventListener.html)) abstraction for [task listeners](#implementations) that can be notified [on task completion](#onTaskCompletion). 9 | 10 | ## Contract 11 | 12 | ###  onTaskCompletion 13 | 14 | ```scala 15 | onTaskCompletion( 16 | context: TaskContext): Unit 17 | ``` 18 | 19 | Used when: 20 | 21 | * `TaskContextImpl` is requested to [addTaskCompletionListener](scheduler/TaskContextImpl.md#addTaskCompletionListener) (and a task has already completed) and [markTaskCompleted](scheduler/TaskContextImpl.md#markTaskCompleted) 22 | * `ShuffleFetchCompletionListener` is requested to [onComplete](storage/ShuffleFetchCompletionListener.md#onComplete) 23 | -------------------------------------------------------------------------------- /docs/TaskFailureListener.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # TaskFailureListener 7 | 8 | `TaskFailureListener` is an [extension](#contract) of the `EventListener` ([Java]({{ java.api }}/java/util/EventListener.html)) abstraction for [task listeners](#implementations) that can be notified [on task failure](#onTaskFailure). 9 | 10 | ## Contract 11 | 12 | ###  onTaskFailure 13 | 14 | ```scala 15 | onTaskFailure( 16 | context: TaskContext, 17 | error: Throwable): Unit 18 | ``` 19 | 20 | Used when: 21 | 22 | * `TaskContextImpl` is requested to [addTaskFailureListener](scheduler/TaskContextImpl.md#addTaskFailureListener) (and a task has already failed) and [markTaskFailed](scheduler/TaskContextImpl.md#markTaskFailed) 23 | -------------------------------------------------------------------------------- /docs/accumulators/AccumulatorContext.md: -------------------------------------------------------------------------------- 1 | == [[AccumulatorContext]] AccumulatorContext 2 | 3 | `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. 4 | 5 | The `originals` lookup table maps accumulator identifier to the accumulator itself. 6 | 7 | Every accumulator has its own unique accumulator id that is assigned using the internal `nextId` counter. 8 | 9 | === [[register]] `register` Method 10 | 11 | CAUTION: FIXME 12 | 13 | === [[newId]] `newId` Method 14 | 15 | CAUTION: FIXME 16 | 17 | === [[AccumulatorContext-SQL_ACCUM_IDENTIFIER]] AccumulatorContext.SQL_ACCUM_IDENTIFIER 18 | 19 | `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. 20 | -------------------------------------------------------------------------------- /docs/accumulators/AccumulatorSource.md: -------------------------------------------------------------------------------- 1 | # AccumulatorSource 2 | 3 | `AccumulatorSource` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/accumulators/InternalAccumulator.md: -------------------------------------------------------------------------------- 1 | # InternalAccumulator 2 | 3 | `InternalAccumulator` is an utility with field names for internal accumulators. 4 | 5 | ## internal.metrics Prefix 6 | 7 | `internal.metrics.` is the prefix of metrics that are considered internal and should not be displayed in web UI. 8 | 9 | `internal.metrics.` is used when: 10 | 11 | * `AccumulatorV2` is requested to [convert itself to AccumulableInfo](AccumulatorV2.md#toInfo) and [writeReplace](AccumulatorV2.md#writeReplace) 12 | * `JsonProtocol` is requested to [accumValueToJson](../history-server/JsonProtocol.md#accumValueToJson) and [accumValueFromJson](../history-server/JsonProtocol.md#accumValueFromJson) 13 | -------------------------------------------------------------------------------- /docs/architecture.md: -------------------------------------------------------------------------------- 1 | = Spark Architecture 2 | 3 | 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. 4 | 5 | .Spark architecture 6 | image::driver-sparkcontext-clustermanager-workers-executors.png[align="center"] 7 | 8 | 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. 9 | 10 | .Spark architecture in detail 11 | image::sparkapp-sparkcontext-master-slaves.png[align="center"] 12 | 13 | Physical machines are called *hosts* or *nodes*. 14 | -------------------------------------------------------------------------------- /docs/barrier-execution-mode/.pages: -------------------------------------------------------------------------------- 1 | title: Barrier Execution Mode 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/barrier-execution-mode/BarrierCoordinatorMessage.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: BarrierCoordinatorMessage 3 | --- 4 | 5 | # BarrierCoordinatorMessage RPC Messages 6 | 7 | `BarrierCoordinatorMessage` is an abstraction of [RPC messages](#implementations) that tasks can send out using [BarrierTaskContext](BarrierTaskContext.md#runBarrier) operators for [BarrierCoordinator](BarrierCoordinator.md) to handle. 8 | 9 | `BarrierCoordinatorMessage` is a `Serializable` ([Java]({{ java.api }}/java/io/Serializable.html)) (so it can be sent from executors to the driver over the wire). 10 | 11 | ## Implementations 12 | 13 | ??? note "Sealed Trait" 14 | `BarrierCoordinatorMessage` is a Scala **sealed trait** which means that all of the implementations are in the same compilation unit (a single file). 15 | 16 | Learn more in the [Scala Language Specification]({{ scala.spec }}/05-classes-and-objects.html#sealed). 17 | 18 | * [RequestToSync](RequestToSync.md) 19 | -------------------------------------------------------------------------------- /docs/barrier-execution-mode/BarrierJobAllocationFailed.md: -------------------------------------------------------------------------------- 1 | # BarrierJobAllocationFailed 2 | 3 | `BarrierJobAllocationFailed` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/barrier-execution-mode/RDDBarrier.md: -------------------------------------------------------------------------------- 1 | # RDDBarrier 2 | 3 | `RDDBarrier` is a wrapper around [RDD](#rdd) with two custom map transformations: 4 | 5 | * [mapPartitions](#mapPartitions) 6 | * [mapPartitionsWithIndex](#mapPartitionsWithIndex) 7 | 8 | Unlike regular [RDD.mapPartitions](../rdd/RDD.md#mapPartitions) transformations, `RDDBarrier` transformations create a [MapPartitionsRDD](../rdd/MapPartitionsRDD.md) with [isFromBarrier](../rdd/MapPartitionsRDD.md#isFromBarrier) flag enabled. 9 | 10 | `RDDBarrier` (of `T` records) marks the current stage as a [barrier stage](index.md#barrier-stage) in [Barrier Execution Mode](index.md). 11 | 12 | ## Creating Instance 13 | 14 | `RDDBarrier` takes the following to be created: 15 | 16 | * [RDD](../rdd/RDD.md) (of `T` records) 17 | 18 | `RDDBarrier` is created when: 19 | 20 | * [RDD.barrier](../rdd/RDD.md#barrier) transformation is used 21 | -------------------------------------------------------------------------------- /docs/barrier-execution-mode/RequestMethod.md: -------------------------------------------------------------------------------- 1 | # RequestMethod 2 | 3 | `RequestMethod` represents the allowed request methods of [RequestToSync](RequestToSync.md#requestMethod)s (that are sent out from barrier tasks using [BarrierTaskContext](BarrierTaskContext.md#runBarrier)). 4 | 5 | [ContextBarrierState](ContextBarrierState.md#requestMethods) tracks `RequestMethod`s (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. 6 | 7 | ## BARRIER { #BARRIER } 8 | 9 | Marks execution of [BarrierTaskContext.barrier](BarrierTaskContext.md#barrier) 10 | 11 | ## ALL_GATHER { #ALL_GATHER } 12 | 13 | Marks execution of [BarrierTaskContext.allGather](BarrierTaskContext.md#allGather) 14 | -------------------------------------------------------------------------------- /docs/core/BlockFetchStarter.md: -------------------------------------------------------------------------------- 1 | # BlockFetchStarter 2 | 3 | *BlockFetchStarter* is the <> of...FIXME...to <>. 4 | 5 | [[contract]] 6 | [[createAndStart]] 7 | [source, java] 8 | ---- 9 | void createAndStart(String[] blockIds, BlockFetchingListener listener) 10 | throws IOException, InterruptedException; 11 | ---- 12 | 13 | `createAndStart` is used when: 14 | 15 | * `NettyBlockTransferService` is requested to storage:NettyBlockTransferService.md#fetchBlocks[fetchBlocks] (when network:TransportConf.md#io.maxRetries[maxIORetries] is `0`) 16 | 17 | * `RetryingBlockFetcher` is requested to core:RetryingBlockFetcher.md#fetchAllOutstanding[fetchAllOutstanding] 18 | -------------------------------------------------------------------------------- /docs/core/BlockFetchingListener.md: -------------------------------------------------------------------------------- 1 | # BlockFetchingListener 2 | 3 | `BlockFetchingListener` is an [extension](#contract) of the `EventListener` ([Java]({{ java.api }}/java/util/EventListener.html)) abstraction that want to be notified about [block fetch success](#onBlockFetchSuccess) and [failures](#onBlockFetchFailure). 4 | 5 | `BlockFetchingListener` is used to create a [OneForOneBlockFetcher](../storage/OneForOneBlockFetcher.md), `OneForOneBlockPusher` and [RetryingBlockFetcher](RetryingBlockFetcher.md). 6 | 7 | ## Contract 8 | 9 | ###  onBlockFetchFailure 10 | 11 | ```java 12 | void onBlockFetchFailure( 13 | String blockId, 14 | Throwable exception) 15 | ``` 16 | 17 | ###  onBlockFetchSuccess 18 | 19 | ```java 20 | void onBlockFetchSuccess( 21 | String blockId, 22 | ManagedBuffer data) 23 | ``` 24 | 25 | ## Implementations 26 | 27 | * "Unnamed" in [ShuffleBlockFetcherIterator](../storage/ShuffleBlockFetcherIterator.md#sendRequest) 28 | * "Unnamed" in [BlockTransferService](../storage/BlockTransferService.md#fetchBlockSync) 29 | * [RetryingBlockFetchListener](RetryingBlockFetcher.md#RetryingBlockFetchListener) 30 | -------------------------------------------------------------------------------- /docs/core/InMemoryStore.md: -------------------------------------------------------------------------------- 1 | # InMemoryStore 2 | 3 | `InMemoryStore` is a [KVStore](KVStore.md). 4 | 5 | ## Creating Instance 6 | 7 | `InMemoryStore` takes no arguments to be created. 8 | 9 | `InMemoryStore` is created when: 10 | 11 | * `FsHistoryProvider` is [created](../history-server/FsHistoryProvider.md#listing) and requested to [createInMemoryStore](../history-server/FsHistoryProvider.md#createInMemoryStore) 12 | * `AppStatusStore` utility is used to [create an AppStatusStore for a live Spark application](../status/AppStatusStore.md#createLiveStore) 13 | -------------------------------------------------------------------------------- /docs/core/LevelDB.md: -------------------------------------------------------------------------------- 1 | # LevelDB 2 | 3 | `LevelDB` is a [KVStore](KVStore.md) for [FsHistoryProvider](../history-server/FsHistoryProvider.md). 4 | 5 | ## Creating Instance 6 | 7 | `LevelDB` takes the following to be created: 8 | 9 | * Path 10 | * `KVStoreSerializer` 11 | 12 | `LevelDB` is created when: 13 | 14 | * `KVUtils` utility is used to `open` (a LevelDB store) 15 | -------------------------------------------------------------------------------- /docs/demo/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | - navigation 5 | --- 6 | 7 | # Demos 8 | 9 | The following demos are available: 10 | 11 | - [DiskBlockManager and Block Data](diskblockmanager-and-block-data.md) 12 | -------------------------------------------------------------------------------- /docs/developer-api.md: -------------------------------------------------------------------------------- 1 | # Developer API 2 | 3 | [TAGS] 4 | -------------------------------------------------------------------------------- /docs/dynamic-allocation/ExecutorAllocationListener.md: -------------------------------------------------------------------------------- 1 | # ExecutorAllocationListener 2 | 3 | `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](ExecutorAllocationManager.md) can manage the pool of dynamically managed executors. 4 | 5 | !!! note "Internal Class" 6 | `ExecutorAllocationListener` is an internal class of [ExecutorAllocationManager](ExecutorAllocationManager.md) with full access to internal registries. 7 | -------------------------------------------------------------------------------- /docs/executor/ExecutorBackend.md: -------------------------------------------------------------------------------- 1 | # ExecutorBackend 2 | 3 | `ExecutorBackend` is an [abstraction](#contract) of [executor backends](#implementations) (that [TaskRunner](TaskRunner.md)s use to [report task status updates](#statusUpdate) to a scheduler). 4 | 5 | ![ExecutorBackend receives notifications from TaskRunners](../images/executor/ExecutorBackend.png) 6 | 7 | `ExecutorBackend` acts as a bridge between executors and the driver. 8 | 9 | ## Contract 10 | 11 | ### Reporting Task Status { #statusUpdate } 12 | 13 | ```scala 14 | statusUpdate( 15 | taskId: Long, 16 | state: TaskState, 17 | data: ByteBuffer): Unit 18 | ``` 19 | 20 | Reports task status of the given task to a scheduler 21 | 22 | See: 23 | 24 | * [CoarseGrainedExecutorBackend](CoarseGrainedExecutorBackend.md#statusUpdate) 25 | 26 | Used when: 27 | 28 | * `TaskRunner` is requested to [run a task](TaskRunner.md#run) 29 | 30 | ## Implementations 31 | 32 | * [CoarseGrainedExecutorBackend](CoarseGrainedExecutorBackend.md) 33 | * [LocalSchedulerBackend](../local/LocalSchedulerBackend.md) 34 | * `MesosExecutorBackend` 35 | -------------------------------------------------------------------------------- /docs/executor/ShuffleReadMetrics.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # ShuffleReadMetrics 7 | 8 | `ShuffleReadMetrics` is a collection of metrics (accumulators) on reading shuffle data. 9 | 10 | ## TaskMetrics 11 | 12 | `ShuffleReadMetrics` is available using [TaskMetrics.shuffleReadMetrics](TaskMetrics.md#shuffleReadMetrics). 13 | 14 | ## Serializable 15 | 16 | `ShuffleReadMetrics` is a `Serializable` ([Java]({{ java.api }}/java/io/Serializable.html)). 17 | -------------------------------------------------------------------------------- /docs/exercises/spark-exercise-failing-stage.md: -------------------------------------------------------------------------------- 1 | == Exercise: Causing Stage to Fail 2 | 3 | The example shows how Spark re-executes a stage in case of stage failure. 4 | 5 | === Recipe 6 | 7 | Start a Spark cluster, e.g. 1-node Hadoop YARN. 8 | 9 | ``` 10 | start-yarn.sh 11 | ``` 12 | 13 | ``` 14 | // 2-stage job -- it _appears_ that a stage can be failed only when there is a shuffle 15 | sc.parallelize(0 to 3e3.toInt, 2).map(n => (n % 2, n)).groupByKey.count 16 | ``` 17 | 18 | Use 2 executors at least so you can kill one and keep the application up and running (on one executor). 19 | 20 | ``` 21 | YARN_CONF_DIR=hadoop-conf ./bin/spark-shell --master yarn \ 22 | -c spark.shuffle.service.enabled=true \ 23 | --num-executors 2 24 | ``` 25 | -------------------------------------------------------------------------------- /docs/exercises/spark-exercise-pairrddfunctions-oneliners.md: -------------------------------------------------------------------------------- 1 | == Exercise: One-liners using PairRDDFunctions 2 | 3 | This is a set of one-liners to give you a entry point into using rdd:PairRDDFunctions.md[PairRDDFunctions]. 4 | 5 | === Exercise 6 | 7 | How would you go about solving a requirement to pair elements of the same key and creating a new RDD out of the matched values? 8 | 9 | [source, scala] 10 | ---- 11 | val users = Seq((1, "user1"), (1, "user2"), (2, "user1"), (2, "user3"), (3,"user2"), (3,"user4"), (3,"user1")) 12 | 13 | // Input RDD 14 | val us = sc.parallelize(users) 15 | 16 | // ...your code here 17 | 18 | // Desired output 19 | Seq("user1","user2"),("user1","user3"),("user1","user4"),("user2","user4")) 20 | ---- 21 | -------------------------------------------------------------------------------- /docs/exercises/spark-hello-world-using-spark-shell.md: -------------------------------------------------------------------------------- 1 | == Exercise: Spark's Hello World using Spark shell and Scala 2 | 3 | Run Spark shell and count the number of words in a file using MapReduce pattern. 4 | 5 | * Use `sc.textFile` to read the file into memory 6 | * Use `RDD.flatMap` for a mapper step 7 | * Use `reduceByKey` for a reducer step 8 | -------------------------------------------------------------------------------- /docs/external-shuffle-service/ExecutorShuffleInfo.md: -------------------------------------------------------------------------------- 1 | # ExecutorShuffleInfo 2 | 3 | `ExecutorShuffleInfo` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/external-shuffle-service/index.md: -------------------------------------------------------------------------------- 1 | # External Shuffle Service 2 | 3 | **External Shuffle Service** is a Spark service to serve RDD and shuffle blocks outside and for [Executor](../executor/Executor.md)s. 4 | 5 | [ExternalShuffleService](ExternalShuffleService.md) can be started as a [command-line application](ExternalShuffleService.md#launch) or automatically as part of a worker node in a Spark cluster (e.g. [Spark Standalone]({{ book.spark_standalone }}/Worker)). 6 | 7 | External Shuffle Service is enabled in a Spark application using [spark.shuffle.service.enabled](configuration-properties.md#spark.shuffle.service.enabled) configuration property. 8 | -------------------------------------------------------------------------------- /docs/features/index.md: -------------------------------------------------------------------------------- 1 | # Features -------------------------------------------------------------------------------- /docs/history-server/EventLogFileWriter.md: -------------------------------------------------------------------------------- 1 | # EventLogFileWriter 2 | 3 | `EventLogFileWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/history-server/JsonProtocol.md: -------------------------------------------------------------------------------- 1 | # JsonProtocol Utility 2 | 3 | `JsonProtocol` is an utility to convert [SparkListenerEvent](../SparkListenerEvent.md)s [to](#sparkEventToJson) and [from JSON format](#sparkEventFromJson). 4 | 5 | ## ObjectMapper 6 | 7 | `JsonProtocol` uses an Jackson Databind [ObjectMapper]({{ jackson.api }}/com/fasterxml/jackson/databind/ObjectMapper.html) for performing conversions to and from JSON. 8 | 9 | ## Converting Spark Event to JSON 10 | 11 | ```scala 12 | sparkEventToJson( 13 | event: SparkListenerEvent): JValue 14 | ``` 15 | 16 | `sparkEventToJson` converts the given [SparkListenerEvent](../SparkListenerEvent.md) to JSON format. 17 | 18 | `sparkEventToJson` is used when...FIXME 19 | 20 | ## Converting JSON to Spark Event 21 | 22 | ```scala 23 | sparkEventFromJson( 24 | json: JValue): SparkListenerEvent 25 | ``` 26 | 27 | `sparkEventFromJson` converts a JSON-encoded event to a [SparkListenerEvent](../SparkListenerEvent.md). 28 | 29 | `sparkEventFromJson` is used when...FIXME 30 | -------------------------------------------------------------------------------- /docs/images/CoarseGrainedExecutorBackend-reviveOffers.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/CoarseGrainedExecutorBackend-reviveOffers.png -------------------------------------------------------------------------------- /docs/images/CoarseGrainedScheduler-rpc-endpoint.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/CoarseGrainedScheduler-rpc-endpoint.png -------------------------------------------------------------------------------- /docs/images/CoarseGrainedSchedulerBackend-DriverEndpoint-CoarseGrainedExecutorBackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/CoarseGrainedSchedulerBackend-DriverEndpoint-CoarseGrainedExecutorBackend.png -------------------------------------------------------------------------------- /docs/images/CoarseGrainedSchedulerBackend-RegisterExecutor-event.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/CoarseGrainedSchedulerBackend-RegisterExecutor-event.png -------------------------------------------------------------------------------- /docs/images/LocalSchedulerBackend-LocalEndpoint-Executor-task-status-updates.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/LocalSchedulerBackend-LocalEndpoint-Executor-task-status-updates.png -------------------------------------------------------------------------------- /docs/images/ShuffleMapTask-runTask.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/ShuffleMapTask-runTask.png -------------------------------------------------------------------------------- /docs/images/SparkDeploySchedulerBackend-AppClient-start.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/SparkDeploySchedulerBackend-AppClient-start.png -------------------------------------------------------------------------------- /docs/images/TaskSetManager-TaskSchedulerImpl-TaskSet.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/TaskSetManager-TaskSchedulerImpl-TaskSet.png -------------------------------------------------------------------------------- /docs/images/TaskSetManager-handleFailedTask.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/TaskSetManager-handleFailedTask.png -------------------------------------------------------------------------------- /docs/images/core/AppStatusStore-createLiveStore.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/core/AppStatusStore-createLiveStore.png -------------------------------------------------------------------------------- /docs/images/core/BroadcastManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/core/BroadcastManager.png -------------------------------------------------------------------------------- /docs/images/core/ContextCleaner.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/core/ContextCleaner.png -------------------------------------------------------------------------------- /docs/images/core/sparkenv-driver-blockmanager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/core/sparkenv-driver-blockmanager.png -------------------------------------------------------------------------------- /docs/images/core/sparkenv-executor-blockmanager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/core/sparkenv-executor-blockmanager.png -------------------------------------------------------------------------------- /docs/images/diagrams/rdd-saveas.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/rdd-saveas.png -------------------------------------------------------------------------------- /docs/images/diagrams/rdd-sparkcontext-transformations-action.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/rdd-sparkcontext-transformations-action.png -------------------------------------------------------------------------------- /docs/images/diagrams/spark-local-architecture.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/spark-local-architecture.png -------------------------------------------------------------------------------- /docs/images/diagrams/spark-platform.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/spark-platform.png -------------------------------------------------------------------------------- /docs/images/diagrams/spark-submit-master-workers.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/spark-submit-master-workers.png -------------------------------------------------------------------------------- /docs/images/diagrams/sparkcontext-createtaskscheduler.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/sparkcontext-createtaskscheduler.png -------------------------------------------------------------------------------- /docs/images/diagrams/sparkcontext-rdds.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/sparkcontext-rdds.png -------------------------------------------------------------------------------- /docs/images/diagrams/sparkcontext-services.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/sparkcontext-services.png -------------------------------------------------------------------------------- /docs/images/diagrams/taskscheduler-single-sparkcontext.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/diagrams/taskscheduler-single-sparkcontext.png -------------------------------------------------------------------------------- /docs/images/driver-sparkcontext-clustermanager-workers-executors.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/driver-sparkcontext-clustermanager-workers-executors.png -------------------------------------------------------------------------------- /docs/images/executor/CoarseGrainedExecutorBackend-statusUpdate.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/CoarseGrainedExecutorBackend-statusUpdate.png -------------------------------------------------------------------------------- /docs/images/executor/CoarseGrainedExecutorBackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/CoarseGrainedExecutorBackend.png -------------------------------------------------------------------------------- /docs/images/executor/ExecutorBackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/ExecutorBackend.png -------------------------------------------------------------------------------- /docs/images/executor/HeartbeatReceiver-Heartbeat.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/HeartbeatReceiver-Heartbeat.png -------------------------------------------------------------------------------- /docs/images/executor/TaskRunner.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/TaskRunner.png -------------------------------------------------------------------------------- /docs/images/executor/executor-heartbeatReceiver-endpoint.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/executor-heartbeatReceiver-endpoint.png -------------------------------------------------------------------------------- /docs/images/executor/executor-taskrunner-executorbackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/executor-taskrunner-executorbackend.png -------------------------------------------------------------------------------- /docs/images/executor/spark-executorsource-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/spark-executorsource-jconsole.png -------------------------------------------------------------------------------- /docs/images/executor/spark-standalone-webui-memory-per-node.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/executor/spark-standalone-webui-memory-per-node.png -------------------------------------------------------------------------------- /docs/images/history-server/spark-history-server-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/history-server/spark-history-server-webui.png -------------------------------------------------------------------------------- /docs/images/memory/MemoryManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/memory/MemoryManager.png -------------------------------------------------------------------------------- /docs/images/memory/TaskMemoryManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/memory/TaskMemoryManager.png -------------------------------------------------------------------------------- /docs/images/mesos-console-slaves.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/mesos-console-slaves.png -------------------------------------------------------------------------------- /docs/images/mesos-console.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/mesos-console.png -------------------------------------------------------------------------------- /docs/images/rdd-lineage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/rdd-lineage.png -------------------------------------------------------------------------------- /docs/images/rpc/rpcenv-endpoints.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/rpc/rpcenv-endpoints.png -------------------------------------------------------------------------------- /docs/images/scheduler/DAGScheduler-MapOutputTrackerMaster-containsShuffle.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/DAGScheduler-MapOutputTrackerMaster-containsShuffle.png -------------------------------------------------------------------------------- /docs/images/scheduler/ShuffleMapTask.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/ShuffleMapTask.png -------------------------------------------------------------------------------- /docs/images/scheduler/TaskSchedulerImpl-initialize.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/TaskSchedulerImpl-initialize.png -------------------------------------------------------------------------------- /docs/images/scheduler/TaskSchedulerImpl-resourceOffers-internal-structures.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/TaskSchedulerImpl-resourceOffers-internal-structures.png -------------------------------------------------------------------------------- /docs/images/scheduler/TaskSchedulerImpl-resourceOffers-rootPool-getSortedTaskSetQueue.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/TaskSchedulerImpl-resourceOffers-rootPool-getSortedTaskSetQueue.png -------------------------------------------------------------------------------- /docs/images/scheduler/action-job.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/action-job.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-handleExecutorLost.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-handleExecutorLost.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-handleJobSubmitted.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-handleJobSubmitted.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-job-resultstage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-job-resultstage.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-new-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-new-instance.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-rdd-lineage-stage-dag.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-rdd-lineage-stage-dag.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-rdd-partitions-job-resultstage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-rdd-partitions-job-resultstage.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-resultstage-partitions.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-resultstage-partitions.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-stages.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-submitjob.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-submitjob.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-tasksetmanager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-tasksetmanager.png -------------------------------------------------------------------------------- /docs/images/scheduler/dagscheduler-webui-skipped-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/dagscheduler-webui-skipped-stages.png -------------------------------------------------------------------------------- /docs/images/scheduler/job-stage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/job-stage.png -------------------------------------------------------------------------------- /docs/images/scheduler/rdd-job-partitions.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/rdd-job-partitions.png -------------------------------------------------------------------------------- /docs/images/scheduler/resultstage-findMissingPartitions.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/resultstage-findMissingPartitions.png -------------------------------------------------------------------------------- /docs/images/scheduler/scheduler-handlemapstagesubmitted.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/scheduler-handlemapstagesubmitted.png -------------------------------------------------------------------------------- /docs/images/scheduler/scheduler-job-shuffles-result-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/scheduler-job-shuffles-result-stages.png -------------------------------------------------------------------------------- /docs/images/scheduler/scheduler-job-splits-into-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/scheduler-job-splits-into-stages.png -------------------------------------------------------------------------------- /docs/images/scheduler/spark-DAGScheduler-getShuffleDependencies.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/spark-DAGScheduler-getShuffleDependencies.png -------------------------------------------------------------------------------- /docs/images/scheduler/spark-rdd-partitions-job-stage-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/spark-rdd-partitions-job-stage-tasks.png -------------------------------------------------------------------------------- /docs/images/scheduler/spark-sparklistener-event-senders.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/spark-sparklistener-event-senders.png -------------------------------------------------------------------------------- /docs/images/scheduler/sparkstandalone-sparkcontext-taskscheduler-schedulerbackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/sparkstandalone-sparkcontext-taskscheduler-schedulerbackend.png -------------------------------------------------------------------------------- /docs/images/scheduler/stage-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/stage-tasks.png -------------------------------------------------------------------------------- /docs/images/scheduler/taskscheduler-resourceOffers.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/taskscheduler-resourceOffers.png -------------------------------------------------------------------------------- /docs/images/scheduler/taskscheduler-uses-schedulerbackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/taskscheduler-uses-schedulerbackend.png -------------------------------------------------------------------------------- /docs/images/scheduler/taskschedulerImpl-submitTasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/taskschedulerImpl-submitTasks.png -------------------------------------------------------------------------------- /docs/images/scheduler/taskschedulerimpl-sparkcontext-schedulerbackend-dagscheduler.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/taskschedulerimpl-sparkcontext-schedulerbackend-dagscheduler.png -------------------------------------------------------------------------------- /docs/images/scheduler/taskschedulerimpl-start-standalone.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/taskschedulerimpl-start-standalone.png -------------------------------------------------------------------------------- /docs/images/scheduler/taskschedulerimpl-tasksetmanager-tasksetfinished.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/scheduler/taskschedulerimpl-tasksetmanager-tasksetfinished.png -------------------------------------------------------------------------------- /docs/images/shuffle/BypassMergeSortShuffleWriter-write.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/BypassMergeSortShuffleWriter-write.png -------------------------------------------------------------------------------- /docs/images/shuffle/IndexShuffleBlockResolver-SortShuffleManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/IndexShuffleBlockResolver-SortShuffleManager.png -------------------------------------------------------------------------------- /docs/images/shuffle/IndexShuffleBlockResolver-writeIndexFileAndCommit.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/IndexShuffleBlockResolver-writeIndexFileAndCommit.png -------------------------------------------------------------------------------- /docs/images/shuffle/ShuffleExternalSorter.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/ShuffleExternalSorter.png -------------------------------------------------------------------------------- /docs/images/shuffle/ShuffleInMemorySorter.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/ShuffleInMemorySorter.png -------------------------------------------------------------------------------- /docs/images/shuffle/SortShuffleManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/SortShuffleManager.png -------------------------------------------------------------------------------- /docs/images/shuffle/UnsafeShuffleWriter-ShuffleExternalSorter.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/UnsafeShuffleWriter-ShuffleExternalSorter.png -------------------------------------------------------------------------------- /docs/images/shuffle/UnsafeShuffleWriter.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/shuffle/UnsafeShuffleWriter.png -------------------------------------------------------------------------------- /docs/images/spark-SQLTab-creating-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-SQLTab-creating-instance.png -------------------------------------------------------------------------------- /docs/images/spark-YarnSchedulerBackend-doRequestTotalExecutors.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-YarnSchedulerBackend-doRequestTotalExecutors.png -------------------------------------------------------------------------------- /docs/images/spark-YarnSchedulerEndpoint-RequestExecutors.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-YarnSchedulerEndpoint-RequestExecutors.png -------------------------------------------------------------------------------- /docs/images/spark-broadcast-webui-executors-rdd-blocks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-broadcast-webui-executors-rdd-blocks.png -------------------------------------------------------------------------------- /docs/images/spark-driver.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-driver.png -------------------------------------------------------------------------------- /docs/images/spark-mesos-completed-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-mesos-completed-tasks.png -------------------------------------------------------------------------------- /docs/images/spark-mesos-frameworks-one-active.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-mesos-frameworks-one-active.png -------------------------------------------------------------------------------- /docs/images/spark-mesos.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-mesos.png -------------------------------------------------------------------------------- /docs/images/spark-metrics-MetricsSystem-driver.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-metrics-MetricsSystem-driver.png -------------------------------------------------------------------------------- /docs/images/spark-metrics-MetricsSystem.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-metrics-MetricsSystem.png -------------------------------------------------------------------------------- /docs/images/spark-metrics-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-metrics-jconsole.png -------------------------------------------------------------------------------- /docs/images/spark-mllib-pipeline.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-mllib-pipeline.png -------------------------------------------------------------------------------- /docs/images/spark-partitions-ui-stages-2-partitions.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-partitions-ui-stages-2-partitions.png -------------------------------------------------------------------------------- /docs/images/spark-partitions-ui-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-partitions-ui-stages.png -------------------------------------------------------------------------------- /docs/images/spark-platform.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-platform.png -------------------------------------------------------------------------------- /docs/images/spark-pool-FairSchedulingAlgorithm.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-pool-FairSchedulingAlgorithm.png -------------------------------------------------------------------------------- /docs/images/spark-pull-request-tasks-progress.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-pull-request-tasks-progress.png -------------------------------------------------------------------------------- /docs/images/spark-rdd-partitioned-distributed.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-rdd-partitioned-distributed.png -------------------------------------------------------------------------------- /docs/images/spark-rdds.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-rdds.png -------------------------------------------------------------------------------- /docs/images/spark-runjob.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-runjob.png -------------------------------------------------------------------------------- /docs/images/spark-shuffle-join-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-shuffle-join-webui.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-console-master-only.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-console-master-only.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-console-one-worker.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-console-one-worker.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-console-two-workers-alive.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-console-two-workers-alive.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-console-worker-dead.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-console-worker-dead.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-console-workers-alive-and-dead.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-console-workers-alive-and-dead.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-master-worker-LaunchDriver.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-master-worker-LaunchDriver.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-webui-appid-notfound.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-webui-appid-notfound.png -------------------------------------------------------------------------------- /docs/images/spark-standalone-webui-executor-summary.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-standalone-webui-executor-summary.png -------------------------------------------------------------------------------- /docs/images/spark-thriftserver-squirrel-addalias.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-thriftserver-squirrel-addalias.png -------------------------------------------------------------------------------- /docs/images/spark-thriftserver-squirrel-adddriver.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-thriftserver-squirrel-adddriver.png -------------------------------------------------------------------------------- /docs/images/spark-thriftserver-squirrel-metadata.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-thriftserver-squirrel-metadata.png -------------------------------------------------------------------------------- /docs/images/spark-thriftserver-squirrel-show-tables.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-thriftserver-squirrel-show-tables.png -------------------------------------------------------------------------------- /docs/images/spark-thriftserver-squirrel.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-thriftserver-squirrel.png -------------------------------------------------------------------------------- /docs/images/spark-thriftserver-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-thriftserver-webui.png -------------------------------------------------------------------------------- /docs/images/spark-transformations-zipWithIndex-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-transformations-zipWithIndex-webui.png -------------------------------------------------------------------------------- /docs/images/spark-ui-rdd-name.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-ui-rdd-name.png -------------------------------------------------------------------------------- /docs/images/spark-ui-repartition-2.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-ui-repartition-2.png -------------------------------------------------------------------------------- /docs/images/spark-ui-storage-hundred-ints.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/spark-ui-storage-hundred-ints.png -------------------------------------------------------------------------------- /docs/images/sparkapp-sparkcontext-master-slaves.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkapp-sparkcontext-master-slaves.png -------------------------------------------------------------------------------- /docs/images/sparkcontext-broadcast-bittorrent-newBroadcast.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkcontext-broadcast-bittorrent-newBroadcast.png -------------------------------------------------------------------------------- /docs/images/sparkcontext-broadcast-bittorrent.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkcontext-broadcast-bittorrent.png -------------------------------------------------------------------------------- /docs/images/sparkcontext-broadcast-executors.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkcontext-broadcast-executors.png -------------------------------------------------------------------------------- /docs/images/sparkcontext-broadcastmanager-contextcleaner.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkcontext-broadcastmanager-contextcleaner.png -------------------------------------------------------------------------------- /docs/images/sparkenv-driver.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkenv-driver.png -------------------------------------------------------------------------------- /docs/images/sparkenv-executor.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkenv-executor.png -------------------------------------------------------------------------------- /docs/images/sparkui-stages-locality-level.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkui-stages-locality-level.png -------------------------------------------------------------------------------- /docs/images/sparkui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/sparkui.png -------------------------------------------------------------------------------- /docs/images/storage/BlockInfoManager-BlockManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/BlockInfoManager-BlockManager.png -------------------------------------------------------------------------------- /docs/images/storage/BlockManager-SparkEnv.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/BlockManager-SparkEnv.png -------------------------------------------------------------------------------- /docs/images/storage/BlockManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/BlockManager.png -------------------------------------------------------------------------------- /docs/images/storage/BlockManagerMaster-RegisterBlockManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/BlockManagerMaster-RegisterBlockManager.png -------------------------------------------------------------------------------- /docs/images/storage/DiskBlockManager-BlockManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/DiskBlockManager-BlockManager.png -------------------------------------------------------------------------------- /docs/images/storage/DiskStore-BlockManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/DiskStore-BlockManager.png -------------------------------------------------------------------------------- /docs/images/storage/MemoryStore-BlockManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/MemoryStore-BlockManager.png -------------------------------------------------------------------------------- /docs/images/storage/NettyBlockRpcServer.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/NettyBlockRpcServer.png -------------------------------------------------------------------------------- /docs/images/storage/NettyBlockTransferService.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/NettyBlockTransferService.png -------------------------------------------------------------------------------- /docs/images/storage/ShuffleMetricsSource.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/ShuffleMetricsSource.png -------------------------------------------------------------------------------- /docs/images/storage/demo-DiskBlockManager-and-Block-Data-webui-storage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/demo-DiskBlockManager-and-Block-Data-webui-storage.png -------------------------------------------------------------------------------- /docs/images/storage/spark-MemoryStore.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/storage/spark-MemoryStore.png -------------------------------------------------------------------------------- /docs/images/taskscheduler-submitTasks-local-mode.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/taskscheduler-submitTasks-local-mode.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-SparkUI.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-SparkUI.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-accumulators.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-accumulators.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-active-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-active-stages.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-environment.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-environment.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-executors.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-executors.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-details-for-job-active-pending-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-details-for-job-active-pending-stages.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-details-for-job-four-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-details-for-job-four-stages.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-details-for-job-no-job.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-details-for-job-no-job.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-details-for-job.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-details-for-job.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-event-timeline.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-event-timeline.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-status-section.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-status-section.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-summary-section.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-summary-section.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-timeline-executors.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-timeline-executors.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs-timeline-popup.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs-timeline-popup.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-jobs.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-jobs.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-pool-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-pool-details.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-pool-summary.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-pool-summary.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-sql-execution-graph.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-sql-execution-graph.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-sql-no-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-sql-no-details-for-query.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-sql.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-sql.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-accumulators.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-accumulators.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-aggregated-metrics-by-executor.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-aggregated-metrics-by-executor.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-dagvisualization.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-dagvisualization.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-details.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-eventtimeline.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-eventtimeline.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-header.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-header.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-summary-metrics-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-summary-metrics-tasks.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stage-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stage-tasks.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stages-alljobs.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stages-alljobs.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stages-completed.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stages-completed.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stages-empty.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stages-empty.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stages-fairschedulerpools.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stages-fairschedulerpools.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-stages.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-stages.png -------------------------------------------------------------------------------- /docs/images/webui/spark-webui-storage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/webui/spark-webui-storage.png -------------------------------------------------------------------------------- /docs/images/yarn-YarnSchedulerBackend.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/docs/images/yarn-YarnSchedulerBackend.png -------------------------------------------------------------------------------- /docs/local-properties.md: -------------------------------------------------------------------------------- 1 | # Local Properties 2 | 3 | `SparkContext.setLocalProperty` lets you set key-value pairs that will be propagated down to tasks and can be accessed there using [TaskContext.getLocalProperty](scheduler/TaskContext.md#getLocalProperty). 4 | 5 | ## Creating Logical Job Groups 6 | 7 | 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. 8 | 9 | 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. 10 | 11 | ```scala 12 | val data = sc.parallelize(0 to 9) 13 | 14 | sc.setLocalProperty("spark.scheduler.pool", "myPool") 15 | 16 | // these two jobs (one per action) will run in the myPool pool 17 | data.count 18 | data.collect 19 | 20 | sc.setLocalProperty("spark.scheduler.pool", null) 21 | 22 | // this job will run in the default pool 23 | data.count 24 | ``` 25 | -------------------------------------------------------------------------------- /docs/master.md: -------------------------------------------------------------------------------- 1 | == Master 2 | 3 | A *master* is a running Spark instance that connects to a cluster manager for resources. 4 | 5 | The master acquires cluster nodes to run executors. 6 | 7 | CAUTION: FIXME Add it to the Spark architecture figure above. 8 | -------------------------------------------------------------------------------- /docs/memory/ExecutionMemoryPool.md: -------------------------------------------------------------------------------- 1 | # ExecutionMemoryPool 2 | 3 | `ExecutionMemoryPool` is a [MemoryPool](MemoryPool.md). 4 | 5 | ## Creating Instance 6 | 7 | `ExecutionMemoryPool` takes the following to be created: 8 | 9 | * Lock Object 10 | * `MemoryMode` (`ON_HEAP` or `OFF_HEAP`) 11 | 12 | `ExecutionMemoryPool` is created when: 13 | 14 | * `MemoryManager` is created (and initializes [on-heap](MemoryManager.md#onHeapExecutionMemoryPool) and [off-heap](MemoryManager.md#offHeapExecutionMemoryPool) execution memory pools) 15 | 16 | ## Acquiring Memory 17 | 18 | ```scala 19 | acquireMemory( 20 | numBytes: Long, 21 | taskAttemptId: Long, 22 | maybeGrowPool: Long => Unit = (additionalSpaceNeeded: Long) => (), 23 | computeMaxPoolSize: () => Long = () => poolSize): Long 24 | ``` 25 | 26 | `acquireMemory`...FIXME 27 | 28 | `acquireMemory` is used when: 29 | 30 | * `UnifiedMemoryManager` is requested to [acquire execution memory](UnifiedMemoryManager.md#acquireExecutionMemory) 31 | -------------------------------------------------------------------------------- /docs/memory/UnsafeInMemorySorter.md: -------------------------------------------------------------------------------- 1 | # UnsafeInMemorySorter 2 | 3 | ## Creating Instance 4 | 5 | `UnsafeInMemorySorter` takes the following to be created: 6 | 7 | * [MemoryConsumer](MemoryConsumer.md) 8 | * [TaskMemoryManager](TaskMemoryManager.md) 9 | * `RecordComparator` 10 | * `PrefixComparator` 11 | * Long Array or Size 12 | * `canUseRadixSort` flag 13 | 14 | `UnsafeInMemorySorter` is created when: 15 | 16 | * `UnsafeExternalSorter` is [created](UnsafeExternalSorter.md#inMemSorter) 17 | * `UnsafeKVExternalSorter` is created 18 | -------------------------------------------------------------------------------- /docs/memory/UnsafeSorterSpillReader.md: -------------------------------------------------------------------------------- 1 | = UnsafeSorterSpillReader 2 | 3 | *UnsafeSorterSpillReader* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/memory/UnsafeSorterSpillWriter.md: -------------------------------------------------------------------------------- 1 | = [[UnsafeSorterSpillWriter]] UnsafeSorterSpillWriter 2 | 3 | *UnsafeSorterSpillWriter* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/memory/index.md: -------------------------------------------------------------------------------- 1 | # Memory System 2 | 3 | **Memory System** is a core component of Apache Spark that is based on [UnifiedMemoryManager](UnifiedMemoryManager.md). 4 | 5 | ## Resources 6 | 7 | * [SPARK-10000: Consolidate storage and execution memory management](https://issues.apache.org/jira/browse/SPARK-10000) 8 | 9 | ### Videos 10 | 11 | * [Deep Dive: Apache Spark Memory Management](https://youtu.be/dPHrykZL8Cg) 12 | * [Deep Dive into Project Tungsten](https://youtu.be/5ajs8EIPWGI) 13 | * [Spark Performance: What's Next](https://youtu.be/JX0CdOTWYX4) 14 | -------------------------------------------------------------------------------- /docs/metrics/JvmSource.md: -------------------------------------------------------------------------------- 1 | # JvmSource 2 | 3 | `JvmSource` is a [metrics source](Source.md). 4 | 5 | 6 | The name of the source is **jvm**. 7 | 8 | `JvmSource` registers the build-in Codahale metrics: 9 | 10 | * `GarbageCollectorMetricSet` 11 | * `MemoryUsageGaugeSet` 12 | * `BufferPoolMetricSet` 13 | 14 | Among the metrics is **total.committed** (from `MemoryUsageGaugeSet`) that describes the current usage of the heap and non-heap memories. 15 | -------------------------------------------------------------------------------- /docs/metrics/Sink.md: -------------------------------------------------------------------------------- 1 | # Sink 2 | 3 | `Sink` is a <> of *metrics sinks*. 4 | 5 | [[contract]] 6 | [source, scala] 7 | ---- 8 | package org.apache.spark.metrics.sink 9 | 10 | trait Sink { 11 | def start(): Unit 12 | def stop(): Unit 13 | def report(): Unit 14 | } 15 | ---- 16 | 17 | NOTE: `Sink` is a `private[spark]` contract. 18 | 19 | .Sink Contract 20 | [cols="1,2",options="header",width="100%"] 21 | |=== 22 | | Method 23 | | Description 24 | 25 | | `start` 26 | | [[start]] Used when...FIXME 27 | 28 | | `stop` 29 | | [[stop]] Used when...FIXME 30 | 31 | | `report` 32 | | [[report]] Used when...FIXME 33 | |=== 34 | 35 | [[implementations]] 36 | .Sinks 37 | [cols="1,2",options="header",width="100%"] 38 | |=== 39 | | Sink 40 | | Description 41 | 42 | | `ConsoleSink` 43 | | [[ConsoleSink]] 44 | 45 | | `CsvSink` 46 | | [[CsvSink]] 47 | 48 | | `GraphiteSink` 49 | | [[GraphiteSink]] 50 | 51 | | `JmxSink` 52 | | [[JmxSink]] 53 | 54 | | spark-metrics-MetricsServlet.md[MetricsServlet] 55 | | [[MetricsServlet]] 56 | 57 | | `Slf4jSink` 58 | | [[Slf4jSink]] 59 | 60 | | `StatsdSink` 61 | | [[StatsdSink]] 62 | |=== 63 | 64 | NOTE: All known <> in Spark 2.3 are in `org.apache.spark.metrics.sink` Scala package. 65 | -------------------------------------------------------------------------------- /docs/network/index.md: -------------------------------------------------------------------------------- 1 | # Network 2 | -------------------------------------------------------------------------------- /docs/plugins/DriverPlugin.md: -------------------------------------------------------------------------------- 1 | # DriverPlugin 2 | 3 | `DriverPlugin` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/plugins/ExecutorPlugin.md: -------------------------------------------------------------------------------- 1 | # ExecutorPlugin 2 | 3 | `ExecutorPlugin` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/plugins/PluginContextImpl.md: -------------------------------------------------------------------------------- 1 | # PluginContextImpl 2 | 3 | `PluginContextImpl` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/plugins/SparkPlugin.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # SparkPlugin 7 | 8 | `SparkPlugin` is an [abstraction](#contract) of custom extensions for Spark applications. 9 | 10 | ## Contract 11 | 12 | ###  Driver-side Component 13 | 14 | ```java 15 | DriverPlugin driverPlugin() 16 | ``` 17 | 18 | Used when: 19 | 20 | * `DriverPluginContainer` is [created](DriverPluginContainer.md#driverPlugins) 21 | 22 | ###  Executor-side Component 23 | 24 | ```java 25 | ExecutorPlugin executorPlugin() 26 | ``` 27 | 28 | Used when: 29 | 30 | * `ExecutorPluginContainer` is [created](ExecutorPluginContainer.md#executorPlugins) 31 | -------------------------------------------------------------------------------- /docs/plugins/index.md: -------------------------------------------------------------------------------- 1 | # Plugin Framework 2 | 3 | **Plugin Framework** is an API for registering custom extensions (_plugins_) to be executed on the driver and executors. 4 | 5 | Plugin Framework uses separate [PluginContainer](PluginContainer.md)s for the driver and executors, and [spark.plugins](../configuration-properties.md#spark.plugins) configuration property for [SparkPlugin](SparkPlugin.md)s to be registered. 6 | 7 | Plugin Framework was introduced in [Spark 2.4.4](https://issues.apache.org/jira/browse/SPARK-24918) (with an API for executors) with further changes in [Spark 3.0.0](https://issues.apache.org/jira/browse/SPARK-29396) (to cover the driver). 8 | 9 | ## Resources 10 | 11 | * [Advanced Instrumentation](https://spark.apache.org/docs/latest/monitoring.html#advanced-instrumentation) in the official documentation of Apache Spark 12 | * [Commit for SPARK-29397](https://github.com/apache/spark/commit/d51d228048d519a9a666f48dc532625de13e7587) 13 | * [Spark Plugin Framework in 3.0 - Part 1: Introduction](http://blog.madhukaraphatak.com/spark-plugin-part-1/) by Madhukara Phatak 14 | * [Spark Memory Monitor](https://github.com/squito/spark-memory) by squito 15 | * [SparkPlugins](https://github.com/cerndb/SparkPlugins) by Luca Canali (CERN) 16 | -------------------------------------------------------------------------------- /docs/push-based-shuffle.md: -------------------------------------------------------------------------------- 1 | # Push-Based Shuffle 2 | 3 | **Push-Based Shuffle** is a new feature of Apache Spark 3.2.0 (cf. [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602)) to improve shuffle efficiency. 4 | 5 | Push-based shuffle is enabled using [spark.shuffle.push.enabled](configuration-properties.md#spark.shuffle.push.enabled) configuration property and [can only be used](Utils.md#isPushBasedShuffleEnabled) in a Spark application submitted to YARN cluster manager, with external shuffle service enabled, IO encryption disabled, and relocation of serialized objects supported. 6 | -------------------------------------------------------------------------------- /docs/rdd/AsyncRDDActions.md: -------------------------------------------------------------------------------- 1 | # AsyncRDDActions 2 | 3 | `AsyncRDDActions` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/rdd/CheckpointRDD.md: -------------------------------------------------------------------------------- 1 | # CheckpointRDD 2 | 3 | `CheckpointRDD` is an extension of the [RDD](RDD.md) abstraction for [RDDs](#implementations) that recovers checkpointed data from storage. 4 | 5 | `CheckpointRDD` cannot be checkpointed again (and [doCheckpoint](RDD.md#doCheckpoint), [checkpoint](RDD.md#checkpoint), and [localCheckpoint](RDD.md#localCheckpoint) are simply noops). 6 | 7 | [getPartitions](RDD.md#getPartitions) and [compute](RDD.md#compute) throw an `NotImplementedError` and are supposed to be overriden by the [implementations](#implementations). 8 | 9 | ## Implementations 10 | 11 | * [LocalCheckpointRDD](LocalCheckpointRDD.md) 12 | * [ReliableCheckpointRDD](ReliableCheckpointRDD.md) 13 | -------------------------------------------------------------------------------- /docs/rdd/CoGroupedRDD.md: -------------------------------------------------------------------------------- 1 | # CoGroupedRDD 2 | 3 | `CoGroupedRDD[K]` is an [RDD](RDD.md) that cogroups the [parent RDDs](#rdds). 4 | 5 | ```scala 6 | RDD[(K, Array[Iterable[_]])] 7 | ``` 8 | 9 | For each key `k` in parent RDDs, the resulting RDD contains a tuple with the list of values for that key. 10 | 11 | ## Creating Instance 12 | 13 | `CoGroupedRDD` takes the following to be created: 14 | 15 | * Key-Value [RDD](RDD.md)s (`Seq[RDD[_ <: Product2[K, _]]]`) 16 | * [Partitioner](Partitioner.md) 17 | 18 | `CoGroupedRDD` is created when: 19 | 20 | * [RDD.cogroup](PairRDDFunctions.md#cogroup) operator is used 21 | -------------------------------------------------------------------------------- /docs/rdd/CoalescedRDD.md: -------------------------------------------------------------------------------- 1 | # CoalescedRDD 2 | 3 | `CoalescedRDD` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/rdd/NewHadoopRDD.md: -------------------------------------------------------------------------------- 1 | == [[NewHadoopRDD]] NewHadoopRDD 2 | 3 | `NewHadoopRDD` is an rdd:index.md[RDD] of `K` keys and `V` values. 4 | 5 | <> when: 6 | 7 | * `SparkContext.newAPIHadoopFile` 8 | * `SparkContext.newAPIHadoopRDD` 9 | * (indirectly) `SparkContext.binaryFiles` 10 | * (indirectly) `SparkContext.wholeTextFiles` 11 | 12 | NOTE: `NewHadoopRDD` is the base RDD of `BinaryFileRDD` and `WholeTextFileRDD`. 13 | 14 | === [[getPreferredLocations]] `getPreferredLocations` Method 15 | 16 | CAUTION: FIXME 17 | 18 | === [[creating-instance]] Creating NewHadoopRDD Instance 19 | 20 | `NewHadoopRDD` takes the following when created: 21 | 22 | * [[sc]] SparkContext.md[] 23 | * [[inputFormatClass]] HDFS' `InputFormat[K, V]` 24 | * [[keyClass]] `K` class name 25 | * [[valueClass]] `V` class name 26 | * [[_conf]] transient HDFS' `Configuration` 27 | 28 | `NewHadoopRDD` initializes the <>. 29 | -------------------------------------------------------------------------------- /docs/rdd/ParallelCollectionRDD.md: -------------------------------------------------------------------------------- 1 | # ParallelCollectionRDD 2 | 3 | `ParallelCollectionRDD` is an RDD of a collection of elements with `numSlices` partitions and optional `locationPrefs`. 4 | 5 | `ParallelCollectionRDD` is the result of `SparkContext.parallelize` and `SparkContext.makeRDD` methods. 6 | 7 | The data collection is split on to `numSlices` slices. 8 | 9 | It uses `ParallelCollectionPartition`. 10 | -------------------------------------------------------------------------------- /docs/rdd/Partition.md: -------------------------------------------------------------------------------- 1 | # Partition 2 | 3 | `Partition` is a <> of a <> of a RDD. 4 | 5 | NOTE: A partition is *missing* when it has not be computed yet. 6 | 7 | [[contract]] 8 | [[index]] 9 | `Partition` is identified by an *partition index* that is a unique identifier of a partition of a RDD. 10 | 11 | [source, scala] 12 | ---- 13 | index: Int 14 | ---- 15 | -------------------------------------------------------------------------------- /docs/rdd/Partitioner.md: -------------------------------------------------------------------------------- 1 | # Partitioner 2 | 3 | `Partitioner` is an [abstraction](#contract) of [partitioners](#implementations) that define how the elements in a key-value pair RDD are partitioned by key. 4 | 5 | `Partitioner` [maps keys to partition IDs](#getPartition) (from 0 to [numPartitions](#numPartitions) exclusive). 6 | 7 | `Partitioner` ensures that records with the same key are in the same partition. 8 | 9 | `Partitioner` is a Java `Serializable`. 10 | 11 | ## Contract 12 | 13 | ###  Partition for Key 14 | 15 | ```scala 16 | getPartition( 17 | key: Any): Int 18 | ``` 19 | 20 | Partition ID for the given key 21 | 22 | ###  Number of Partitions 23 | 24 | ```scala 25 | numPartitions: Int 26 | ``` 27 | 28 | ## Implementations 29 | 30 | * [HashPartitioner](HashPartitioner.md) 31 | * [RangePartitioner](RangePartitioner.md) 32 | -------------------------------------------------------------------------------- /docs/rdd/RDDOperationScope.md: -------------------------------------------------------------------------------- 1 | # RDDOperationScope 2 | 3 | ## withScope { #withScope } 4 | 5 | ```scala 6 | withScope[T]( 7 | sc: SparkContext, 8 | name: String, 9 | allowNesting: Boolean, 10 | ignoreParent: Boolean)( 11 | body: => T): T 12 | withScope[T]( 13 | sc: SparkContext, 14 | allowNesting: Boolean = false)( 15 | body: => T): T 16 | ``` 17 | 18 | ??? note "name Argument" 19 | Value | Caller 20 | ------|------- 21 | `checkpoint` | [RDD.doCheckpoint](RDD.md#doCheckpoint) 22 | _Some_ method name | Executed without `name` 23 | The name of a physical operator (with no `Exec` suffix) | `SparkPlan.executeQuery` ([Spark SQL]({{ book.spark_sql }}/physical-operators/SparkPlan/#executeQuery)) 24 | 25 | `withScope`...FIXME 26 | 27 | --- 28 | 29 | `withScope` is used when: 30 | 31 | * `RDD` is requested to [doCheckpoint](RDD.md#doCheckpoint) and [withScope](RDD.md#withScope) (for most, if not all, `RDD` API operators) 32 | * `SparkContext` is requested to [withScope](../SparkContext.md#withScope) (for most, if not all, `SparkContext` API operators) 33 | * `SparkPlan` ([Spark SQL]({{ book.spark_sql }}/physical-operators/SparkPlan/#executeQuery)) is requested to `executeQuery` 34 | -------------------------------------------------------------------------------- /docs/rdd/ZippedPartitionsWithEvaluatorRDD.md: -------------------------------------------------------------------------------- 1 | # ZippedPartitionsWithEvaluatorRDD 2 | 3 | `ZippedPartitionsWithEvaluatorRDD` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/rdd/spark-rdd-operations.md: -------------------------------------------------------------------------------- 1 | == Operators - Transformations and Actions 2 | 3 | RDDs have two types of operations: spark-rdd-transformations.md[transformations] and spark-rdd-actions.md[actions]. 4 | 5 | NOTE: Operators are also called *operations*. 6 | 7 | === Gotchas - things to watch for 8 | 9 | Even if you don't access it explicitly it cannot be referenced inside a closure as it is serialized and carried around across executors. 10 | 11 | See https://issues.apache.org/jira/browse/SPARK-5063 12 | -------------------------------------------------------------------------------- /docs/rpc/NettyRpcEnv.md: -------------------------------------------------------------------------------- 1 | # NettyRpcEnv 2 | 3 | `NettyRpcEnv` is an [RpcEnv](RpcEnv.md) that uses [Netty](https://netty.io/) (_"an asynchronous event-driven network application framework for rapid development of maintainable high performance protocol servers & clients"_). 4 | 5 | ## Creating Instance 6 | 7 | `NettyRpcEnv` takes the following to be created: 8 | 9 | * [SparkConf](../SparkConf.md) 10 | * [JavaSerializerInstance](../serializer/JavaSerializerInstance.md) 11 | * Host Name 12 | * `SecurityManager` 13 | * Number of CPU Cores 14 | 15 | `NettyRpcEnv` is created when: 16 | 17 | * `NettyRpcEnvFactory` is requested to [create an RpcEnv](NettyRpcEnvFactory.md#create) 18 | -------------------------------------------------------------------------------- /docs/rpc/RpcAddress.md: -------------------------------------------------------------------------------- 1 | # RpcAddress 2 | 3 | `RpcAddress` is a logical address of an RPC system, with hostname and port. 4 | 5 | `RpcAddress` can be encoded as a **Spark URL** in the format of `spark://host:port`. 6 | 7 | ## Creating Instance 8 | 9 | `RpcAddress` takes the following to be created: 10 | 11 | * Host 12 | * Port 13 | 14 | ## Creating RpcAddress based on Spark URL 15 | 16 | ```scala 17 | fromSparkURL( 18 | sparkUrl: String): RpcAddress 19 | ``` 20 | 21 | `fromSparkURL` [extract a host and a port](../Utils.md#extractHostPortFromSparkUrl) from the input Spark URL and creates an [RpcAddress](#creating-instance). 22 | 23 | `fromSparkURL` is used when: 24 | 25 | * `StandaloneAppClient` (Spark Standalone) is created 26 | * `ClientApp` (Spark Standalone) is requested to `start` 27 | * `Worker` (Spark Standalone) is requested to `startRpcEnvAndEndpoint` 28 | -------------------------------------------------------------------------------- /docs/rpc/RpcEndpointAddress.md: -------------------------------------------------------------------------------- 1 | = RpcEndpointAddress 2 | 3 | *RpcEndpointAddress* is a logical address of an endpoint in an RPC system, with <> and *name*. 4 | 5 | RpcEndpointAddress is in the format of `spark://[name]@[rpcAddress.host]:[rpcAddress.port]`. 6 | -------------------------------------------------------------------------------- /docs/rpc/RpcEndpointRef.md: -------------------------------------------------------------------------------- 1 | # RpcEndpointRef 2 | 3 | `RpcEndpointRef` is a reference to a rpc:RpcEndpoint.md[RpcEndpoint] in a rpc:index.md[RpcEnv]. 4 | 5 | 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). 6 | 7 | A RpcEndpointRef has <> (a Spark URL), and a name. 8 | 9 | You can send asynchronous one-way messages to the corresponding RpcEndpoint using <> method. 10 | 11 | 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. 12 | 13 | * `spark.rpc.numRetries` (default: `3`) - the number of times to retry connection attempts. 14 | * `spark.rpc.retry.wait` (default: `3s`) - the number of milliseconds to wait on each retry. 15 | 16 | It also uses rpc:index.md#endpoint-lookup-timeout[lookup timeouts]. 17 | 18 | == [[send]] send Method 19 | 20 | CAUTION: FIXME 21 | 22 | == [[askWithRetry]] askWithRetry Method 23 | 24 | CAUTION: FIXME 25 | -------------------------------------------------------------------------------- /docs/rpc/RpcEnvFactory.md: -------------------------------------------------------------------------------- 1 | = RpcEnvFactory 2 | 3 | RpcEnvFactory is an abstraction of <> to <>. 4 | 5 | == [[implementations]] Available RpcEnvFactories 6 | 7 | rpc:NettyRpcEnvFactory.md[] is the default and only known RpcEnvFactory in Apache Spark (as of https://github.com/apache/spark/commit/4f5a24d7e73104771f233af041eeba4f41675974[this commit]). 8 | 9 | == [[create]] Creating RpcEnv 10 | 11 | [source,scala] 12 | ---- 13 | create( 14 | config: RpcEnvConfig): RpcEnv 15 | ---- 16 | 17 | create is used when RpcEnv utility is requested to rpc:RpcEnv.md#create[create an RpcEnv]. 18 | -------------------------------------------------------------------------------- /docs/rpc/RpcEnvFileServer.md: -------------------------------------------------------------------------------- 1 | = RpcEnvFileServer 2 | 3 | *RpcEnvFileServer* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/rpc/index.md: -------------------------------------------------------------------------------- 1 | # RPC System 2 | 3 | **RPC System** is a communication system of Spark services. 4 | 5 | The main abstractions are [RpcEnv](RpcEnv.md) and [RpcEndpoint](RpcEndpoint.md). 6 | 7 | ![RpcEnvironment with RpcEndpoints and RpcEndpointRefs](../images/rpc/rpcenv-endpoints.png) 8 | -------------------------------------------------------------------------------- /docs/scheduler/BlacklistTracker.md: -------------------------------------------------------------------------------- 1 | # BlacklistTracker 2 | 3 | `BlacklistTracker` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/scheduler/CompressedMapStatus.md: -------------------------------------------------------------------------------- 1 | # CompressedMapStatus 2 | 3 | `CompressedMapStatus` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/scheduler/DAGSchedulerSource.md: -------------------------------------------------------------------------------- 1 | # DAGSchedulerSource 2 | 3 | `DAGSchedulerSource` is the [metrics source](../metrics/Source.md) of [DAGScheduler](DAGScheduler.md#DAGSchedulerSource). 4 | 5 | The name of the source is **DAGScheduler**. 6 | 7 | `DAGSchedulerSource` emits the following metrics: 8 | 9 | * **stage.failedStages** - the number of failed stages 10 | * **stage.runningStages** - the number of running stages 11 | * **stage.waitingStages** - the number of waiting stages 12 | * **job.allJobs** - the number of all jobs 13 | * **job.activeJobs** - the number of active jobs 14 | -------------------------------------------------------------------------------- /docs/scheduler/FIFOSchedulableBuilder.md: -------------------------------------------------------------------------------- 1 | == FIFOSchedulableBuilder - SchedulableBuilder for FIFO Scheduling Mode 2 | 3 | `FIFOSchedulableBuilder` is a <> that holds a single spark-scheduler-Pool.md[Pool] (that is given when <>). 4 | 5 | NOTE: `FIFOSchedulableBuilder` is the scheduler:TaskSchedulerImpl.md#creating-instance[default `SchedulableBuilder` for `TaskSchedulerImpl`]. 6 | 7 | NOTE: When `FIFOSchedulableBuilder` is created, the `TaskSchedulerImpl` passes its own `rootPool` (a part of scheduler:TaskScheduler.md#contract[TaskScheduler Contract]). 8 | 9 | `FIFOSchedulableBuilder` obeys the <> as follows: 10 | 11 | * <> does nothing. 12 | * `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. 13 | 14 | === [[creating-instance]] Creating FIFOSchedulableBuilder Instance 15 | 16 | `FIFOSchedulableBuilder` takes the following when created: 17 | 18 | * [[rootPool]] `rootPool` spark-scheduler-Pool.md[Pool] 19 | -------------------------------------------------------------------------------- /docs/scheduler/HighlyCompressedMapStatus.md: -------------------------------------------------------------------------------- 1 | # HighlyCompressedMapStatus 2 | 3 | `HighlyCompressedMapStatus` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/scheduler/JobListener.md: -------------------------------------------------------------------------------- 1 | # JobListener 2 | 3 | `JobListener` is an [abstraction](#contract) of [listeners](#implementations) that listen for [job completion](#taskSucceeded) or [failure](#jobFailed) events (after submitting a job to the [DAGScheduler](DAGScheduler.md)). 4 | 5 | ## Contract 6 | 7 | ###  taskSucceeded 8 | 9 | ```scala 10 | taskSucceeded( 11 | index: Int, 12 | result: Any): Unit 13 | ``` 14 | 15 | Used when `DAGScheduler` is requested to [handleTaskCompletion](DAGScheduler.md#handleTaskCompletion) or [markMapStageJobAsFinished](DAGScheduler.md#markMapStageJobAsFinished) 16 | 17 | ###  jobFailed 18 | 19 | ```scala 20 | jobFailed( 21 | exception: Exception): Unit 22 | ``` 23 | 24 | Used when `DAGScheduler` is requested to [cleanUpAfterSchedulerStop](DAGScheduler.md#cleanUpAfterSchedulerStop), [handleJobSubmitted](DAGScheduler.md#handleJobSubmitted), [handleMapStageSubmitted](DAGScheduler.md#handleMapStageSubmitted), [handleTaskCompletion](DAGScheduler.md#handleTaskCompletion) or [failJobAndIndependentStages](DAGScheduler.md#failJobAndIndependentStages) 25 | 26 | ## Implementations 27 | 28 | * ApproximateActionListener 29 | * [JobWaiter](JobWaiter.md) 30 | -------------------------------------------------------------------------------- /docs/scheduler/MapOutputStatistics.md: -------------------------------------------------------------------------------- 1 | # MapOutputStatistics 2 | 3 | `MapOutputStatistics` holds statistics about the [output partition sizes](#bytesByPartitionId) in a [map stage](#shuffleId). 4 | 5 | `MapOutputStatistics` is the result of executing the following (currently internal APIs): 6 | 7 | * `SparkContext` is requested to [submitMapStage](../SparkContext.md#submitMapStage) 8 | * `DAGScheduler` is requested to [submitMapStage](DAGScheduler.md#submitMapStage) 9 | 10 | ## Creating Instance 11 | 12 | `MapOutputStatistics` takes the following to be created: 13 | 14 | * Shuffle Id (of a [ShuffleDependency](../rdd/ShuffleDependency.md)) 15 | * Output Partition Sizes (`Array[Long]`) 16 | 17 | `MapOutputStatistics` is created when: 18 | 19 | * `MapOutputTrackerMaster` is requested for the [statistics (of a ShuffleDependency)](MapOutputTrackerMaster.md#getStatistics) 20 | -------------------------------------------------------------------------------- /docs/scheduler/TaskSetBlacklist.md: -------------------------------------------------------------------------------- 1 | == [[TaskSetBlacklist]] `TaskSetBlacklist` -- Blacklisting Executors and Nodes For TaskSet 2 | 3 | CAUTION: FIXME 4 | 5 | === [[updateBlacklistForFailedTask]] `updateBlacklistForFailedTask` Method 6 | 7 | CAUTION: FIXME 8 | 9 | === [[isExecutorBlacklistedForTaskSet]] `isExecutorBlacklistedForTaskSet` Method 10 | 11 | CAUTION: FIXME 12 | 13 | === [[isNodeBlacklistedForTaskSet]] `isNodeBlacklistedForTaskSet` Method 14 | 15 | CAUTION: FIXME 16 | -------------------------------------------------------------------------------- /docs/serializer/JavaSerializerInstance.md: -------------------------------------------------------------------------------- 1 | # JavaSerializerInstance 2 | 3 | `JavaSerializerInstance` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/serializer/index.md: -------------------------------------------------------------------------------- 1 | # Serialization System 2 | 3 | **Serialization System** is a core component of Apache Spark with pluggable serializers for task closures and block data. 4 | 5 | Serialization System uses [SerializerManager](SerializerManager.md) to select the [Serializer](Serializer.md) (based on [spark.serializer](../configuration-properties.md#spark.serializer) configuration property). 6 | -------------------------------------------------------------------------------- /docs/shuffle/DownloadFileManager.md: -------------------------------------------------------------------------------- 1 | # DownloadFileManager 2 | 3 | `DownloadFileManager` is an [abstraction](#contract) of [file managers](#implementations) that can [createTempFile](#createTempFile) and [registerTempFileToClean](#registerTempFileToClean). 4 | 5 | ## Contract 6 | 7 | ###  createTempFile 8 | 9 | ```java 10 | DownloadFile createTempFile( 11 | TransportConf transportConf) 12 | ``` 13 | 14 | Used when: 15 | 16 | * `DownloadCallback` (of [OneForOneBlockFetcher](../storage/OneForOneBlockFetcher.md)) is created 17 | 18 | ###  registerTempFileToClean 19 | 20 | ```java 21 | boolean registerTempFileToClean( 22 | DownloadFile file) 23 | ``` 24 | 25 | Used when: 26 | 27 | * `DownloadCallback` (of [OneForOneBlockFetcher](../storage/OneForOneBlockFetcher.md)) is requested to `onComplete` 28 | 29 | ## Implementations 30 | 31 | * RemoteBlockDownloadFileManager 32 | * [ShuffleBlockFetcherIterator](../storage/ShuffleBlockFetcherIterator.md) 33 | -------------------------------------------------------------------------------- /docs/shuffle/ExecutorDiskUtils.md: -------------------------------------------------------------------------------- 1 | # ExecutorDiskUtils 2 | -------------------------------------------------------------------------------- /docs/shuffle/LocalDiskShuffleDataIO.md: -------------------------------------------------------------------------------- 1 | # LocalDiskShuffleDataIO 2 | 3 | `LocalDiskShuffleDataIO` is a [ShuffleDataIO](ShuffleDataIO.md). 4 | 5 | ## ShuffleExecutorComponents 6 | 7 | ```java 8 | ShuffleExecutorComponents executor() 9 | ``` 10 | 11 | `executor` is part of the [ShuffleDataIO](ShuffleDataIO.md#executor) abstraction. 12 | 13 | `executor` creates a new [LocalDiskShuffleExecutorComponents](LocalDiskShuffleExecutorComponents.md). 14 | -------------------------------------------------------------------------------- /docs/shuffle/LocalDiskShuffleExecutorComponents.md: -------------------------------------------------------------------------------- 1 | # LocalDiskShuffleExecutorComponents 2 | 3 | `LocalDiskShuffleExecutorComponents` is a [ShuffleExecutorComponents](ShuffleExecutorComponents.md). 4 | 5 | ## Creating Instance 6 | 7 | `LocalDiskShuffleExecutorComponents` takes the following to be created: 8 | 9 | * [SparkConf](../SparkConf.md) 10 | 11 | `LocalDiskShuffleExecutorComponents` is created when: 12 | 13 | * `LocalDiskShuffleDataIO` is requested for a [ShuffleExecutorComponents](LocalDiskShuffleDataIO.md#executor) 14 | -------------------------------------------------------------------------------- /docs/shuffle/LocalDiskShuffleMapOutputWriter.md: -------------------------------------------------------------------------------- 1 | # LocalDiskShuffleMapOutputWriter 2 | 3 | `LocalDiskShuffleMapOutputWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/shuffle/LocalDiskSingleSpillMapOutputWriter.md: -------------------------------------------------------------------------------- 1 | # LocalDiskSingleSpillMapOutputWriter 2 | 3 | `LocalDiskSingleSpillMapOutputWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/shuffle/SerializedShuffleHandle.md: -------------------------------------------------------------------------------- 1 | # SerializedShuffleHandle 2 | 3 | `SerializedShuffleHandle` is a [BaseShuffleHandle](BaseShuffleHandle.md) that `SortShuffleManager` uses when [canUseSerializedShuffle](SortShuffleWriter.md#canUseSerializedShuffle) (when requested to [register a shuffle](SortShuffleManager.md#registerShuffle) and [BypassMergeSortShuffleHandles](BypassMergeSortShuffleHandle.md) could not be selected). 4 | 5 | `SerializedShuffleHandle` tells `SortShuffleManager` to use [UnsafeShuffleWriter](UnsafeShuffleWriter.md) when requested for a [ShuffleWriter](SortShuffleManager.md#getWriter). 6 | 7 | ## Creating Instance 8 | 9 | `SerializedShuffleHandle` takes the following to be created: 10 | 11 | * Shuffle ID 12 | * [ShuffleDependency](../rdd/ShuffleDependency.md) 13 | 14 | `SerializedShuffleHandle` is created when: 15 | 16 | * `SortShuffleManager` is requested for a [ShuffleHandle](SortShuffleManager.md#registerShuffle) (for the [ShuffleDependency](#dependency)) 17 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleBlockPusher.md: -------------------------------------------------------------------------------- 1 | # ShuffleBlockPusher 2 | 3 | `ShuffleBlockPusher` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleDataIO.md: -------------------------------------------------------------------------------- 1 | # ShuffleDataIO 2 | 3 | `ShuffleDataIO` is an [abstraction](#contract) of pluggable [temporary shuffle block store plugins](#implementations) for storing shuffle blocks in arbitrary storage backends. 4 | 5 | ## Contract 6 | 7 | ###  ShuffleDriverComponents 8 | 9 | ```java 10 | ShuffleDriverComponents driver() 11 | ``` 12 | 13 | Used when: 14 | 15 | * `SparkContext` is [created](../SparkContext.md#shuffleDriverComponents) 16 | 17 | ###  ShuffleExecutorComponents 18 | 19 | ```java 20 | ShuffleExecutorComponents executor() 21 | ``` 22 | 23 | Used when: 24 | 25 | * `SortShuffleManager` utility is used to [load the ShuffleExecutorComponents](SortShuffleManager.md#loadShuffleExecutorComponents) 26 | 27 | ## Implementations 28 | 29 | * [LocalDiskShuffleDataIO](LocalDiskShuffleDataIO.md) 30 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleDataIOUtils.md: -------------------------------------------------------------------------------- 1 | # ShuffleDataIOUtils 2 | 3 | ## Loading ShuffleDataIO 4 | 5 | ```scala 6 | loadShuffleDataIO( 7 | conf: SparkConf): ShuffleDataIO 8 | ``` 9 | 10 | `loadShuffleDataIO` uses the [spark.shuffle.sort.io.plugin.class](../configuration-properties.md#spark.shuffle.sort.io.plugin.class) configuration property to load the [ShuffleDataIO](ShuffleDataIO.md). 11 | 12 | `loadShuffleDataIO` is used when: 13 | 14 | * `SparkContext` is [created](../SparkContext.md#shuffleDriverComponents) 15 | * `SortShuffleManager` utility is used to [loadShuffleExecutorComponents](SortShuffleManager.md#loadShuffleExecutorComponents) 16 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleDriverComponents.md: -------------------------------------------------------------------------------- 1 | # ShuffleDriverComponents 2 | 3 | `ShuffleDriverComponents` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleHandle.md: -------------------------------------------------------------------------------- 1 | # ShuffleHandle 2 | 3 | `ShuffleHandle` is an abstraction of [shuffle handles](#implementations) for [ShuffleManager](ShuffleManager.md) to pass information about shuffles to tasks. 4 | 5 | `ShuffleHandle` is `Serializable` ([Java]({{ java.api }}/java/io/Serializable)). 6 | 7 | ## Implementations 8 | 9 | * [BaseShuffleHandle](BaseShuffleHandle.md) 10 | 11 | ## Creating Instance 12 | 13 | `ShuffleHandle` takes the following to be created: 14 | 15 | * Shuffle ID 16 | 17 | !!! note "Abstract Class" 18 | `ShuffleHandle` is an abstract class and cannot be created directly. It is created indirectly for the [concrete ShuffleHandles](#implementations). 19 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleMapOutputWriter.md: -------------------------------------------------------------------------------- 1 | # ShuffleMapOutputWriter 2 | 3 | `ShuffleMapOutputWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/shuffle/ShuffleReader.md: -------------------------------------------------------------------------------- 1 | # ShuffleReader 2 | 3 | `ShuffleReader` is an [abstraction](#contract) of [shuffle block readers](#implementations) that can [read combined key-value records for a reduce task](#read). 4 | 5 | ## Contract 6 | 7 | ###  Reading Combined Records (for Reduce Task) 8 | 9 | ```scala 10 | read(): Iterator[Product2[K, C]] 11 | ``` 12 | 13 | Used when: 14 | 15 | * [CoGroupedRDD](../rdd/CoGroupedRDD.md#compute), [ShuffledRDD](../rdd/ShuffledRDD.md#compute) are requested to compute a partition (for a `ShuffleDependency` dependency) 16 | * `ShuffledRowRDD` ([Spark SQL]({{ book.spark_sql }}/ShuffledRowRDD)) is requested to `compute` a partition 17 | 18 | ## Implementations 19 | 20 | * [BlockStoreShuffleReader](BlockStoreShuffleReader.md) 21 | -------------------------------------------------------------------------------- /docs/shuffle/SingleSpillShuffleMapOutputWriter.md: -------------------------------------------------------------------------------- 1 | # SingleSpillShuffleMapOutputWriter 2 | 3 | `SingleSpillShuffleMapOutputWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/shuffle/index.md: -------------------------------------------------------------------------------- 1 | # Shuffle System 2 | 3 | **Shuffle System** is a core service of Apache Spark that is responsible for shuffle blocks. 4 | 5 | The main core abstraction is [ShuffleManager](ShuffleManager.md) with [SortShuffleManager](SortShuffleManager.md) as the default and only known implementation. 6 | 7 | [spark.shuffle.manager](../configuration-properties.md#spark.shuffle.manager) configuration property allows for a custom [ShuffleManager](ShuffleManager.md). 8 | 9 | Shuffle System uses shuffle [handles](ShuffleHandle.md), [readers](ShuffleReader.md) and [writers](ShuffleWriter.md). 10 | 11 | ## Resources 12 | 13 | * [Improving Apache Spark Downscaling](https://databricks.com/session_eu19/improving-apache-spark-downscaling) by Christopher Crosbie (Google) Ben Sidhom (Google) 14 | * [Spark shuffle introduction](http://www.slideshare.net/colorant/spark-shuffle-introduction) by Raymond Liu (aka _colorant_) 15 | -------------------------------------------------------------------------------- /docs/spark-debugging.md: -------------------------------------------------------------------------------- 1 | # Debugging Spark 2 | 3 | ## Using spark-shell and IntelliJ IDEA 4 | 5 | Start `spark-shell` with `SPARK_SUBMIT_OPTS` environment variable that configures the JVM's JDWP. 6 | 7 | ```text 8 | SPARK_SUBMIT_OPTS="-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005" ./bin/spark-shell 9 | ``` 10 | 11 | Attach IntelliJ IDEA to the JVM process using **Run > Attach to Local Process** menu. 12 | 13 | ## Using sbt 14 | 15 | 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. 16 | 17 | ```text 18 | $ sbt -jvm-debug 5005 19 | Listening for transport dt_socket at address: 5005 20 | ... 21 | ``` 22 | 23 | Run Spark context and the breakpoints get triggered. 24 | 25 | ```text 26 | scala> val sc = new SparkContext(conf) 27 | 15/11/14 22:58:46 INFO SparkContext: Running Spark version 1.6.0-SNAPSHOT 28 | ``` 29 | 30 | !!! tip 31 | Read [Debugging](https://www.jetbrains.com/help/idea/debugging-code.html) chapter in IntelliJ IDEA's Help. 32 | -------------------------------------------------------------------------------- /docs/stage-level-scheduling/.pages: -------------------------------------------------------------------------------- 1 | title: Stage-Level Scheduling 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/stage-level-scheduling/ExecutorResourceInfo.md: -------------------------------------------------------------------------------- 1 | # ExecutorResourceInfo 2 | 3 | `ExecutorResourceInfo` is a [ResourceAllocator](ResourceAllocator.md). 4 | 5 | ## Creating Instance 6 | 7 | `ExecutorResourceInfo` takes the following to be created: 8 | 9 | * Resource Name 10 | * Addresses 11 | * Number of slots (per address) 12 | 13 | `ExecutorResourceInfo` is created when: 14 | 15 | * `DriverEndpoint` is requested to [handle a RegisterExecutor event](../scheduler/DriverEndpoint.md#RegisterExecutor) 16 | -------------------------------------------------------------------------------- /docs/stage-level-scheduling/ExecutorResourceRequest.md: -------------------------------------------------------------------------------- 1 | # ExecutorResourceRequest 2 | 3 | ## Creating Instance 4 | 5 | `ExecutorResourceRequest` takes the following to be created: 6 | 7 | * Resource Name 8 | * Amount 9 | * Discovery Script 10 | * Vendor 11 | 12 | `ExecutorResourceRequest` is created when: 13 | 14 | * `ExecutorResourceRequests` is requested to [memory](ExecutorResourceRequests.md#memory), [offHeapMemory](ExecutorResourceRequests.md#offHeapMemory), [memoryOverhead](ExecutorResourceRequests.md#memoryOverhead), [pysparkMemory](ExecutorResourceRequests.md#pysparkMemory), [cores](ExecutorResourceRequests.md#cores) and [resource](ExecutorResourceRequests.md#resource) 15 | * `JsonProtocol` utility is used to [executorResourceRequestFromJson](../history-server/JsonProtocol.md#executorResourceRequestFromJson) 16 | 17 | ## Serializable 18 | 19 | `ExecutorResourceRequest` is a `Serializable` ([Java]({{ java.api }}/java/io/Serializable.html)). 20 | -------------------------------------------------------------------------------- /docs/stage-level-scheduling/ResourceID.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # ResourceID 7 | 8 | `ResourceID` is...FIXME 9 | -------------------------------------------------------------------------------- /docs/stage-level-scheduling/TaskResourceProfile.md: -------------------------------------------------------------------------------- 1 | # TaskResourceProfile 2 | 3 | `TaskResourceProfile` is a [ResourceProfile](ResourceProfile.md). 4 | 5 | ## Creating Instance 6 | 7 | `TaskResourceProfile` takes the following to be created: 8 | 9 | * [Task Resources](ResourceProfile.md#taskResources) 10 | 11 | `TaskResourceProfile` is created when: 12 | 13 | * `ResourceProfileBuilder` is requested to [build a ResourceProfile](ResourceProfileBuilder.md#build) 14 | * `DAGScheduler` is requested to [merge ResourceProfiles](../scheduler/DAGScheduler.md#mergeResourceProfiles) 15 | 16 | ## getCustomExecutorResources { #getCustomExecutorResources } 17 | 18 | ??? note "ResourceProfile" 19 | 20 | ```scala 21 | getCustomExecutorResources(): Map[String, ExecutorResourceRequest] 22 | ``` 23 | 24 | `getCustomExecutorResources` is part of the [ResourceProfile](ResourceProfile.md#getCustomExecutorResources) abstraction. 25 | 26 | `getCustomExecutorResources`...FIXME 27 | -------------------------------------------------------------------------------- /docs/stage-level-scheduling/TaskResourceRequest.md: -------------------------------------------------------------------------------- 1 | # TaskResourceRequest 2 | 3 | `TaskResourceRequest` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/status/AppStatusSource.md: -------------------------------------------------------------------------------- 1 | # AppStatusSource 2 | 3 | `AppStatusSource` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/status/LiveEntity.md: -------------------------------------------------------------------------------- 1 | # LiveEntity 2 | 3 | `LiveEntity` is an [abstraction](#contract) of [entities](#implementations) of a running (_live_) Spark application. 4 | 5 | ## Contract 6 | 7 | ###  doUpdate 8 | 9 | ```scala 10 | doUpdate(): Any 11 | ``` 12 | 13 | Updated view of this entity's data 14 | 15 | Used when: 16 | 17 | * `LiveEntity` is requested to [write out to the store](#write) 18 | 19 | ## Implementations 20 | 21 | * LiveExecutionData (Spark SQL) 22 | * LiveExecutionData (Spark Thrift Server) 23 | * LiveExecutor 24 | * LiveExecutorStageSummary 25 | * LiveJob 26 | * LiveRDD 27 | * LiveResourceProfile 28 | * LiveSessionData 29 | * LiveStage 30 | * LiveTask 31 | * SchedulerPool 32 | 33 | ## Writing Out to Store 34 | 35 | ```scala 36 | write( 37 | store: ElementTrackingStore, 38 | now: Long, 39 | checkTriggers: Boolean = false): Unit 40 | ``` 41 | 42 | `write`...FIXME 43 | 44 | `write` is used when: 45 | 46 | * `AppStatusListener` is requested to [update](AppStatusListener.md#update) 47 | * `HiveThriftServer2Listener` (Spark Thrift Server) is requested to `updateStoreWithTriggerEnabled` and `updateLiveStore` 48 | * `SQLAppStatusListener` (Spark SQL) is requested to `update` 49 | -------------------------------------------------------------------------------- /docs/status/index.md: -------------------------------------------------------------------------------- 1 | # Status 2 | 3 | **Status** system uses [AppStatusListener](AppStatusListener.md) to write the state of a Spark application to [AppStatusStore](AppStatusStore.md) for reporting and monitoring: 4 | 5 | * [web UI](../webui/SparkUI.md) 6 | * [REST API](../rest/index.md) 7 | * [Spark History Server](../history-server/index.md) 8 | * [Metrics](../metrics/index.md) 9 | -------------------------------------------------------------------------------- /docs/storage/BlockData.md: -------------------------------------------------------------------------------- 1 | = BlockData 2 | 3 | *BlockData* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/BlockEvictionHandler.md: -------------------------------------------------------------------------------- 1 | # BlockEvictionHandler 2 | 3 | `BlockEvictionHandler` is an [abstraction](#contract) of [block eviction handlers](#implementations) that can [drop blocks from memory](#dropFromMemory). 4 | 5 | ## Contract 6 | 7 | ###  Dropping Block from Memory 8 | 9 | ```scala 10 | dropFromMemory[T: ClassTag]( 11 | blockId: BlockId, 12 | data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel 13 | ``` 14 | 15 | Used when: 16 | 17 | * `MemoryStore` is requested to [evict blocks](MemoryStore.md#evictBlocksToFreeSpace) 18 | 19 | ## Implementations 20 | 21 | * [BlockManager](BlockManager.md) 22 | -------------------------------------------------------------------------------- /docs/storage/BlockManagerDecommissioner.md: -------------------------------------------------------------------------------- 1 | # BlockManagerDecommissioner 2 | 3 | `BlockManagerDecommissioner` is a decommissioning process used by [BlockManager](BlockManager.md#decommissioner). 4 | 5 | ## Creating Instance 6 | 7 | `BlockManagerDecommissioner` takes the following to be created: 8 | 9 | * [SparkConf](../SparkConf.md) 10 | * [BlockManager](BlockManager.md) 11 | 12 | `BlockManagerDecommissioner` is created when: 13 | 14 | * `BlockManager` is requested to [decommissionSelf](BlockManager.md#decommissionSelf) 15 | -------------------------------------------------------------------------------- /docs/storage/BlockManagerId.md: -------------------------------------------------------------------------------- 1 | # BlockManagerId 2 | 3 | BlockManagerId is a unique identifier (_address_) of a [BlockManager](BlockManager.md). 4 | -------------------------------------------------------------------------------- /docs/storage/BlockManagerInfo.md: -------------------------------------------------------------------------------- 1 | = BlockManagerInfo 2 | 3 | *BlockManagerInfo* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/BlockManagerMasterHeartbeatEndpoint.md: -------------------------------------------------------------------------------- 1 | # BlockManagerMasterHeartbeatEndpoint 2 | 3 | `BlockManagerMasterHeartbeatEndpoint` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/BlockManagerStorageEndpoint.md: -------------------------------------------------------------------------------- 1 | # BlockManagerStorageEndpoint 2 | 3 | `BlockManagerStorageEndpoint` is an [IsolatedRpcEndpoint](../rpc/RpcEndpoint.md#IsolatedRpcEndpoint). 4 | 5 | ## Creating Instance 6 | 7 | `BlockManagerStorageEndpoint` takes the following to be created: 8 | 9 | * [RpcEnv](../rpc/RpcEnv.md) 10 | * [BlockManager](BlockManager.md) 11 | * [MapOutputTracker](../scheduler/MapOutputTracker.md) 12 | 13 | `BlockManagerStorageEndpoint` is created when: 14 | 15 | * `BlockManager` is [created](BlockManager.md#storageEndpoint) 16 | 17 | ## Messages 18 | 19 | ### DecommissionBlockManager 20 | 21 | When received, `receiveAndReply` requests the [BlockManager](#blockManager) to [decommissionSelf](BlockManager.md#decommissionSelf). 22 | 23 | `DecommissionBlockManager` is sent out when `BlockManager` is requested to [decommissionBlockManager](BlockManager.md#decommissionBlockManager). 24 | -------------------------------------------------------------------------------- /docs/storage/BlockReplicationPolicy.md: -------------------------------------------------------------------------------- 1 | # BlockReplicationPolicy 2 | 3 | `BlockReplicationPolicy` is...FIXME -------------------------------------------------------------------------------- /docs/storage/ExternalBlockStoreClient.md: -------------------------------------------------------------------------------- 1 | # ExternalBlockStoreClient 2 | 3 | `ExternalBlockStoreClient` is a [BlockStoreClient](BlockStoreClient.md) that the driver and executors use when [spark.shuffle.service.enabled](../external-shuffle-service/configuration-properties.md#spark.shuffle.service.enabled) configuration property is enabled. 4 | 5 | ## Creating Instance 6 | 7 | `ExternalBlockStoreClient` takes the following to be created: 8 | 9 | * [TransportConf](../network/TransportConf.md) 10 | * `SecretKeyHolder` 11 | * `authEnabled` flag 12 | * `registrationTimeoutMs` 13 | 14 | `ExternalBlockStoreClient` is created when: 15 | 16 | * `SparkEnv` utility is requested to [create a SparkEnv](../SparkEnv.md#create) (for the driver and executors) with [spark.shuffle.service.enabled](../external-shuffle-service/configuration-properties.md#spark.shuffle.service.enabled) configuration property enabled 17 | -------------------------------------------------------------------------------- /docs/storage/FallbackStorage.md: -------------------------------------------------------------------------------- 1 | # FallbackStorage 2 | 3 | `FallbackStorage` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/RDDInfo.md: -------------------------------------------------------------------------------- 1 | # RDDInfo 2 | 3 | `RDDInfo` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/RandomBlockReplicationPolicy.md: -------------------------------------------------------------------------------- 1 | # RandomBlockReplicationPolicy 2 | 3 | `RandomBlockReplicationPolicy` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/ShuffleMigrationRunnable.md: -------------------------------------------------------------------------------- 1 | # ShuffleMigrationRunnable 2 | 3 | `ShuffleMigrationRunnable` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/storage/StorageUtils.md: -------------------------------------------------------------------------------- 1 | # StorageUtils 2 | 3 | ## Port of External Shuffle Service 4 | 5 | ```scala 6 | externalShuffleServicePort( 7 | conf: SparkConf): Int 8 | ``` 9 | 10 | `externalShuffleServicePort`...FIXME 11 | 12 | `externalShuffleServicePort` is used when: 13 | 14 | * `BlockManager` is [created](BlockManager.md#externalShuffleServicePort) 15 | * `BlockManagerMasterEndpoint` is [created](BlockManagerMasterEndpoint.md#externalShuffleServicePort) 16 | -------------------------------------------------------------------------------- /docs/storage/index.md: -------------------------------------------------------------------------------- 1 | # Storage System 2 | 3 | **Storage System** is a core component of Apache Spark that uses [BlockManager](BlockManager.md) to manage blocks in [memory](MemoryStore.md) and on [disk](DiskStore.md) (based on [StorageLevel](StorageLevel.md)). 4 | -------------------------------------------------------------------------------- /docs/tips-and-tricks/.pages: -------------------------------------------------------------------------------- 1 | title: Spark's Tips and Tricks 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/tips-and-tricks/access-private-members-spark-shell.md: -------------------------------------------------------------------------------- 1 | # Access private members in Scala in Spark shell 2 | 3 | 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 [REPL: support for package definition](https://issues.scala-lang.org/browse/SI-5299). 4 | 5 | Open `spark-shell` and execute `:paste -raw` that allows you to enter any valid Scala code, including `package`. 6 | 7 | The following snippet shows how to access `private[spark]` member `DAGScheduler.RESUBMIT_TIMEOUT`: 8 | 9 | ```text 10 | scala> :paste -raw 11 | // Entering paste mode (ctrl-D to finish) 12 | 13 | package org.apache.spark 14 | 15 | object spark { 16 | def test = { 17 | import org.apache.spark.scheduler._ 18 | println(DAGScheduler.RESUBMIT_TIMEOUT == 200) 19 | } 20 | } 21 | 22 | scala> spark.test 23 | true 24 | 25 | scala> sc.version 26 | res0: String = 1.6.0-SNAPSHOT 27 | ``` 28 | -------------------------------------------------------------------------------- /docs/tools/AbstractLauncher.md: -------------------------------------------------------------------------------- 1 | # AbstractLauncher 2 | 3 | `AbstractLauncher` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/tools/JavaMainApplication.md: -------------------------------------------------------------------------------- 1 | # JavaMainApplication 2 | 3 | `JavaMainApplication` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/tools/SparkApplication.md: -------------------------------------------------------------------------------- 1 | # SparkApplication 2 | 3 | `SparkApplication` is an [abstraction](#contract) of [entry points](#implementations) to Spark applications that can be [started](#start) (_submitted for execution_ using [spark-submit](spark-submit/index.md)). 4 | 5 | ## Contract 6 | 7 | ###  Starting Spark Application 8 | 9 | ```scala 10 | start( 11 | args: Array[String], conf: SparkConf): Unit 12 | ``` 13 | 14 | Used when: 15 | 16 | * `SparkSubmit` is requested to [submit an application for execution](spark-submit/SparkSubmit.md#runMain) 17 | 18 | ## Implementations 19 | 20 | * `ClientApp` 21 | * [JavaMainApplication](JavaMainApplication.md) 22 | * `KubernetesClientApplication` ([Spark on Kubernetes]({{ book.spark_k8s }}/KubernetesClientApplication)) 23 | * `RestSubmissionClientApp` 24 | * `YarnClusterApplication` 25 | -------------------------------------------------------------------------------- /docs/tools/SparkClassCommandBuilder.md: -------------------------------------------------------------------------------- 1 | # SparkClassCommandBuilder 2 | 3 | `SparkClassCommandBuilder` is an [AbstractCommandBuilder](AbstractCommandBuilder.md). 4 | 5 | ## Creating Instance 6 | 7 | `SparkClassCommandBuilder` takes the following to be created: 8 | 9 | * Class Name 10 | * Class Arguments (`List`) 11 | 12 | `SparkClassCommandBuilder` is created when: 13 | 14 | * `Main` standalone application is [launched](Main.md#main) 15 | -------------------------------------------------------------------------------- /docs/tools/index.md: -------------------------------------------------------------------------------- 1 | # Spark Tools 2 | 3 | Main abstractions: 4 | 5 | * [AbstractCommandBuilder](AbstractCommandBuilder.md) 6 | -------------------------------------------------------------------------------- /docs/tools/spark-submit/.pages: -------------------------------------------------------------------------------- 1 | title: spark-submit 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/tools/spark-submit/SparkSubmitCommandBuilder.OptionParser.md: -------------------------------------------------------------------------------- 1 | # SparkSubmitCommandBuilder.OptionParser 2 | 3 | `SparkSubmitCommandBuilder.OptionParser` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/tools/spark-submit/SparkSubmitUtils.md: -------------------------------------------------------------------------------- 1 | # SparkSubmitUtils 2 | 3 | `SparkSubmitUtils` provides utilities for [SparkSubmit](SparkSubmit.md). 4 | 5 | ## getSubmitOperations 6 | 7 | ```scala 8 | getSubmitOperations( 9 | master: String): SparkSubmitOperation 10 | ``` 11 | 12 | `getSubmitOperations`...FIXME 13 | 14 | `getSubmitOperations` is used when: 15 | 16 | * `SparkSubmit` is requested to [kill a submission](SparkSubmit.md#kill) and [requestStatus](SparkSubmit.md#requestStatus) 17 | -------------------------------------------------------------------------------- /docs/webui/EnvironmentPage.md: -------------------------------------------------------------------------------- 1 | # EnvironmentPage 2 | 3 | ## Review Me 4 | 5 | [[prefix]] 6 | `EnvironmentPage` is a spark-webui-WebUIPage.md[WebUIPage] with an empty spark-webui-WebUIPage.md#prefix[prefix]. 7 | 8 | `EnvironmentPage` is <> exclusively when `EnvironmentTab` is spark-webui-EnvironmentTab.md#creating-instance[created]. 9 | 10 | == [[creating-instance]] Creating EnvironmentPage Instance 11 | 12 | `EnvironmentPage` takes the following when created: 13 | 14 | * [[parent]] Parent spark-webui-EnvironmentTab.md[EnvironmentTab] 15 | * [[conf]] SparkConf.md[SparkConf] 16 | * [[store]] core:AppStatusStore.md[] 17 | -------------------------------------------------------------------------------- /docs/webui/EnvironmentTab.md: -------------------------------------------------------------------------------- 1 | # EnvironmentTab 2 | 3 | ![Environment Tab in Web UI](../images/webui/spark-webui-environment.png) 4 | 5 | ## Review Me 6 | 7 | [[prefix]] 8 | `EnvironmentTab` is a spark-webui-SparkUITab.md[SparkUITab] with *environment* spark-webui-SparkUITab.md#prefix[prefix]. 9 | 10 | `EnvironmentTab` is <> exclusively when `SparkUI` is spark-webui-SparkUI.md#initialize[initialized]. 11 | 12 | [[creating-instance]] 13 | `EnvironmentTab` takes the following when created: 14 | 15 | * [[parent]] Parent spark-webui-SparkUI.md[SparkUI] 16 | * [[store]] core:AppStatusStore.md[] 17 | 18 | When created, `EnvironmentTab` creates the spark-webui-EnvironmentPage.md#creating-instance[EnvironmentPage] page and spark-webui-WebUITab.md#attachPage[attaches] it immediately. 19 | -------------------------------------------------------------------------------- /docs/webui/ExecutorThreadDumpPage.md: -------------------------------------------------------------------------------- 1 | # ExecutorThreadDumpPage 2 | 3 | ## Review Me 4 | 5 | [[prefix]] 6 | `ExecutorThreadDumpPage` is a spark-webui-WebUIPage.md[WebUIPage] with *threadDump* spark-webui-WebUIPage.md#prefix[prefix]. 7 | 8 | `ExecutorThreadDumpPage` is <> exclusively when `ExecutorsTab` is spark-webui-ExecutorsTab.md#creating-instance[created] (with `spark.ui.threadDumpsEnabled` configuration property enabled). 9 | 10 | NOTE: `spark.ui.threadDumpsEnabled` configuration property is enabled (i.e. `true`) by default. 11 | 12 | === [[creating-instance]] Creating ExecutorThreadDumpPage Instance 13 | 14 | `ExecutorThreadDumpPage` takes the following when created: 15 | 16 | * [[parent]] spark-webui-SparkUITab.md[SparkUITab] 17 | * [[sc]] Optional SparkContext.md[] 18 | -------------------------------------------------------------------------------- /docs/webui/ExecutorsPage.md: -------------------------------------------------------------------------------- 1 | # ExecutorsPage 2 | 3 | ## Review Me 4 | 5 | [[prefix]] 6 | `ExecutorsPage` is a spark-webui-WebUIPage.md[WebUIPage] with an empty spark-webui-WebUIPage.md#prefix[prefix]. 7 | 8 | `ExecutorsPage` is <> exclusively when `ExecutorsTab` is spark-webui-ExecutorsTab.md#creating-instance[created]. 9 | 10 | === [[creating-instance]] Creating ExecutorsPage Instance 11 | 12 | `ExecutorsPage` takes the following when created: 13 | 14 | * [[parent]] Parent spark-webui-SparkUITab.md[SparkUITab] 15 | * [[threadDumpEnabled]] `threadDumpEnabled` flag 16 | -------------------------------------------------------------------------------- /docs/webui/ExecutorsTab.md: -------------------------------------------------------------------------------- 1 | # ExecutorsTab 2 | 3 | ![Executors Tab in web UI (local mode)](../images/webui/spark-webui-executors.png) 4 | 5 | ## Review Me 6 | 7 | [[prefix]] 8 | `ExecutorsTab` is a spark-webui-SparkUITab.md[SparkUITab] with *executors* spark-webui-SparkUITab.md#prefix[prefix]. 9 | 10 | `ExecutorsTab` is <> exclusively when `SparkUI` is spark-webui-SparkUI.md#initialize[initialized]. 11 | 12 | [[creating-instance]] 13 | [[parent]] 14 | `ExecutorsTab` takes the parent spark-webui-SparkUI.md[SparkUI] when created. 15 | 16 | When <>, `ExecutorsTab` creates the following pages and spark-webui-WebUITab.md#attachPage[attaches] them immediately: 17 | 18 | * spark-webui-ExecutorsPage.md[ExecutorsPage] 19 | 20 | * spark-webui-ExecutorThreadDumpPage.md[ExecutorThreadDumpPage] 21 | -------------------------------------------------------------------------------- /docs/webui/JobPage.md: -------------------------------------------------------------------------------- 1 | # JobPage 2 | 3 | ## Review Me 4 | 5 | [[prefix]] 6 | `JobPage` is a spark-webui-WebUIPage.md[WebUIPage] with *job* spark-webui-WebUIPage.md#prefix[prefix]. 7 | 8 | `JobPage` is <> exclusively when `JobsTab` is [created](JobsTab.md#creating-instance). 9 | 10 | === [[creating-instance]] Creating JobPage Instance 11 | 12 | `JobPage` takes the following when created: 13 | 14 | * [[parent]] Parent [JobsTab](JobsTab.md) 15 | * [[store]] core:AppStatusStore.md[] 16 | -------------------------------------------------------------------------------- /docs/webui/PrometheusResource.md: -------------------------------------------------------------------------------- 1 | # PrometheusResource 2 | 3 | ## getServletHandler 4 | 5 | ```scala 6 | getServletHandler( 7 | uiRoot: UIRoot): ServletContextHandler 8 | ``` 9 | 10 | `getServletHandler`...FIXME 11 | 12 | `getServletHandler` is used when: 13 | 14 | * `SparkUI` is requested to [initialize](SparkUI.md#initialize) 15 | -------------------------------------------------------------------------------- /docs/webui/RDDPage.md: -------------------------------------------------------------------------------- 1 | == [[RDDPage]] RDDPage 2 | 3 | [[prefix]] 4 | `RDDPage` is a spark-webui-WebUIPage.md[WebUIPage] with *rdd* spark-webui-WebUIPage.md#prefix[prefix]. 5 | 6 | `RDDPage` is <> exclusively when `StorageTab` is spark-webui-StorageTab.md#creating-instance[created]. 7 | 8 | [[creating-instance]] 9 | `RDDPage` takes the following when created: 10 | 11 | * [[parent]] Parent spark-webui-SparkUITab.md[SparkUITab] 12 | * [[store]] core:AppStatusStore.md[] 13 | 14 | === [[render]] `render` Method 15 | 16 | [source, scala] 17 | ---- 18 | render(request: HttpServletRequest): Seq[Node] 19 | ---- 20 | 21 | NOTE: `render` is part of spark-webui-WebUIPage.md#render[WebUIPage Contract] to...FIXME. 22 | 23 | `render`...FIXME 24 | -------------------------------------------------------------------------------- /docs/webui/SparkUITab.md: -------------------------------------------------------------------------------- 1 | # SparkUITab 2 | 3 | `SparkUITab` is an extension of the [WebUITab](WebUITab.md) abstraction for [UI tabs](#implementations) with the [application name](#appName) and [Spark version](#appSparkVersion). 4 | 5 | ## Implementations 6 | 7 | * [EnvironmentTab](EnvironmentTab.md) 8 | * [ExecutorsTab](ExecutorsTab.md) 9 | * [JobsTab](JobsTab.md) 10 | * [StagesTab](StagesTab.md) 11 | * [StorageTab](StorageTab.md) 12 | 13 | ## Creating Instance 14 | 15 | `SparkUITab` takes the following to be created: 16 | 17 | * Parent [SparkUI](SparkUI.md) 18 | * URL Prefix 19 | 20 | ??? note "Abstract Class" 21 | `SparkUITab` is an abstract class and cannot be created directly. It is created indirectly for the [concrete SparkUITabs](#implementations). 22 | 23 | ## Application Name 24 | 25 | ```scala 26 | appName: String 27 | ``` 28 | 29 | `appName` requests the [parent SparkUI](#parent) for the [appName](SparkUI.md#appName). 30 | 31 | ## Spark Version 32 | 33 | ```scala 34 | appSparkVersion: String 35 | ``` 36 | 37 | `appSparkVersion` requests the [parent SparkUI](#parent) for the [appSparkVersion](SparkUI.md#appSparkVersion). 38 | -------------------------------------------------------------------------------- /docs/webui/StorageTab.md: -------------------------------------------------------------------------------- 1 | # StorageTab 2 | 3 | `StorageTab` is a [SparkUITab](SparkUITab.md) with `storage` [URL prefix](SparkUITab.md#prefix). 4 | 5 | ![Storage Tab in Web UI](../images/webui/spark-webui-storage.png) 6 | 7 | ## Creating Instance 8 | 9 | `StorageTab` takes the following to be created: 10 | 11 | * Parent [SparkUI](SparkUI.md) 12 | * [AppStatusStore](../status/AppStatusStore.md) 13 | 14 | `StorageTab` is created when: 15 | 16 | * `SparkUI` is requested to [initialize](SparkUI.md#initialize) 17 | 18 | ## Pages 19 | 20 | When [created](#creating-instance), `StorageTab` [attaches](WebUITab.md#attachPage) the following pages (with a reference to itself and the [AppStatusStore](#store)): 21 | 22 | * [StoragePage](StoragePage.md) 23 | * [RDDPage](RDDPage.md) 24 | -------------------------------------------------------------------------------- /docs/webui/UIUtils.md: -------------------------------------------------------------------------------- 1 | == [[UIUtils]] UIUtils 2 | 3 | `UIUtils` is a utility object for...FIXME 4 | 5 | === [[headerSparkPage]] `headerSparkPage` Method 6 | 7 | [source, scala] 8 | ---- 9 | headerSparkPage( 10 | request: HttpServletRequest, 11 | title: String, 12 | content: => Seq[Node], 13 | activeTab: SparkUITab, 14 | refreshInterval: Option[Int] = None, 15 | helpText: Option[String] = None, 16 | showVisualization: Boolean = false, 17 | useDataTables: Boolean = false): Seq[Node] 18 | ---- 19 | 20 | `headerSparkPage`...FIXME 21 | 22 | NOTE: `headerSparkPage` is used when...FIXME 23 | -------------------------------------------------------------------------------- /docs/webui/index.md: -------------------------------------------------------------------------------- 1 | # Web UIs 2 | 3 | **web UI** is the web interface of Spark applications or infrastructure for monitoring and inspection. 4 | 5 | The main abstraction is [WebUI](WebUI.md). 6 | -------------------------------------------------------------------------------- /graffles/CoarseGrainedExecutorBackend-reviveOffers.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/CoarseGrainedExecutorBackend-reviveOffers.graffle -------------------------------------------------------------------------------- /graffles/CoarseGrainedScheduler-rpc-endpoint.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/CoarseGrainedScheduler-rpc-endpoint.graffle -------------------------------------------------------------------------------- /graffles/CoarseGrainedSchedulerBackend-DriverEndpoint-CoarseGrainedExecutorBackend.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/CoarseGrainedSchedulerBackend-DriverEndpoint-CoarseGrainedExecutorBackend.graffle -------------------------------------------------------------------------------- /graffles/CoarseGrainedSchedulerBackend-RegisterExecutor-event.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/CoarseGrainedSchedulerBackend-RegisterExecutor-event.graffle -------------------------------------------------------------------------------- /graffles/LocalSchedulerBackend-LocalEndpoint-Executor-task-status-updates.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/LocalSchedulerBackend-LocalEndpoint-Executor-task-status-updates.graffle -------------------------------------------------------------------------------- /graffles/SparkDeploySchedulerBackend-AppClient-start.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/SparkDeploySchedulerBackend-AppClient-start.graffle -------------------------------------------------------------------------------- /graffles/TaskSetManager-TaskSchedulerImpl-TaskSet.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/TaskSetManager-TaskSchedulerImpl-TaskSet.graffle -------------------------------------------------------------------------------- /graffles/TaskSetManager-handleFailedTask.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/TaskSetManager-handleFailedTask.graffle -------------------------------------------------------------------------------- /graffles/core/AppStatusStore-createLiveStore.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/AppStatusStore-createLiveStore.graffle -------------------------------------------------------------------------------- /graffles/core/BroadcastManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/BroadcastManager.graffle -------------------------------------------------------------------------------- /graffles/core/ContextCleaner.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/ContextCleaner.graffle -------------------------------------------------------------------------------- /graffles/core/sparkcontext-broadcast-bittorrent-newBroadcast.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/sparkcontext-broadcast-bittorrent-newBroadcast.graffle -------------------------------------------------------------------------------- /graffles/core/sparkcontext-broadcast-bittorrent.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/sparkcontext-broadcast-bittorrent.graffle -------------------------------------------------------------------------------- /graffles/core/sparkenv-driver-blockmanager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/sparkenv-driver-blockmanager.graffle -------------------------------------------------------------------------------- /graffles/core/sparkenv-driver.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/sparkenv-driver.graffle -------------------------------------------------------------------------------- /graffles/core/sparkenv-executor-blockmanager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/sparkenv-executor-blockmanager.graffle -------------------------------------------------------------------------------- /graffles/core/sparkenv-executor.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/core/sparkenv-executor.graffle -------------------------------------------------------------------------------- /graffles/driver-sparkcontext-clustermanager-workers-executors.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/driver-sparkcontext-clustermanager-workers-executors.graffle -------------------------------------------------------------------------------- /graffles/executor/CoarseGrainedExecutorBackend-statusUpdate.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/CoarseGrainedExecutorBackend-statusUpdate.graffle -------------------------------------------------------------------------------- /graffles/executor/CoarseGrainedExecutorBackend.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/CoarseGrainedExecutorBackend.graffle -------------------------------------------------------------------------------- /graffles/executor/ExecutorBackend.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/ExecutorBackend.graffle -------------------------------------------------------------------------------- /graffles/executor/HeartbeatReceiver-Heartbeat.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/HeartbeatReceiver-Heartbeat.graffle -------------------------------------------------------------------------------- /graffles/executor/TaskRunner.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/TaskRunner.graffle -------------------------------------------------------------------------------- /graffles/executor/executor-heartbeatReceiver-endpoint.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/executor-heartbeatReceiver-endpoint.graffle -------------------------------------------------------------------------------- /graffles/executor/executor-taskrunner-executorbackend.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/executor/executor-taskrunner-executorbackend.graffle -------------------------------------------------------------------------------- /graffles/memory/MemoryManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/memory/MemoryManager.graffle -------------------------------------------------------------------------------- /graffles/memory/TaskMemoryManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/memory/TaskMemoryManager.graffle -------------------------------------------------------------------------------- /graffles/rdd-lineage.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/rdd-lineage.graffle -------------------------------------------------------------------------------- /graffles/scheduler/DAGScheduler-MapOutputTrackerMaster-containsShuffle.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/DAGScheduler-MapOutputTrackerMaster-containsShuffle.graffle -------------------------------------------------------------------------------- /graffles/scheduler/ShuffleMapTask-runTask.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/ShuffleMapTask-runTask.graffle -------------------------------------------------------------------------------- /graffles/scheduler/ShuffleMapTask.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/ShuffleMapTask.graffle -------------------------------------------------------------------------------- /graffles/scheduler/TaskSchedulerImpl-initialize.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/TaskSchedulerImpl-initialize.graffle -------------------------------------------------------------------------------- /graffles/scheduler/TaskSchedulerImpl-resourceOffers-internal-structures.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/TaskSchedulerImpl-resourceOffers-internal-structures.graffle -------------------------------------------------------------------------------- /graffles/scheduler/TaskSchedulerImpl-resourceOffers-rootPool-getSortedTaskSetQueue.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/TaskSchedulerImpl-resourceOffers-rootPool-getSortedTaskSetQueue.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-handleExecutorLost.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-handleExecutorLost.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-handleJobSubmitted.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-handleJobSubmitted.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-new-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-new-instance.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-rdd-lineage-stage-dag.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-rdd-lineage-stage-dag.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-rdd-partitions-job-resultstage.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-rdd-partitions-job-resultstage.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-resultstage-partitions.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-resultstage-partitions.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-stages.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-stages.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-submitjob.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-submitjob.graffle -------------------------------------------------------------------------------- /graffles/scheduler/dagscheduler-tasksetmanager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/dagscheduler-tasksetmanager.graffle -------------------------------------------------------------------------------- /graffles/scheduler/resultstage-findMissingPartitions.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/resultstage-findMissingPartitions.graffle -------------------------------------------------------------------------------- /graffles/scheduler/spark-DAGScheduler-getShuffleDependencies.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/spark-DAGScheduler-getShuffleDependencies.graffle -------------------------------------------------------------------------------- /graffles/scheduler/spark-rdd-partitions-job-stage-tasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/spark-rdd-partitions-job-stage-tasks.graffle -------------------------------------------------------------------------------- /graffles/scheduler/spark-sparklistener-event-senders.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/spark-sparklistener-event-senders.graffle -------------------------------------------------------------------------------- /graffles/scheduler/sparkstandalone-sparkcontext-taskscheduler-schedulerbackend.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/sparkstandalone-sparkcontext-taskscheduler-schedulerbackend.graffle -------------------------------------------------------------------------------- /graffles/scheduler/taskscheduler-resourceOffers.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/taskscheduler-resourceOffers.graffle -------------------------------------------------------------------------------- /graffles/scheduler/taskschedulerImpl-submitTasks.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/taskschedulerImpl-submitTasks.graffle -------------------------------------------------------------------------------- /graffles/scheduler/taskschedulerimpl-sparkcontext-schedulerbackend-dagscheduler.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/taskschedulerimpl-sparkcontext-schedulerbackend-dagscheduler.graffle -------------------------------------------------------------------------------- /graffles/scheduler/taskschedulerimpl-start-standalone.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/taskschedulerimpl-start-standalone.graffle -------------------------------------------------------------------------------- /graffles/scheduler/taskschedulerimpl-tasksetmanager-tasksetfinished.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/scheduler/taskschedulerimpl-tasksetmanager-tasksetfinished.graffle -------------------------------------------------------------------------------- /graffles/shuffle/BypassMergeSortShuffleWriter-write.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/BypassMergeSortShuffleWriter-write.graffle -------------------------------------------------------------------------------- /graffles/shuffle/IndexShuffleBlockResolver-SortShuffleManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/IndexShuffleBlockResolver-SortShuffleManager.graffle -------------------------------------------------------------------------------- /graffles/shuffle/IndexShuffleBlockResolver-writeIndexFileAndCommit.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/IndexShuffleBlockResolver-writeIndexFileAndCommit.graffle -------------------------------------------------------------------------------- /graffles/shuffle/ShuffleExternalSorter.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/ShuffleExternalSorter.graffle -------------------------------------------------------------------------------- /graffles/shuffle/ShuffleInMemorySorter.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/ShuffleInMemorySorter.graffle -------------------------------------------------------------------------------- /graffles/shuffle/SortShuffleManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/SortShuffleManager.graffle -------------------------------------------------------------------------------- /graffles/shuffle/UnsafeShuffleWriter-ShuffleExternalSorter.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/UnsafeShuffleWriter-ShuffleExternalSorter.graffle -------------------------------------------------------------------------------- /graffles/shuffle/UnsafeShuffleWriter.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/shuffle/UnsafeShuffleWriter.graffle -------------------------------------------------------------------------------- /graffles/spark-SQLTab-creating-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-SQLTab-creating-instance.graffle -------------------------------------------------------------------------------- /graffles/spark-YarnSchedulerBackend-doRequestTotalExecutors.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-YarnSchedulerBackend-doRequestTotalExecutors.graffle -------------------------------------------------------------------------------- /graffles/spark-YarnSchedulerEndpoint-RequestExecutors.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-YarnSchedulerEndpoint-RequestExecutors.graffle -------------------------------------------------------------------------------- /graffles/spark-driver.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-driver.graffle -------------------------------------------------------------------------------- /graffles/spark-mesos.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-mesos.graffle -------------------------------------------------------------------------------- /graffles/spark-metrics-MetricsSystem-driver.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-metrics-MetricsSystem-driver.graffle -------------------------------------------------------------------------------- /graffles/spark-metrics-MetricsSystem.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-metrics-MetricsSystem.graffle -------------------------------------------------------------------------------- /graffles/spark-mllib-pipeline.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-mllib-pipeline.graffle -------------------------------------------------------------------------------- /graffles/spark-pool-FairSchedulingAlgorithm.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-pool-FairSchedulingAlgorithm.graffle -------------------------------------------------------------------------------- /graffles/spark-runjob.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-runjob.graffle -------------------------------------------------------------------------------- /graffles/spark-standalone-master-worker-LaunchDriver.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-standalone-master-worker-LaunchDriver.graffle -------------------------------------------------------------------------------- /graffles/spark-webui-SparkUI.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-webui-SparkUI.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-ApplicationMaster-client-submitApplication.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-ApplicationMaster-client-submitApplication.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-ApplicationMaster-main.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-ApplicationMaster-main.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-ApplicationMaster-registerAM.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-ApplicationMaster-registerAM.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-ApplicationMaster-runAMEndpoint.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-ApplicationMaster-runAMEndpoint.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-ApplicationMaster.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-ApplicationMaster.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-Client-YarnClient.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-Client-YarnClient.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-ExecutorRunnable.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-ExecutorRunnable.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-YarnAllocator-amClient-ResourceManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-YarnAllocator-amClient-ResourceManager.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-YarnAllocator-runAllocatedContainers.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-YarnAllocator-runAllocatedContainers.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-YarnAllocator.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-YarnAllocator.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-YarnClientSchedulerBackend-start.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-YarnClientSchedulerBackend-start.graffle -------------------------------------------------------------------------------- /graffles/spark-yarn-YarnRMClient-register.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/spark-yarn-YarnRMClient-register.graffle -------------------------------------------------------------------------------- /graffles/sparkapp-sparkcontext-master-slaves.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/sparkapp-sparkcontext-master-slaves.graffle -------------------------------------------------------------------------------- /graffles/sparkcontext-broadcast-executors.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/sparkcontext-broadcast-executors.graffle -------------------------------------------------------------------------------- /graffles/sparkcontext-broadcastmanager-contextcleaner.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/sparkcontext-broadcastmanager-contextcleaner.graffle -------------------------------------------------------------------------------- /graffles/storage/BlockInfoManager-BlockManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/BlockInfoManager-BlockManager.graffle -------------------------------------------------------------------------------- /graffles/storage/BlockManager-SparkEnv.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/BlockManager-SparkEnv.graffle -------------------------------------------------------------------------------- /graffles/storage/BlockManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/BlockManager.graffle -------------------------------------------------------------------------------- /graffles/storage/BlockManagerMaster-RegisterBlockManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/BlockManagerMaster-RegisterBlockManager.graffle -------------------------------------------------------------------------------- /graffles/storage/DiskBlockManager-BlockManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/DiskBlockManager-BlockManager.graffle -------------------------------------------------------------------------------- /graffles/storage/DiskStore-BlockManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/DiskStore-BlockManager.graffle -------------------------------------------------------------------------------- /graffles/storage/MemoryStore-BlockManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/MemoryStore-BlockManager.graffle -------------------------------------------------------------------------------- /graffles/storage/NettyBlockRpcServer.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/NettyBlockRpcServer.graffle -------------------------------------------------------------------------------- /graffles/storage/NettyBlockTransferService.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/NettyBlockTransferService.graffle -------------------------------------------------------------------------------- /graffles/storage/ShuffleMetricsSource.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/ShuffleMetricsSource.graffle -------------------------------------------------------------------------------- /graffles/storage/spark-MemoryStore.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/storage/spark-MemoryStore.graffle -------------------------------------------------------------------------------- /graffles/taskscheduler-submitTasks-local-mode.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/taskscheduler-submitTasks-local-mode.graffle -------------------------------------------------------------------------------- /graffles/yarn-YarnSchedulerBackend.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/graffles/yarn-YarnSchedulerBackend.graffle -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-ALSModelReader.adoc: -------------------------------------------------------------------------------- 1 | == [[ALSModelReader]] ALSModelReader 2 | 3 | `ALSModelReader` is...FIXME 4 | 5 | === [[load]] `load` Method 6 | 7 | [source, scala] 8 | ---- 9 | load(path: String): ALSModel 10 | ---- 11 | 12 | NOTE: `load` is part of link:spark-mllib-MLReader.md#load[MLReader Contract]. 13 | 14 | `load`...FIXME 15 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-Classifier.adoc: -------------------------------------------------------------------------------- 1 | == [[Classifier]] Classifier 2 | 3 | `Classifier` is a link:spark-mllib-Predictor.md[Predictor] that...FIXME 4 | 5 | `Classifier` accepts parameters. 6 | 7 | === [[extractLabeledPoints]] `extractLabeledPoints` Method 8 | 9 | [source, scala] 10 | ---- 11 | extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] 12 | ---- 13 | 14 | `extractLabeledPoints`...FIXME 15 | 16 | NOTE: `extractLabeledPoints` is used when...FIXME 17 | 18 | === [[getNumClasses]] `getNumClasses` Method 19 | 20 | [source, scala] 21 | ---- 22 | getNumClasses(dataset: Dataset[_], maxNumClasses: Int = 100): Int 23 | ---- 24 | 25 | `getNumClasses`...FIXME 26 | 27 | NOTE: `getNumClasses` is used when...FIXME 28 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-CrossValidatorModel.adoc: -------------------------------------------------------------------------------- 1 | == [[CrossValidatorModel]] CrossValidatorModel 2 | 3 | `CrossValidatorModel` is a link:spark-mllib-Model.md[Model] that is <> when `CrossValidator` is requested to link:spark-mllib-CrossValidator.md#fit[find the best model] (per parameters and dataset). 4 | 5 | `CrossValidatorModel` is link:spark-mllib-MLWritable.md[MLWritable], i.e. FIXME 6 | 7 | === [[creating-instance]] Creating CrossValidatorModel Instance 8 | 9 | `CrossValidatorModel` takes the following when created: 10 | 11 | * [[uid]] Unique ID 12 | * [[bestModel]] Best link:spark-mllib-Model.md[Model] 13 | * [[avgMetrics]] Average cross-validation metrics 14 | 15 | `CrossValidatorModel` initializes the <>. 16 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-DecisionTreeClassifier.adoc: -------------------------------------------------------------------------------- 1 | == [[DecisionTreeClassifier]] DecisionTreeClassifier 2 | 3 | `DecisionTreeClassifier` is a probabilistic link:spark-mllib-Classifier.md[Classifier] for...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-HasParallelism.adoc: -------------------------------------------------------------------------------- 1 | == [[HasParallelism]] HasParallelism 2 | 3 | `HasParallelism` is a Scala trait for Spark MLlib components that allow for specifying the level of parallelism for multi-threaded execution and provide a thread-pool-based execution context. 4 | 5 | `HasParallelism` defines <> parameter that controls the number of threads in a cached thread pool. 6 | 7 | .HasParallelism' Parameters 8 | [cols="1,1,2",options="header",width="100%"] 9 | |=== 10 | | Parameter 11 | | Default Value 12 | | Description 13 | 14 | | [[parallelism]] `parallelism` 15 | | `1` 16 | | The number of threads to use when running parallel algorithms 17 | 18 | Must be at least `1`. 19 | |=== 20 | 21 | === [[getExecutionContext]] `getExecutionContext` Method 22 | 23 | [source, scala] 24 | ---- 25 | getExecutionContext: ExecutionContext 26 | ---- 27 | 28 | `getExecutionContext`...FIXME 29 | 30 | NOTE: `getExecutionContext` is used when...FIXME 31 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-Instrumentation.adoc: -------------------------------------------------------------------------------- 1 | == [[Instrumentation]] Instrumentation 2 | 3 | `Instrumentation` is...FIXME 4 | 5 | === [[logParams]] Printing Out Parameters to Logs -- `logParams` Method 6 | 7 | [source, scala] 8 | ---- 9 | logParams(params: Param[_]*): Unit 10 | ---- 11 | 12 | `logParams`...FIXME 13 | 14 | NOTE: `logParams` is used when...FIXME 15 | 16 | === [[create]] Creating Instrumentation -- `create` Method 17 | 18 | [source, scala] 19 | ---- 20 | create[E](estimator: E, dataset: Dataset[_]): Instrumentation[E] 21 | ---- 22 | 23 | `create`...FIXME 24 | 25 | NOTE: `create` is used when...FIXME 26 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-LogisticRegression.adoc: -------------------------------------------------------------------------------- 1 | == [[LogisticRegression]] LogisticRegression 2 | 3 | `LogisticRegression` is...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-MLReader.adoc: -------------------------------------------------------------------------------- 1 | == [[MLReader]] MLReader 2 | 3 | `MLReader` is the <> for...FIXME 4 | 5 | === [[contract]] MLReader Contract 6 | 7 | [source, scala] 8 | ---- 9 | package org.apache.spark.ml.util 10 | 11 | abstract class MLReader[T] { 12 | def load(path: String): T 13 | } 14 | ---- 15 | 16 | .MLReader Contract 17 | [cols="1,2",options="header",width="100%"] 18 | |=== 19 | | Method 20 | | Description 21 | 22 | | [[load]] `load` 23 | | Used when... 24 | |=== 25 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-MLUtils.adoc: -------------------------------------------------------------------------------- 1 | == [[MLUtils]] MLUtils 2 | 3 | `MLUtils` is...FIXME 4 | 5 | === [[kFold]] `kFold` Method 6 | 7 | [source, scala] 8 | ---- 9 | kFold[T](rdd: RDD[T], numFolds: Int, seed: Long): Array[(RDD[T], RDD[T])] 10 | ---- 11 | 12 | `kFold`...FIXME 13 | 14 | NOTE: `kFold` is used when...FIXME 15 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-MLWritable.adoc: -------------------------------------------------------------------------------- 1 | == [[MLWritable]] MLWritable 2 | 3 | `MLWritable` is...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-Model.adoc: -------------------------------------------------------------------------------- 1 | == [[Model]] Model 2 | 3 | `Model` is the <> for a fitted model, i.e. a link:spark-mllib-transformers.md[Transformer] that was produced by an link:spark-mllib-estimators.md[Estimator]. 4 | 5 | === [[contract]] Model Contract 6 | 7 | [source, scala] 8 | ---- 9 | package org.apache.spark.ml 10 | 11 | abstract class Model[M] extends Transformer { 12 | def copy(extra: ParamMap): M 13 | } 14 | ---- 15 | 16 | .Model Contract 17 | [cols="1,2",options="header",width="100%"] 18 | |=== 19 | | Method 20 | | Description 21 | 22 | | [[copy]] `copy` 23 | | Used when... 24 | 25 | | [[parent]] `parent` 26 | | link:spark-mllib-estimators.md[Estimator] that produced this model. 27 | |=== 28 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-MulticlassClassificationEvaluator.adoc: -------------------------------------------------------------------------------- 1 | == [[MulticlassClassificationEvaluator]] MulticlassClassificationEvaluator -- Evaluator of Multiclass Classification Models 2 | 3 | `MulticlassClassificationEvaluator` is an link:spark-mllib-Evaluator.md[Evaluator] that takes datasets with the following two columns: 4 | 5 | * `prediction` (of `DoubleType` values) 6 | * `label` (of `float` or `double` values) 7 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-ParamGridBuilder.adoc: -------------------------------------------------------------------------------- 1 | == [[ParamGridBuilder]] ParamGridBuilder 2 | 3 | `ParamGridBuilder` is...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-Pipeline.adoc: -------------------------------------------------------------------------------- 1 | == [[Pipeline]] Pipeline -- ML Pipeline Component 2 | 3 | `Pipeline` is a ML component in Spark MLlib 2 that...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-PipelineStage.adoc: -------------------------------------------------------------------------------- 1 | == [[PipelineStage]] PipelineStage -- ML Pipeline Component 2 | 3 | The https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.ml.PipelineStage[PipelineStage] abstract class represents a single stage in a link:spark-mllib-Pipeline.md[Pipeline]. 4 | 5 | `PipelineStage` has the following direct implementations (of which few are abstract classes, too): 6 | 7 | * link:spark-mllib-estimators.md[Estimators] 8 | * link:spark-mllib-models.md[Models] 9 | * link:spark-mllib-Pipeline.md[Pipeline] 10 | * link:spark-mllib-estimators.md#Predictor[Predictor] 11 | * link:spark-mllib-transformers.md[Transformer] 12 | 13 | [[transformSchema]] 14 | Each `PipelineStage` transforms schema using `transformSchema` family of methods: 15 | 16 | ``` 17 | transformSchema(schema: StructType): StructType 18 | transformSchema(schema: StructType, logging: Boolean): StructType 19 | ``` 20 | 21 | NOTE: link:spark-sql-StructType.md[StructType] describes a schema of a DataFrame. 22 | 23 | [TIP] 24 | ==== 25 | Enable `DEBUG` logging level for the respective `PipelineStage` implementations to see what happens beneath. 26 | ==== 27 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-RandomForestClassifier.adoc: -------------------------------------------------------------------------------- 1 | == [[RandomForestClassifier]] RandomForestClassifier 2 | 3 | `RandomForestClassifier` is a probabilistic link:spark-mllib-Classifier.md[Classifier] for...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-Regressor.adoc: -------------------------------------------------------------------------------- 1 | == [[Regressor]] Regressor 2 | 3 | `Regressor` is...FIXME 4 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-TrainValidationSplit.adoc: -------------------------------------------------------------------------------- 1 | == [[TrainValidationSplit]] TrainValidationSplit 2 | 3 | `TrainValidationSplit` is...FIXME 4 | 5 | === [[transformSchema]] Validating and Transforming Schema -- `transformSchema` Method 6 | 7 | [source, scala] 8 | ---- 9 | transformSchema(schema: StructType): StructType 10 | ---- 11 | 12 | NOTE: `transformSchema` is part of link:spark-mllib-PipelineStage.md#transformSchema[PipelineStage Contract]. 13 | 14 | `transformSchema` simply passes the call to link:spark-mllib-ValidatorParams.md#transformSchemaImpl[transformSchemaImpl] (that is shared between link:spark-mllib-CrossValidator.md[CrossValidator] and `TrainValidationSplit`). 15 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-Transformer.adoc: -------------------------------------------------------------------------------- 1 | == [[Transformer]] Transformer 2 | 3 | `Transformer` is the <> in Spark MLlib for transformers that <>. 4 | 5 | `Transformer` is a link:spark-mllib-PipelineStage.md[PipelineStage] and so...FIXME 6 | 7 | === [[transform-paramMap]] Transforming Dataset with Extra Parameters -- `transform` Method 8 | 9 | CAUTION: FIXME 10 | 11 | === [[contract]] Transformer Contract 12 | 13 | [source, scala] 14 | ---- 15 | package org.apache.spark.ml 16 | 17 | abstract class Evaluator { 18 | // only required methods that have no implementation 19 | def transform(dataset: Dataset[_]): DataFrame 20 | def copy(extra: ParamMap): Transformer 21 | } 22 | ---- 23 | 24 | .Transformer Contract 25 | [cols="1,2",options="header",width="100%"] 26 | |=== 27 | | Method 28 | | Description 29 | 30 | | [[copy]] `copy` 31 | | Used when... 32 | 33 | | [[transform]] `transform` 34 | | Used when... 35 | |=== 36 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-labeledpoint.adoc: -------------------------------------------------------------------------------- 1 | == LabeledPoint 2 | 3 | CAUTION: FIXME 4 | 5 | `LabeledPoint` is a convenient class for declaring a schema for DataFrames that are used as input data for link:spark-mllib-estimators.md#LinearRegression[Linear Regression] in Spark MLlib. 6 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-latent-dirichlet-allocation.adoc: -------------------------------------------------------------------------------- 1 | == Latent Dirichlet Allocation (LDA) 2 | 3 | NOTE: Information here are based almost exclusively from the blog post https://databricks.com/blog/2015/03/25/topic-modeling-with-lda-mllib-meets-graphx.html[Topic modeling with LDA: MLlib meets GraphX]. 4 | 5 | *Topic modeling* is a type of model that can be very useful in identifying hidden thematic structure in documents. Broadly speaking, it aims to find structure within an unstructured collection of documents. Once the structure is "discovered", you may answer questions like: 6 | 7 | * What is document X about? 8 | * How similar are documents X and Y? 9 | * If I am interested in topic Z, which documents should I read first? 10 | 11 | Spark MLlib offers out-of-the-box support for *Latent Dirichlet Allocation (LDA)* which is the first MLlib algorithm built upon link:spark-graphx.md[GraphX]. 12 | 13 | *Topic models* automatically infer the topics discussed in a collection of documents. 14 | 15 | === [[example]] Example 16 | 17 | CAUTION: FIXME Use Tokenizer, StopWordsRemover, CountVectorizer, and finally LDA in a pipeline. 18 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-logistic-regression.adoc: -------------------------------------------------------------------------------- 1 | == Logistic Regression 2 | 3 | [quote, 'Wikipedia, the free encyclopedia', 'https://en.wikipedia.org/wiki/Logistic_regression[Logistic regression]'] 4 | ____ 5 | In statistics, *logistic regression*, or *logit regression*, or *logit model* is a regression model where the dependent variable (DV) is categorical. 6 | ____ 7 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-streaming.adoc: -------------------------------------------------------------------------------- 1 | == Streaming MLlib 2 | 3 | The following Machine Learning algorithms have their streaming variants in MLlib: 4 | 5 | * <> 6 | * <> 7 | * <> 8 | 9 | They can train models and predict on streaming data. 10 | 11 | NOTE: The streaming algorithms belong to `spark.mllib` (the older RDD-based API). 12 | 13 | === [[kmeans]] Streaming k-means 14 | 15 | `org.apache.spark.mllib.clustering.StreamingKMeans` 16 | 17 | === [[linear-regression]] Streaming Linear Regression 18 | 19 | `org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD` 20 | 21 | === [[logistic-regression]] Streaming Logistic Regression 22 | 23 | `org.apache.spark.mllib.classification.StreamingLogisticRegressionWithSGD` 24 | 25 | === Sources 26 | 27 | * https://youtu.be/uUQTSPvD1mc[Streaming Machine Learning in Spark- Jeremy Freeman (HHMI Janelia Research Center)] 28 | -------------------------------------------------------------------------------- /modules/spark-mllib/pages/spark-mllib-transformers-Tokenizer.adoc: -------------------------------------------------------------------------------- 1 | == [[Tokenizer]] Tokenizer 2 | 3 | `Tokenizer` is a link:spark-mllib-transformers.md#UnaryTransformer[unary transformer] that converts the column of String values to lowercase and then splits it by white spaces. 4 | 5 | [source,scala] 6 | ---- 7 | import org.apache.spark.ml.feature.Tokenizer 8 | val tok = new Tokenizer() 9 | 10 | // dataset to transform 11 | val df = Seq( 12 | (1, "Hello world!"), 13 | (2, "Here is yet another sentence.")).toDF("id", "sentence") 14 | 15 | val tokenized = tok.setInputCol("sentence").setOutputCol("tokens").transform(df) 16 | 17 | scala> tokenized.show(truncate = false) 18 | +---+-----------------------------+-----------------------------------+ 19 | |id |sentence |tokens | 20 | +---+-----------------------------+-----------------------------------+ 21 | |1 |Hello world! |[hello, world!] | 22 | |2 |Here is yet another sentence.|[here, is, yet, another, sentence.]| 23 | +---+-----------------------------+-----------------------------------+ 24 | ---- 25 | -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-client-submitApplication.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-client-submitApplication.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-main.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-main.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-registerAM.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-registerAM.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-runAMEndpoint.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster-runAMEndpoint.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-ApplicationMaster.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-Client-YarnClient.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-Client-YarnClient.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-ExecutorRunnable.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-ExecutorRunnable.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-YarnAllocator-amClient-ResourceManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-YarnAllocator-amClient-ResourceManager.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-YarnAllocator-runAllocatedContainers.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-YarnAllocator-runAllocatedContainers.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-YarnAllocator.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-YarnAllocator.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-YarnClientSchedulerBackend-start.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-YarnClientSchedulerBackend-start.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-YarnRMClient-register.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-YarnRMClient-register.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/assets/images/spark-yarn-console-progress-10.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/apache-spark-internals/ac57607de8ec69ea125d2300ab06441cc487a252/modules/spark-on-yarn/assets/images/spark-yarn-console-progress-10.png -------------------------------------------------------------------------------- /modules/spark-on-yarn/pages/spark-yarn-ConfigurableCredentialManager.adoc: -------------------------------------------------------------------------------- 1 | == ConfigurableCredentialManager 2 | 3 | CAUTION: FIXME 4 | 5 | === [[creating-instance]] Creating ConfigurableCredentialManager Instance 6 | 7 | CAUTION: FIXME 8 | 9 | === [[credentialRenewer]] `credentialRenewer` Method 10 | 11 | CAUTION: FIXME 12 | 13 | === [[obtainCredentials]] Obtaining Security Tokens from Credential Providers -- `obtainCredentials` Method 14 | 15 | CAUTION: FIXME 16 | -------------------------------------------------------------------------------- /modules/spark-on-yarn/pages/spark-yarn-cluster-setup.adoc: -------------------------------------------------------------------------------- 1 | == Setting up YARN Cluster 2 | 3 | YARN uses the following environment variables: 4 | 5 | * `YARN_CONF_DIR` 6 | * `HADOOP_CONF_DIR` 7 | * `HADOOP_HOME` 8 | -------------------------------------------------------------------------------- /modules/spark-on-yarn/pages/spark-yarn-kerberos.adoc: -------------------------------------------------------------------------------- 1 | == Kerberos 2 | 3 | * Microsoft incorporated Kerberos authentication into Windows 2000 4 | * Two open source Kerberos implementations exist: the MIT reference implementation and the Heimdal Kerberos implementation. 5 | 6 | YARN supports user authentication via Kerberos (so do the other services: HDFS, HBase, Hive). 7 | 8 | === [[delegation-tokens]] Service Delegation Tokens 9 | 10 | CAUTION: FIXME 11 | 12 | === [[i-want-more]] Further reading or watching 13 | 14 | * (video training) https://www.safaribooksonline.com/library/view/introduction-to-hadoop/9781771375054/[Introduction to Hadoop Security] 15 | * https://www.safaribooksonline.com/library/view/hadoop-security/9781491900970/[Hadoop Security] 16 | * https://www.safaribooksonline.com/library/view/kerberos-the-definitive/0596004036/[Kerberos: The Definitive Guide] 17 | -------------------------------------------------------------------------------- /modules/spark-on-yarn/pages/spark-yarn-schedulerbackends.adoc: -------------------------------------------------------------------------------- 1 | == SchedulerBackends for YARN 2 | 3 | There are currently two xref:scheduler:SchedulerBackend.md[SchedulerBackends] for link:README.md[Spark on YARN] per link:spark-submit/index.md#deploy-mode[deploy mode]: 4 | 5 | * link:spark-yarn-client-yarnclientschedulerbackend.md[YarnClientSchedulerBackend] for *client* deploy mode 6 | * link:spark-yarn-yarnschedulerbackend.md[YarnSchedulerBackend] for *cluster* deploy mode 7 | 8 | They are concrete link:spark-yarn-yarnschedulerbackend.md[YarnSchedulerBackends]. 9 | -------------------------------------------------------------------------------- /modules/spark-on-yarn/pages/spark-yarn-taskschedulers.adoc: -------------------------------------------------------------------------------- 1 | == TaskSchedulers for YARN 2 | 3 | There are two xref:scheduler:TaskScheduler.md[TaskSchedulers] for link:README.md[Spark on YARN] per link:spark-submit/index.md#deploy-mode[deploy mode]: 4 | 5 | * link:spark-yarn-yarnscheduler.md[YarnScheduler] for *client* deploy mode 6 | * link:spark-yarn-yarnclusterscheduler.md[YarnClusterScheduler] for *cluster* deploy mode 7 | -------------------------------------------------------------------------------- /modules/spark-standalone/pages/spark-standalone-submission-gateways.adoc: -------------------------------------------------------------------------------- 1 | == Submission Gateways 2 | 3 | CAUTION: FIXME 4 | 5 | From `SparkSubmit.submit`: 6 | 7 | In standalone cluster mode, there are two submission gateways: 8 | 9 | 1. The traditional legacy RPC gateway using o.a.s.deploy.Client as a wrapper 10 | 2. The new REST-based gateway introduced in Spark 1.3 11 | 12 | The latter is the default behaviour as of Spark 1.3, but Spark submit will fail over to use the legacy gateway if the master endpoint turns out to be not a REST server. 13 | -------------------------------------------------------------------------------- /modules/spark-standalone/pages/spark-standalone-webui-ApplicationPage.adoc: -------------------------------------------------------------------------------- 1 | == [[ApplicationPage]] ApplicationPage 2 | 3 | `ApplicationPage` is a link:spark-webui-WebUIPage.md[WebUIPage] with *app* link:spark-webui-WebUIPage.md#prefix[prefix]. 4 | 5 | `ApplicationPage` is <> exclusively when Spark Standalone's `MasterWebUI` is initialized. 6 | 7 | [[creating-instance]] 8 | [[parent]] 9 | `ApplicationPage` takes a `MasterWebUI` when created: 10 | -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | git+https://${GH_TOKEN}@github.com/squidfunk/mkdocs-material-insiders.git 2 | mkdocs-minify-plugin>=0.7.2 3 | mkdocs-git-revision-date-localized-plugin>=1.2.2 4 | mkdocs-git-revision-date-plugin>=0.3.2 5 | mkdocs-awesome-pages-plugin>=2.9.2 6 | mkdocs-redirects>=1.2.1 7 | mkdocs-macros-plugin>=1.0.5 8 | --------------------------------------------------------------------------------