aboutsummaryrefslogtreecommitdiff
path: root/docs/configuration.md
Commit message (Collapse)AuthorAgeFilesLines
* SPARK-4585. Spark dynamic executor allocation should use minExecutors as...Sandy Ryza2015-02-021-6/+14
| | | | | | | | | | | ... initial number Author: Sandy Ryza <sandy@cloudera.com> Closes #4051 from sryza/sandy-spark-4585 and squashes the following commits: d1dd039 [Sandy Ryza] Add spark.dynamicAllocation.initialNumExecutors and make min and max not required b7c59dc [Sandy Ryza] SPARK-4585. Spark dynamic executor allocation should use minExecutors as initial number
* [SPARK-4387][PySpark] Refactoring python profiling code to make it extensibleYandu Oppacher2015-01-281-0/+3
| | | | | | | | | | | | | | | | | | | | | | | | This PR is based on #3255 , fix conflicts and code style. Closes #3255. Author: Yandu Oppacher <yandu.oppacher@jadedpixel.com> Author: Davies Liu <davies@databricks.com> Closes #3901 from davies/refactor-python-profile-code and squashes the following commits: b4a9306 [Davies Liu] fix tests 4b79ce8 [Davies Liu] add docstring for profiler_cls 2700e47 [Davies Liu] use BasicProfiler as default 349e341 [Davies Liu] more refactor 6a5d4df [Davies Liu] refactor and fix tests 31bf6b6 [Davies Liu] fix code style 0864b5d [Yandu Oppacher] Remove unused method 76a6c37 [Yandu Oppacher] Added a profile collector to accumulate the profilers per stage 9eefc36 [Yandu Oppacher] Fix doc 9ace076 [Yandu Oppacher] Refactor of profiler, and moved tests around 8739aff [Yandu Oppacher] Code review fixes 9bda3ec [Yandu Oppacher] Refactor profiler code
* SPARK-3852 [DOCS] Document spark.driver.extra* configsSean Owen2015-01-251-0/+21
| | | | | | | | | | As per the JIRA. I copied the `spark.executor.extra*` text, but removed info that appears to be specific to the `executor` config and not `driver`. Author: Sean Owen <sowen@cloudera.com> Closes #4185 from srowen/SPARK-3852 and squashes the following commits: f60a8a1 [Sean Owen] Document spark.driver.extra* configs
* [SPARK-1507][YARN]specify # cores for ApplicationMasterWangTaoTheTonic2015-01-161-4/+11
| | | | | | | | | | | | | | | | | | | Based on top of changes in https://github.com/apache/spark/pull/3806. https://issues.apache.org/jira/browse/SPARK-1507 `--driver-cores` and `spark.driver.cores` for all cluster modes and `spark.yarn.am.cores` for yarn client mode. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #4018 from WangTaoTheTonic/SPARK-1507 and squashes the following commits: 01419d3 [WangTaoTheTonic] amend the args name b255795 [WangTaoTheTonic] indet thing d86557c [WangTaoTheTonic] some comments amend 43c9392 [WangTao] fix compile error b39a100 [WangTao] specify # cores for ApplicationMaster
* [SPARK-5131][Streaming][DOC]: There is a discrepancy in WAL implementation ↵uncleGen2015-01-131-1/+1
| | | | | | | | | | | | | and configuration doc. There is a discrepancy in WAL implementation and configuration doc. Author: uncleGen <hustyugm@gmail.com> Closes #3930 from uncleGen/master-clean-doc and squashes the following commits: 3a4245f [uncleGen] doc typo 8e407d3 [uncleGen] doc typo
* [SPARK-5073] spark.storage.memoryMapThreshold have two default valuelewuathe2015-01-111-1/+1
| | | | | | | | | | | | | Because major OS page sizes is about 4KB, the default value of spark.storage.memoryMapThreshold is integrated to 2 * 4096 Author: lewuathe <lewuathe@me.com> Closes #3900 from Lewuathe/integrate-memoryMapThreshold and squashes the following commits: e417acd [lewuathe] [SPARK-5073] Update docs/configuration 834aba4 [lewuathe] [SPARK-5073] Fix style adcea33 [lewuathe] [SPARK-5073] Integrate memory map threshold to 2MB fcce2e5 [lewuathe] [SPARK-5073] spark.storage.memoryMapThreshold have two default value
* [SPARK-5093] Set spark.network.timeout to 120s consistently.Reynold Xin2015-01-051-3/+3
| | | | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #3903 from rxin/timeout-120 and squashes the following commits: 7c2138e [Reynold Xin] [SPARK-5093] Set spark.network.timeout to 120s consistently.
* [SPARK-4688] Have a single shared network timeout in SparkVarun Saxena2015-01-051-0/+10
| | | | | | | | | | | | | | | | [SPARK-4688] Have a single shared network timeout in Spark Author: Varun Saxena <vsaxena.varun@gmail.com> Author: varunsaxena <vsaxena.varun@gmail.com> Closes #3562 from varunsaxena/SPARK-4688 and squashes the following commits: 6e97f72 [Varun Saxena] [SPARK-4688] Single shared network timeout cd783a2 [Varun Saxena] SPARK-4688 d6f8c29 [Varun Saxena] SCALA-4688 9562b15 [Varun Saxena] SPARK-4688 a75f014 [varunsaxena] SPARK-4688 594226c [varunsaxena] SPARK-4688
* [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobsJosh Rosen2015-01-041-1/+3
| | | | | | | | | | | | | | | | | | | | | This patch disables output spec. validation for jobs launched through Spark Streaming, since this interferes with checkpoint recovery. Hadoop OutputFormats have a `checkOutputSpecs` method which performs certain checks prior to writing output, such as checking whether the output directory already exists. SPARK-1100 added checks for FileOutputFormat, SPARK-1677 (#947) added a SparkConf configuration to disable these checks, and SPARK-2309 (#1088) extended these checks to run for all OutputFormats, not just FileOutputFormat. In Spark Streaming, we might have to re-process a batch during checkpoint recovery, so `save` actions may be called multiple times. In addition to `DStream`'s own save actions, users might use `transform` or `foreachRDD` and call the `RDD` and `PairRDD` save actions. When output spec. validation is enabled, the second calls to these actions will fail due to existing output. This patch automatically disables output spec. validation for jobs submitted by the Spark Streaming scheduler. This is done by using Scala's `DynamicVariable` to propagate the bypass setting without having to mutate SparkConf or introduce a global variable. Author: Josh Rosen <joshrosen@databricks.com> Closes #3832 from JoshRosen/SPARK-4835 and squashes the following commits: 36eaf35 [Josh Rosen] Add comment explaining use of transform() in test. 6485cf8 [Josh Rosen] Add test case in Streaming; fix bug for transform() 7b3e06a [Josh Rosen] Remove Streaming-specific setting to undo this change; update conf. guide bf9094d [Josh Rosen] Revise disableOutputSpecValidation() comment to not refer to Spark Streaming. e581d17 [Josh Rosen] Deduplicate isOutputSpecValidationEnabled logic. 762e473 [Josh Rosen] [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs.
* [SPARK-4982][DOC] `spark.ui.retainedJobs` description is wrong in Spark UI ↵wangxiaojing2014-12-291-1/+1
| | | | | | | | | | configuration guide Author: wangxiaojing <u9jing@gmail.com> Closes #3818 from wangxiaojing/SPARK-4982 and squashes the following commits: fe2ad5f [wangxiaojing] change stages to jobs
* [SPARK-4864] Add documentation to Netty-based configsAaron Davidson2014-12-221-0/+35
| | | | | | | | | Author: Aaron Davidson <aaron@databricks.com> Closes #3713 from aarondav/netty-configs and squashes the following commits: 8a8b373 [Aaron Davidson] Address Patrick's comments 3b1f84e [Aaron Davidson] [SPARK-4864] Add documentation to Netty-based configs
* [SPARK-4140] Document dynamic allocationAndrew Or2014-12-191-0/+61
| | | | | | | | | | | | | | | | | | | | Once the external shuffle service is also documented, the dynamic allocation section will link to it. Let me know if the whole dynamic allocation should be moved to its separate page; I personally think the organization might be cleaner that way. This patch builds on top of oza's work in #3689. aarondav pwendell Author: Andrew Or <andrew@databricks.com> Author: Tsuyoshi Ozawa <ozawa.tsuyoshi@gmail.com> Closes #3731 from andrewor14/document-dynamic-allocation and squashes the following commits: 1281447 [Andrew Or] Address a few comments b9843f2 [Andrew Or] Document the configs as well 246fb44 [Andrew Or] Merge branch 'SPARK-4839' of github.com:oza/spark into document-dynamic-allocation 8c64004 [Andrew Or] Add documentation for dynamic allocation (without configs) 6827b56 [Tsuyoshi Ozawa] Fixing a documentation of spark.dynamicAllocation.enabled. 53cff58 [Tsuyoshi Ozawa] Adding a documentation about dynamic resource allocation.
* [SPARK-4668] Fix some documentation typos.Ryan Williams2014-12-151-3/+3
| | | | | | | | | | | | | | | | | Author: Ryan Williams <ryan.blake.williams@gmail.com> Closes #3523 from ryan-williams/tweaks and squashes the following commits: d2eddaa [Ryan Williams] code review feedback ce27fc1 [Ryan Williams] CoGroupedRDD comment nit c6cfad9 [Ryan Williams] remove unnecessary if statement b74ea35 [Ryan Williams] comment fix b0221f0 [Ryan Williams] fix a gendered pronoun c71ffed [Ryan Williams] use names on a few boolean parameters 89954aa [Ryan Williams] clarify some comments in {Security,Shuffle}Manager e465dac [Ryan Williams] Saved building-spark.md with Dillinger.io 83e8358 [Ryan Williams] fix pom.xml typo dc4662b [Ryan Williams] typo fixes in tuning.md, configuration.md
* [SPARK-4806] Streaming doc update for 1.2Tathagata Das2014-12-111-59/+74
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
* SPARK-4770. [DOC] [YARN] spark.scheduler.minRegisteredResourcesRatio doc...Sandy Ryza2014-12-081-2/+2
| | | | | | | | | | ...umented default is incorrect for YARN Author: Sandy Ryza <sandy@cloudera.com> Closes #3624 from sryza/sandy-spark-4770 and squashes the following commits: bd81a3a [Sandy Ryza] SPARK-4770. [DOC] [YARN] spark.scheduler.minRegisteredResourcesRatio documented default is incorrect for YARN
* [SPARK-4686] Link to allowed master URLs is brokenKay Ousterhout2014-12-021-1/+1
| | | | | | | | | | | | The link points to the old scala programming guide; it should point to the submitting applications page. This should be backported to 1.1.2 (it's been broken as of 1.0). Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #3542 from kayousterhout/SPARK-4686 and squashes the following commits: a8fc43b [Kay Ousterhout] [SPARK-4686] Link to allowed master URLs is broken
* [SPARK-4344][DOCS] adding documentation on spark.yarn.user.classpath.firstarahuja2014-11-251-0/+1
| | | | | | | | | | The documentation for the two parameters is the same with a pointer from the standalone parameter to the yarn parameter Author: arahuja <aahuja11@gmail.com> Closes #3209 from arahuja/yarn-classpath-first-param and squashes the following commits: 51cb9b2 [arahuja] [SPARK-4344][DOCS] adding documentation for YARN on userClassPathFirst
* [SPARK-3722][Docs]minor improvement and fix in docsWangTao2014-11-141-1/+1
| | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-3722 Author: WangTao <barneystinson@aliyun.com> Closes #2579 from WangTaoTheTonic/docsWork and squashes the following commits: 6f91cec [WangTao] use more wording express 29d22fa [WangTao] delete the specified version link 34cb4ea [WangTao] Update running-on-yarn.md 4ee1a26 [WangTao] minor improvement and fix in docs
* SPARK-4230. Doc for spark.default.parallelism is incorrectSandy Ryza2014-11-101-2/+5
| | | | | | | | | Author: Sandy Ryza <sandy@cloudera.com> Closes #3107 from sryza/sandy-spark-4230 and squashes the following commits: 37a1d19 [Sandy Ryza] Clear up a couple things 34d53de [Sandy Ryza] SPARK-4230. Doc for spark.default.parallelism is incorrect
* SPARK-4040. Update documentation to exemplify use of local (n) value, fo...jay@apache.org2014-11-051-2/+8
| | | | | | | | | | This is a minor docs update which helps to clarify the way local[n] is used for streaming apps. Author: jay@apache.org <jayunit100> Closes #2964 from jayunit100/SPARK-4040 and squashes the following commits: 35b5a5e [jay@apache.org] SPARK-4040: Update documentation to exemplify use of local (n) value.
* [SPARK-4183] Enable NettyBlockTransferService by defaultAaron Davidson2014-11-021-0/+10
| | | | | | | | | | Note that we're turning this on for at least the first part of the QA period as a trial. We want to enable this (and deprecate the NioBlockTransferService) as soon as possible in the hopes that NettyBlockTransferService will be more stable and easier to maintain. We will turn it off if we run into major issues. Author: Aaron Davidson <aaron@databricks.com> Closes #3049 from aarondav/enable-netty and squashes the following commits: bb981cc [Aaron Davidson] [SPARK-4183] Enable NettyBlockTransferService by default
* [SPARK-3466] Limit size of results that a driver collects for each actionDavies Liu2014-11-021-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | Right now, operations like collect() and take() can crash the driver with an OOM if they bring back too many data. This PR will introduce spark.driver.maxResultSize, after setting it, the driver will abort a job if its result is bigger than it. By default, it's 1g (for backward compatibility for most the cases). In local mode, the driver and executor share the same JVM, the default setting can not protect JVM from OOM. cc mateiz Author: Davies Liu <davies@databricks.com> Closes #3003 from davies/collect and squashes the following commits: 248ed5e [Davies Liu] fix compile 272522e [Davies Liu] address comments 2c35773 [Davies Liu] add sizes in message of abort() 5d62303 [Davies Liu] address comments bc3c077 [Davies Liu] Merge branch 'master' of github.com:apache/spark into collect 11f97c5 [Davies Liu] address comments 47b144f [Davies Liu] check the size of result before send and fetch 3d81af2 [Davies Liu] address comments ca8267d [Davies Liu] limit the size of data by collect
* Revert "[SPARK-4183] Enable NettyBlockTransferService by default"Patrick Wendell2014-11-011-10/+0
| | | | This reverts commit 59e626c701227634336110e1bc23afd94c535ede.
* [SPARK-4183] Enable NettyBlockTransferService by defaultAaron Davidson2014-11-011-0/+10
| | | | | | | | | | Note that we're turning this on for at least the first part of the QA period as a trial. We want to enable this (and deprecate the NioBlockTransferService) as soon as possible in the hopes that NettyBlockTransferService will be more stable and easier to maintain. We will turn it off if we run into major issues. Author: Aaron Davidson <aaron@databricks.com> Closes #3049 from aarondav/enable-netty and squashes the following commits: bb981cc [Aaron Davidson] [SPARK-4183] Enable NettyBlockTransferService by default
* [SPARK-2321] Stable pull-based progress / status APIJosh Rosen2014-10-251-1/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This pull request is a first step towards the implementation of a stable, pull-based progress / status API for Spark (see [SPARK-2321](https://issues.apache.org/jira/browse/SPARK-2321)). For now, I'd like to discuss the basic implementation, API names, and overall interface design. Once we arrive at a good design, I'll go back and add additional methods to expose more information via these API. #### Design goals: - Pull-based API - Usable from Java / Scala / Python (eventually, likely with a wrapper) - Can be extended to expose more information without introducing binary incompatibilities. - Returns immutable objects. - Don't leak any implementation details, preserving our freedom to change the implementation. #### Implementation: - Add public methods (`getJobInfo`, `getStageInfo`) to SparkContext to allow status / progress information to be retrieved. - Add public interfaces (`SparkJobInfo`, `SparkStageInfo`) for our API return values. These interfaces consist entirely of Java-style getter methods. The interfaces are currently implemented in Java. I decided to explicitly separate the interface from its implementation (`SparkJobInfoImpl`, `SparkStageInfoImpl`) in order to prevent users from constructing these responses themselves. -Allow an existing JobProgressListener to be used when constructing a live SparkUI. This allows us to re-use this listeners in the implementation of this status API. There are a few reasons why this listener re-use makes sense: - The status API and web UI are guaranteed to show consistent information. - These listeners are already well-tested. - The same garbage-collection / information retention configurations can apply to both this API and the web UI. - Extend JobProgressListener to maintain `jobId -> Job` and `stageId -> Stage` mappings. The progress API methods are implemented in a separate trait that's mixed into SparkContext. This helps to avoid SparkContext.scala from becoming larger and more difficult to read. Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <joshrosen@apache.org> Closes #2696 from JoshRosen/progress-reporting-api and squashes the following commits: e6aa78d [Josh Rosen] Add tests. b585c16 [Josh Rosen] Accept SparkListenerBus instead of more specific subclasses. c96402d [Josh Rosen] Address review comments. 2707f98 [Josh Rosen] Expose current stage attempt id c28ba76 [Josh Rosen] Update demo code: 646ff1d [Josh Rosen] Document spark.ui.retainedJobs. 7f47d6d [Josh Rosen] Clean up SparkUI constructors, per Andrew's feedback. b77b3d8 [Josh Rosen] Merge remote-tracking branch 'origin/master' into progress-reporting-api 787444c [Josh Rosen] Move status API methods into trait that can be mixed into SparkContext. f9a9a00 [Josh Rosen] More review comments: 3dc79af [Josh Rosen] Remove creation of unused listeners in SparkContext. 249ca16 [Josh Rosen] Address several review comments: da5648e [Josh Rosen] Add example of basic progress reporting in Java. 7319ffd [Josh Rosen] Add getJobIdsForGroup() and num*Tasks() methods. cc568e5 [Josh Rosen] Add note explaining that interfaces should not be implemented outside of Spark. 6e840d4 [Josh Rosen] Remove getter-style names and "consistent snapshot" semantics: 08cbec9 [Josh Rosen] Begin to sketch the interfaces for a stable, public status API. ac2d13a [Josh Rosen] Add jobId->stage, stageId->stage mappings in JobProgressListener 24de263 [Josh Rosen] Create UI listeners in SparkContext instead of in Tabs:
* SPARK-1813. Add a utility to SparkConf that makes using Kryo really easySandy Ryza2014-10-211-2/+13
| | | | | | | | | | | | Author: Sandy Ryza <sandy@cloudera.com> Closes #789 from sryza/sandy-spark-1813 and squashes the following commits: 48b05e9 [Sandy Ryza] Simplify b824932 [Sandy Ryza] Allow both spark.kryo.classesToRegister and spark.kryo.registrator at the same time 6a15bb7 [Sandy Ryza] Small fix a2278c0 [Sandy Ryza] Respond to review comments 6ef592e [Sandy Ryza] SPARK-1813. Add a utility to SparkConf that makes using Kryo really easy
* [SPARK-2546] Clone JobConf for each task (branch-1.0 / 1.1 backport)Josh Rosen2014-10-191-0/+9
| | | | | | | | | | | | | | | | | | | | This patch attempts to fix SPARK-2546 in `branch-1.0` and `branch-1.1`. The underlying problem is that thread-safety issues in Hadoop Configuration objects may cause Spark tasks to get stuck in infinite loops. The approach taken here is to clone a new copy of the JobConf for each task rather than sharing a single copy between tasks. Note that there are still Configuration thread-safety issues that may affect the driver, but these seem much less likely to occur in practice and will be more complex to fix (see discussion on the SPARK-2546 ticket). This cloning is guarded by a new configuration option (`spark.hadoop.cloneConf`) and is disabled by default in order to avoid unexpected performance regressions for workloads that are unaffected by the Configuration thread-safety issues. Author: Josh Rosen <joshrosen@apache.org> Closes #2684 from JoshRosen/jobconf-fix-backport and squashes the following commits: f14f259 [Josh Rosen] Add configuration option to control cloning of Hadoop JobConf. b562451 [Josh Rosen] Remove unused jobConfCacheKey field. dd25697 [Josh Rosen] [SPARK-2546] [1.0 / 1.1 backport] Clone JobConf for each task. (cherry picked from commit 2cd40db2b3ab5ddcb323fd05c171dbd9025f9e71) Signed-off-by: Josh Rosen <joshrosen@databricks.com> Conflicts: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
* [SPARK-3890][Docs]remove redundant spark.executor.memory in docWangTaoTheTonic2014-10-161-12/+4
| | | | | | | | | | | | | Introduced in https://github.com/pwendell/spark/commit/f7e79bc42c1635686c3af01eef147dae92de2529, I'm not sure why we need two spark.executor.memory here. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #2745 from WangTaoTheTonic/redundantconfig and squashes the following commits: e7564dc [WangTao] too long line fdbdb1f [WangTaoTheTonic] trivial workaround d06b6e5 [WangTaoTheTonic] remove redundant spark.executor.memory in doc
* [SPARK-3923] Increase Akka heartbeat pause above heartbeat intervalAaron Davidson2014-10-161-1/+1
| | | | | | | | | | | | | Something about the 2.3.4 upgrade seems to have made the issue manifest where all the services disconnect from each other after exactly 1000 seconds (which is the heartbeat interval). [This post](https://groups.google.com/forum/#!topic/akka-user/X3xzpTCbEFs) suggests that heartbeat pause should be greater than heartbeat interval, and increasing the pause from 600s to 6000s seems to have rectified the issue. My current cluster has now exceeded 1400s of uptime without failure! I do not know why this fixed it, because the threshold we have set for the failure detector is the exponent of a timeout, and 300 is extremely large. Perhaps the default failure detector changed in 2.3.4 and now ignores threshold. Author: Aaron Davidson <aaron@databricks.com> Closes #2784 from aarondav/fix-timeout and squashes the following commits: bd1151a [Aaron Davidson] Increase pause, don't decrease interval 9cb0372 [Aaron Davidson] [SPARK-3923] Decrease Akka heartbeat interval below heartbeat pause
* add spark.driver.memory to config docsnartz2014-10-091-0/+8
| | | | | | | | | | | | | | It took me a minute to track this down, so I thought it could be useful to have it in the docs. I'm unsure if 512mb is the default for spark.driver.memory? Also - there could be a better value for the 'description' to differentiate it from spark.executor.memory. Author: nartz <nartzpod@gmail.com> Author: Nathan Artz <nathanartz@Nathans-MacBook-Pro.local> Closes #2410 from nartz/docs/add-spark-driver-memory-to-config-docs and squashes the following commits: a2f6c62 [nartz] Update configuration.md 74521b8 [Nathan Artz] add spark.driver.memory to config docs
* [SPARK-3535][Mesos] Fix resource handling.Brenden Matthews2014-10-031-0/+11
| | | | | | | | Author: Brenden Matthews <brenden@diddyinc.com> Closes #2401 from brndnmtthws/master and squashes the following commits: 4abaa5d [Brenden Matthews] [SPARK-3535][Mesos] Fix resource handling.
* SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIREugenCepoi2014-10-031-0/+7
| | | | | | | | | | | | | Update of PR #997. With this PR, setting SPARK_CONF_DIR overrides SPARK_HOME/conf (not only spark-defaults.conf and spark-env). Author: EugenCepoi <cepoi.eugen@gmail.com> Closes #2481 from EugenCepoi/SPARK-2058 and squashes the following commits: 0bb32c2 [EugenCepoi] use orElse orNull and fixing trailing percent in compute-classpath.cmd 77f35d7 [EugenCepoi] SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIR
* [SPARK-3766][Doc]Snappy is also the default compress codec for broadcast ↵scwf2014-10-021-4/+5
| | | | | | | | | | variables Author: scwf <wangfei1@huawei.com> Closes #2632 from scwf/compress-doc and squashes the following commits: 7983a1a [scwf] snappy is the default compression codec for broadcast
* [SPARK-3478] [PySpark] Profile the Python tasksDavies Liu2014-09-301-0/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD<id=3> ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>) 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>) 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD<id=1> ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>) 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" This is bugfix of #2351 cc JoshRosen Author: Davies Liu <davies.liu@gmail.com> Closes #2556 from davies/profiler and squashes the following commits: e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 858e74c [Davies Liu] compatitable with python 2.6 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python
* Revert "[SPARK-3478] [PySpark] Profile the Python tasks"Josh Rosen2014-09-261-19/+0
| | | | This reverts commit 1aa549ba9839565274a12c52fa1075b424f138a6.
* [SPARK-3478] [PySpark] Profile the Python tasksDavies Liu2014-09-261-0/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD<id=3> ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>) 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>) 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD<id=1> ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>) 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" Author: Davies Liu <davies.liu@gmail.com> Closes #2351 from davies/profiler and squashes the following commits: 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python
* [SPARK-3565]Fix configuration item not consistent with documentWangTaoTheTonic2014-09-171-1/+1
| | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-3565 "spark.ports.maxRetries" should be "spark.port.maxRetries". Make the configuration keys in document and code consistent. Author: WangTaoTheTonic <barneystinson@aliyun.com> Closes #2427 from WangTaoTheTonic/fixPortRetries and squashes the following commits: c178813 [WangTaoTheTonic] Use blank lines trigger Jenkins 646f3fe [WangTaoTheTonic] also in SparkBuild.scala 3700dba [WangTaoTheTonic] Fix configuration item not consistent with document
* [Docs] Correct spark.files.fetchTimeout default valueviper-kun2014-09-171-2/+2
| | | | | | | | | | | change the value of spark.files.fetchTimeout Author: viper-kun <xukun.xu@huawei.com> Closes #2406 from viper-kun/master and squashes the following commits: ecb0d46 [viper-kun] [Docs] Correct spark.files.fetchTimeout default value 7cf4c7a [viper-kun] Update configuration.md
* [SPARK-3030] [PySpark] Reuse Python workerDavies Liu2014-09-131-0/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu <davies.liu@gmail.com> Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker
* [SPARK-3280] Made sort-based shuffle the default implementationReynold Xin2014-09-071-5/+4
| | | | | | | | | | | | Sort-based shuffle has lower memory usage and seems to outperform hash-based in almost all of our testing. Author: Reynold Xin <rxin@apache.org> Closes #2178 from rxin/sort-shuffle and squashes the following commits: 713d341 [Reynold Xin] Fixed test failures by setting spark.shuffle.compress to the same value as spark.shuffle.spill.compress. 85165e6 [Reynold Xin] Fixed a comment typo. aa0d372 [Reynold Xin] [SPARK-3280] Made sort-based shuffle the default implementation
* [SPARK-3264] Allow users to set executor Spark home in MesosAndrew Or2014-08-281-0/+10
| | | | | | | | | | | | | | The executors and the driver may not share the same Spark home. There is currently one way to set the executor side Spark home in Mesos, through setting `spark.home`. However, this is neither documented nor intuitive. This PR adds a more specific config `spark.mesos.executor.home` and exposes this to the user. liancheng tnachen Author: Andrew Or <andrewor14@gmail.com> Closes #2166 from andrewor14/mesos-spark-home and squashes the following commits: b87965e [Andrew Or] Merge branch 'master' of github.com:apache/spark into mesos-spark-home f6abb2e [Andrew Or] Document spark.mesos.executor.home ca7846d [Andrew Or] Add more specific configuration for executor Spark home in Mesos
* [SPARK-2677] BasicBlockFetchIterator#next can wait foreverKousuke Saruta2014-08-161-0/+9
| | | | | | | | | | | | | | | | | | | Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #1632 from sarutak/SPARK-2677 and squashes the following commits: cddbc7b [Kousuke Saruta] Removed Exception throwing when ConnectionManager#handleMessage receives ack for non-referenced message d3bd2a8 [Kousuke Saruta] Modified configuration.md for spark.core.connection.ack.timeout e85f88b [Kousuke Saruta] Removed useless synchronized blocks 7ed48be [Kousuke Saruta] Modified ConnectionManager to use ackTimeoutMonitor ConnectionManager-wide 9b620a6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 0dd9ad3 [Kousuke Saruta] Modified typo in ConnectionManagerSuite.scala 7cbb8ca [Kousuke Saruta] Modified to match with scalastyle 8a73974 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 ade279a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 0174d6a [Kousuke Saruta] Modified ConnectionManager.scala to handle the case remote Executor cannot ack a454239 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 9b7b7c1 [Kousuke Saruta] (WIP) Modifying ConnectionManager.scala
* [SPARK-3029] Disable local execution of Spark jobs by defaultAaron Davidson2014-08-141-0/+9
| | | | | | | | | | | | | | | Currently, local execution of Spark jobs is only used by take(), and it can be problematic as it can load a significant amount of data onto the driver. The worst case scenarios occur if the RDD is cached (guaranteed to load whole partition), has very large elements, or the partition is just large and we apply a filter with high selectivity or computational overhead. Additionally, jobs that run locally in this manner do not show up in the web UI, and are thus harder to track or understand what is occurring. This PR adds a flag to disable local execution, which is turned OFF by default, with the intention of perhaps eventually removing this functionality altogether. Removing it now is a tougher proposition since it is part of the public runJob API. An alternative solution would be to limit the flag to take()/first() to avoid impacting any external users of this API, but such usage (or, at least, reliance upon the feature) is hopefully minimal. Author: Aaron Davidson <aaron@databricks.com> Closes #1321 from aarondav/allowlocal and squashes the following commits: 136b253 [Aaron Davidson] Fix DAGSchedulerSuite 5599d55 [Aaron Davidson] [RFC] Disable local execution of Spark jobs by default
* [Docs] Add missing <code> tags (minor)Andrew Or2014-08-131-2/+2
| | | | | | | | | | These configs looked inconsistent from the rest. Author: Andrew Or <andrewor14@gmail.com> Closes #1936 from andrewor14/docs-code and squashes the following commits: 15f578a [Andrew Or] Add <code> tag
* [SPARK-2953] Allow using short names for io compression codecsReynold Xin2014-08-121-3/+5
| | | | | | | | | | | | Instead of requiring "org.apache.spark.io.LZ4CompressionCodec", it is easier for users if Spark just accepts "lz4", "lzf", "snappy". Author: Reynold Xin <rxin@apache.org> Closes #1873 from rxin/compressionCodecShortForm and squashes the following commits: 9f50962 [Reynold Xin] Specify short-form compression codec names first. 63f78ee [Reynold Xin] Updated configuration documentation. 47b3848 [Reynold Xin] [SPARK-2953] Allow using short names for io compression codecs
* [SPARK-2635] Fix race condition at SchedulerBackend.isReady in standalone modeli-zhihui2014-08-081-6/+7
| | | | | | | | | | | | | | | | | In SPARK-1946(PR #900), configuration <code>spark.scheduler.minRegisteredExecutorsRatio</code> was introduced. However, in standalone mode, there is a race condition where isReady() can return true because totalExpectedExecutors has not been correctly set. Because expected executors is uncertain in standalone mode, the PR try to use CPU cores(<code>--total-executor-cores</code>) as expected resources to judge whether SchedulerBackend is ready. Author: li-zhihui <zhihui.li@intel.com> Author: Li Zhihui <zhihui.li@intel.com> Closes #1525 from li-zhihui/fixre4s and squashes the following commits: e9a630b [Li Zhihui] Rename variable totalExecutors and clean codes abf4860 [Li Zhihui] Push down variable totalExpectedResources to children classes ca54bd9 [li-zhihui] Format log with String interpolation 88c7dc6 [li-zhihui] Few codes and docs refactor 41cf47e [li-zhihui] Fix race condition at SchedulerBackend.isReady in standalone mode
* SPARK-2787: Make sort-based shuffle write files directly when there's no ↵Matei Zaharia2014-08-071-0/+18
| | | | | | | | | | | | | | | | | | | | sorting/aggregation and # partitions is small As described in https://issues.apache.org/jira/browse/SPARK-2787, right now sort-based shuffle is more expensive than hash-based for map operations that do no partial aggregation or sorting, such as groupByKey. This is because it has to serialize each data item twice (once when spilling to intermediate files, and then again when merging these files object-by-object). This patch adds a code path to just write separate files directly if the # of output partitions is small, and concatenate them at the end to produce a sorted file. On the unit test side, I added some tests that force or don't force this bypass path to be used, and checked that our tests for other features (e.g. all the operations) cover both cases. Author: Matei Zaharia <matei@databricks.com> Closes #1799 from mateiz/SPARK-2787 and squashes the following commits: 88cf26a [Matei Zaharia] Fix rebase 10233af [Matei Zaharia] Review comments 398cb95 [Matei Zaharia] Fix looking up shuffle manager in conf ca3efd9 [Matei Zaharia] Add docs for shuffle manager properties, and allow short names for them d0ae3c5 [Matei Zaharia] Fix some comments 90d084f [Matei Zaharia] Add code path to bypass merge-sort in ExternalSorter, and tests 31e5d7c [Matei Zaharia] Move existing logic for writing partitioned files into ExternalSorter
* [SPARK-2157] Enable tight firewall rules for SparkAndrew Or2014-08-061-0/+46
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | The goal of this PR is to allow users of Spark to write tight firewall rules for their clusters. This is currently not possible because Spark uses random ports in many places, notably the communication between executors and drivers. The changes in this PR are based on top of ash211's changes in #1107. The list covered here may or may not be the complete set of port needed for Spark to operate perfectly. However, as of the latest commit there are no known sources of random ports (except in tests). I have not documented a few of the more obscure configs. My spark-env.sh looks like this: ``` export SPARK_MASTER_PORT=6060 export SPARK_WORKER_PORT=7070 export SPARK_MASTER_WEBUI_PORT=9090 export SPARK_WORKER_WEBUI_PORT=9091 ``` and my spark-defaults.conf looks like this: ``` spark.master spark://andrews-mbp:6060 spark.driver.port 5001 spark.fileserver.port 5011 spark.broadcast.port 5021 spark.replClassServer.port 5031 spark.blockManager.port 5041 spark.executor.port 5051 ``` Author: Andrew Or <andrewor14@gmail.com> Author: Andrew Ash <andrew@andrewash.com> Closes #1777 from andrewor14/configure-ports and squashes the following commits: 621267b [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports 8a6b820 [Andrew Or] Use a random UI port during tests 7da0493 [Andrew Or] Fix tests 523c30e [Andrew Or] Add test for isBindCollision b97b02a [Andrew Or] Minor fixes c22ad00 [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports 93d359f [Andrew Or] Executors connect to wrong port when collision occurs d502e5f [Andrew Or] Handle port collisions when creating Akka systems a2dd05c [Andrew Or] Patrick's comment nit 86461e2 [Andrew Or] Remove spark.executor.env.port and spark.standalone.client.port 1d2d5c6 [Andrew Or] Fix ports for standalone cluster mode cb3be88 [Andrew Or] Various doc fixes (broken link, format etc.) e837cde [Andrew Or] Remove outdated TODOs bfbab28 [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports de1b207 [Andrew Or] Update docs to reflect new ports b565079 [Andrew Or] Add spark.ports.maxRetries 2551eb2 [Andrew Or] Remove spark.worker.watcher.port 151327a [Andrew Or] Merge branch 'master' of github.com:apache/spark into configure-ports 9868358 [Andrew Or] Add a few miscellaneous ports 6016e77 [Andrew Or] Add spark.executor.port 8d836e6 [Andrew Or] Also document SPARK_{MASTER/WORKER}_WEBUI_PORT 4d9e6f3 [Andrew Or] Fix super subtle bug 3f8e51b [Andrew Or] Correct erroneous docs... e111d08 [Andrew Or] Add names for UI services 470f38c [Andrew Or] Special case non-"Address already in use" exceptions 1d7e408 [Andrew Or] Treat 0 ports specially + return correct ConnectionManager port ba32280 [Andrew Or] Minor fixes 6b550b0 [Andrew Or] Assorted fixes 73fbe89 [Andrew Or] Move start service logic to Utils ec676f4 [Andrew Or] Merge branch 'SPARK-2157' of github.com:ash211/spark into configure-ports 038a579 [Andrew Ash] Trust the server start function to report the port the service started on 7c5bdc4 [Andrew Ash] Fix style issue 0347aef [Andrew Ash] Unify port fallback logic to a single place 24a4c32 [Andrew Ash] Remove type on val to match surrounding style 9e4ad96 [Andrew Ash] Reformat for style checker 5d84e0e [Andrew Ash] Document new port configuration options 066dc7a [Andrew Ash] Fix up HttpServer port increments cad16da [Andrew Ash] Add fallover increment logic for HttpServer c5a0568 [Andrew Ash] Fix ConnectionManager to retry with increment b80d2fd [Andrew Ash] Make Spark's block manager port configurable 17c79bb [Andrew Ash] Add a configuration option for spark-shell's class server f34115d [Andrew Ash] SPARK-1176 Add port configuration for HttpBroadcast 49ee29b [Andrew Ash] SPARK-1174 Add port configuration for HttpFileServer 1c0981a [Andrew Ash] Make port in HttpServer configurable
* [SPARK-2503] Lower shuffle output buffer (spark.shuffle.file.buffer.kb) to 32KB.Reynold Xin2014-08-051-1/+1
| | | | | | | | | | This can substantially reduce memory usage during shuffle. Author: Reynold Xin <rxin@apache.org> Closes #1781 from rxin/SPARK-2503-spark.shuffle.file.buffer.kb and squashes the following commits: 104b8d8 [Reynold Xin] [SPARK-2503] Lower shuffle output buffer (spark.shuffle.file.buffer.kb) to 32KB.
* SPARK-1680: use configs for specifying environment variables on YARNThomas Graves2014-08-051-0/+8
| | | | | | | | | | | | | | Note that this also documents spark.executorEnv.* which to me means its public. If we don't want that please speak up. Author: Thomas Graves <tgraves@apache.org> Closes #1512 from tgravescs/SPARK-1680 and squashes the following commits: 11525df [Thomas Graves] more doc changes 553bad0 [Thomas Graves] fix documentation 152bf7c [Thomas Graves] fix docs 5382326 [Thomas Graves] try fix docs 32f86a4 [Thomas Graves] use configs for specifying environment variables on YARN