aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-5363] [PySpark] check ending mark in non-block wayDavies Liu2015-02-161-4/+17
| | | | | | | | | | | | | | | | | | | | | | | There is chance of dead lock that the Python process is waiting for ending mark from JVM, but which is eaten by corrupted stream. This PR checks the ending mark from Python in non-block way, so it will not blocked by Python process. There is a small chance that the ending mark is sent by Python process but not available right now, then Python process will not be used. cc JoshRosen pwendell Author: Davies Liu <davies@databricks.com> Closes #4601 from davies/freeze and squashes the following commits: e15a8c3 [Davies Liu] update logging 890329c [Davies Liu] Merge branch 'freeze' of github.com:davies/spark into freeze 2bd2228 [Davies Liu] add more logging 656d544 [Davies Liu] Update PythonRDD.scala 05e1085 [Davies Liu] check ending mark in non-block way (cherry picked from commit ac6fe67e1d8bf01ee565f9cc09ad48d88a275829) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-5395] [PySpark] fix python process leak while coalesce()Davies Liu2015-02-161-5/+8
| | | | | | | | | | | | | | | | | Currently, the Python process is released into pool only after the task had finished, it cause many process forked if coalesce() is called. This PR will change it to release the process as soon as read all the data from it (finish the partition), then a process could be reused to process multiple partitions in a single task. Author: Davies Liu <davies@databricks.com> Closes #4238 from davies/py_leak and squashes the following commits: ec80a43 [Davies Liu] add @volatile 6da437a [Davies Liu] address comments 24ed322 [Davies Liu] fix python process leak while coalesce() (cherry picked from commit 5c746eedda8cff2fc1692cf6dce376f4b0ca6fac) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-5788] [PySpark] capture the exception in python write threadDavies Liu2015-02-161-2/+2
| | | | | | | | | | | | | | The exception in Python writer thread will shutdown executor. Author: Davies Liu <davies@databricks.com> Closes #4577 from davies/exception and squashes the following commits: eb0ceff [Davies Liu] Update PythonRDD.scala 139b0db [Davies Liu] capture the exception in python write thread (cherry picked from commit b1bd1dd3228ef50fa7310d466afd834b8cb1f22e) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-5361]Multiple Java RDD <-> Python RDD conversions not working correctlyWinston Chen2015-02-162-1/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This is found through reading RDD from `sc.newAPIHadoopRDD` and writing it back using `rdd.saveAsNewAPIHadoopFile` in pyspark. It turns out that whenever there are multiple RDD conversions from JavaRDD to PythonRDD then back to JavaRDD, the exception below happens: ``` 15/01/16 10:28:31 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 7) java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to java.util.ArrayList at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:157) at org.apache.spark.api.python.SerDeUtil$$anonfun$pythonToJava$1$$anonfun$apply$1.apply(SerDeUtil.scala:153) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:308) ``` The test case code below reproduces it: ``` from pyspark.rdd import RDD dl = [ (u'2', {u'director': u'David Lean'}), (u'7', {u'director': u'Andrew Dominik'}) ] dl_rdd = sc.parallelize(dl) tmp = dl_rdd._to_java_object_rdd() tmp2 = sc._jvm.SerDe.javaToPython(tmp) t = RDD(tmp2, sc) t.count() tmp = t._to_java_object_rdd() tmp2 = sc._jvm.SerDe.javaToPython(tmp) t = RDD(tmp2, sc) t.count() # it blows up here during the 2nd time of conversion ``` Author: Winston Chen <wchen@quid.com> Closes #4146 from wingchen/master and squashes the following commits: 903df7d [Winston Chen] SPARK-5361, update to toSeq based on the PR 5d90a83 [Winston Chen] SPARK-5361, make python pretty, so to pass PEP 8 checks 126be6b [Winston Chen] SPARK-5361, add in test case 4cf1187 [Winston Chen] SPARK-5361, add in test case 9f1a097 [Winston Chen] add in tuple handling while converting form python RDD back to JavaRDD (cherry picked from commit 453d7999b88be87bda30d9e73038eb484ee063bd) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-5441][pyspark] Make SerDeUtil PairRDD to Python conversions more robustMichael Nazario2015-02-162-4/+47
| | | | | | | | | | | SerDeUtil.pairRDDToPython and SerDeUtil.pythonToPairRDD now both support empty RDDs by checking the result of take(1) instead of calling first which throws an exception. Author: Michael Nazario <mnazario@palantir.com> Closes #4236 from mnazario/feature/empty-first and squashes the following commits: a531c0c [Michael Nazario] Added regression tests for SPARK-5441 e3b2fb6 [Michael Nazario] Added acceptance of the empty case
* [SPARK-5227] [SPARK-5679] Disable FileSystem cache in ↵Josh Rosen2015-02-131-2/+10
| | | | | | | | | | | | | | | | | WholeTextFileRecordReaderSuite This patch fixes two difficult-to-reproduce Jenkins test failures in InputOutputMetricsSuite (SPARK-5227 and SPARK-5679). The problem was that WholeTextFileRecordReaderSuite modifies the `fs.local.block.size` Hadoop configuration and this change was affecting subsequent test suites due to Hadoop's caching of FileSystem instances (see HADOOP-8490 for more details). The fix implemented here is to disable FileSystem caching in WholeTextFileRecordReaderSuite. Author: Josh Rosen <joshrosen@databricks.com> Closes #4599 from JoshRosen/inputoutputsuite-fix and squashes the following commits: 47dc447 [Josh Rosen] [SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite (cherry picked from commit d06d5ee9b33505774ef1e5becc01b47492f1a2dc) Signed-off-by: Patrick Wendell <patrick@databricks.com>
* Revert "[SPARK-5762] Fix shuffle write time for sort-based shuffle"Andrew Or2015-02-121-3/+0
| | | | This reverts commit 9c5454d06e56917521a15697c36f76a33a94dd1e.
* [SPARK-5762] Fix shuffle write time for sort-based shuffleKay Ousterhout2015-02-121-0/+3
| | | | | | | | | | | | | | | | | mateiz was excluding the time to write this final file from the shuffle write time intentional? Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #4559 from kayousterhout/SPARK-5762 and squashes the following commits: 5c6f3d9 [Kay Ousterhout] Use foreach 94e4237 [Kay Ousterhout] Removed open time metrics added inadvertently ace156c [Kay Ousterhout] Moved metrics to finally block d773276 [Kay Ousterhout] Use nano time 5a59906 [Kay Ousterhout] [SPARK-5762] Fix shuffle write time for sort-based shuffle (cherry picked from commit 47c73d410ab533c3196184d2b6004081e79daeaa) Signed-off-by: Andrew Or <andrew@databricks.com>
* [SPARK-5655] Don't chmod700 application files if running in YARNAndrew Rowson2015-02-121-8/+3
| | | | | | | | | | | | | | | | [Was previously PR4507] As per SPARK-5655, recently committed code chmod 700s all application files created on the local fs by a spark executor. This is both unnecessary and broken on YARN, where files created in the nodemanager's working directory are already owned by the user running the job and the 'yarn' group. Group read permission is also needed for the auxiliary shuffle service to be able to read the files, as this is running as the 'yarn' user. Author: Andrew Rowson <github@growse.com> Closes #4509 from growse/master and squashes the following commits: 7ca993c [Andrew Rowson] Moved chmod700 functionality into Utils.getOrCreateLocalRootDirs f57ce6b [Andrew Rowson] [SPARK-5655] Don't chmod700 application files if running in a YARN container (cherry picked from commit 466b1f671b21f575d28f9c103f51765790914fe3) Signed-off-by: Sean Owen <sowen@cloudera.com>
* [SPARK-5703] AllJobsPage throws empty.max exceptionAndrew Or2015-02-091-1/+3
| | | | | | | | | | | | | | | If you have a `SparkListenerJobEnd` event without the corresponding `SparkListenerJobStart` event, then `JobProgressListener` will create an empty `JobUIData` with an empty `stageIds` list. However, later in `AllJobsPage` we call `stageIds.max`. If this is empty, it will throw an exception. This crashed my history server. Author: Andrew Or <andrew@databricks.com> Closes #4490 from andrewor14/jobs-page-max and squashes the following commits: 21797d3 [Andrew Or] Check nonEmpty before calling max (cherry picked from commit a95ed52157473fb0e42e910ee15270e7f0edf943) Signed-off-by: Andrew Or <andrew@databricks.com>
* [SPARK-5698] Do not let user request negative # of executorsAndrew Or2015-02-091-0/+5
| | | | | | | | | | | | | Otherwise we might crash the ApplicationMaster. Why? Please see https://issues.apache.org/jira/browse/SPARK-5698. sryza I believe this is also relevant in your patch #4168. Author: Andrew Or <andrew@databricks.com> Closes #4483 from andrewor14/da-negative and squashes the following commits: 53ed955 [Andrew Or] Throw IllegalArgumentException instead 0e89fd5 [Andrew Or] Check against negative requests
* [SPARK-5691] Fixing wrong data structure lookup for dupe app registrationmcheah2015-02-091-1/+1
| | | | | | | In Master's registerApplication method, it checks if the application had already registered by examining the addressToWorker hash map. In reality, it should refer to the addressToApp data structure, as this is what really tracks which apps have been registered.
* SPARK-5425: Use synchronised methods in system properties to create SparkConfJacek Lewandowski2015-02-074-6/+43
| | | | | | | | | | | | | | | | | | | SPARK-5425: Fixed usages of system properties This patch fixes few problems caused by the fact that the Scala wrapper over system properties is not thread-safe and is basically invalid because it doesn't take into account the default values which could have been set in the properties object. The problem is fixed by modifying `Utils.getSystemProperties` method so that it uses `stringPropertyNames` method of the `Properties` class, which is thread-safe (internally it creates a defensive copy in a synchronized method) and returns keys of the properties which were set explicitly and which are defined as defaults. The other related problem, which is fixed here. was in `ResetSystemProperties` mix-in. It created a copy of the system properties in the wrong way. This patch also introduces a test case for thread-safeness of SparkConf creation. Refer to the discussion in https://github.com/apache/spark/pull/4220 for more details. Author: Jacek Lewandowski <lewandowski.jacek@gmail.com> Closes #4221 from jacek-lewandowski/SPARK-5425-1.2 and squashes the following commits: 87951a2 [Jacek Lewandowski] SPARK-5425: Modified Utils.getSystemProperties to return a map of all system properties - explicit + defaults 01dd5cb [Jacek Lewandowski] SPARK-5425: Use SerializationUtils to save properties in ResetSystemProperties trait 94aeacf [Jacek Lewandowski] SPARK-5425: Use synchronised methods in system properties to create SparkConf
* [SPARK-4989][CORE] backport for branch-1.2 catch eventlog exception for ↵Zhang, Liye2015-02-061-18/+18
| | | | | | | | | | | | wrong eventlog conf JIRA is [SPARK-4989](https://issues.apache.org/jira/browse/SPARK-4989) Author: Zhang, Liye <liye.zhang@intel.com> Closes #3969 from liyezhang556520/apache-branch-1.2 and squashes the following commits: 5c77e42 [Zhang, Liye] catch eventlog exception for wrong eventlog conf
* [SPARK-4939] revive offers periodically in LocalBackendDavies Liu2015-02-031-1/+10
| | | | | | | | | | | | | | | | | | | | The locality timeout assume that the SchedulerBackend can revive offers periodically, but currently LocalBackend did do that, then some job with mixed locality levels in local mode will hang forever. This PR let LocalBackend revive offers periodically, just like in cluster mode. Author: Davies Liu <davies@databricks.com> Closes #4147 from davies/revive and squashes the following commits: 2acdf9d [Davies Liu] Update LocalBackend.scala 3c8ca7c [Davies Liu] Update LocalBackend.scala d1b60d2 [Davies Liu] address comments from Kay 33ac9bb [Davies Liu] fix build d0da0d5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into revive 6cf5972 [Davies Liu] fix thread-safety ed62a31 [Davies Liu] fix scala style df9008b [Davies Liu] fix typo bfc1396 [Davies Liu] revive offers periodically in LocalBackend
* Preparing development version 1.2.2-SNAPSHOTPatrick Wendell2015-02-031-1/+1
|
* Preparing Spark release v1.2.1-rc3v1.2.1Patrick Wendell2015-02-031-1/+1
|
* Revert "Preparing Spark release v1.2.1-rc2"Patrick Wendell2015-02-021-1/+1
| | | | This reverts commit b77f87673d1f9f03d4c83cf583158227c551359b.
* Revert "Preparing development version 1.2.2-SNAPSHOT"Patrick Wendell2015-02-021-1/+1
| | | | This reverts commit 0a16abadc59082b7d3a24d7f3625236658632813.
* Disabling Utils.chmod700 for WindowsMartin Weindel2015-02-021-1/+3
| | | | | | | | | | | | | | | | | | | | This patch makes Spark 1.2.1rc2 work again on Windows. Without it you get following log output on creating a Spark context: INFO org.apache.spark.SparkEnv:59 - Registering BlockManagerMaster ERROR org.apache.spark.util.Utils:75 - Failed to create local root dir in .... Ignoring this directory. ERROR org.apache.spark.storage.DiskBlockManager:75 - Failed to create any local dir. Author: Martin Weindel <martin.weindel@gmail.com> Author: mweindel <m.weindel@usu-software.de> Closes #4299 from MartinWeindel/branch-1.2 and squashes the following commits: 535cb7f [Martin Weindel] fixed last commit f17072e [Martin Weindel] moved condition to caller to avoid confusion on chmod700() return value 4de5e91 [Martin Weindel] reverted to unix line ends fe2740b [mweindel] moved comment ac4749c [mweindel] fixed chmod700 for Windows
* Preparing development version 1.2.2-SNAPSHOTPatrick Wendell2015-01-281-1/+1
|
* Preparing Spark release v1.2.1-rc2Patrick Wendell2015-01-281-1/+1
|
* Revert "Preparing Spark release v1.2.1-rc1"Patrick Wendell2015-01-271-1/+1
| | | | This reverts commit 3e2d7d310b76c293b9ac787f204e6880f508f6ec.
* Revert "Preparing development version 1.2.2-SNAPSHOT"Patrick Wendell2015-01-271-1/+1
| | | | This reverts commit f53a4319ba5f0843c077e64ae5a41e2fac835a5b.
* Preparing development version 1.2.2-SNAPSHOTPatrick Wendell2015-01-271-1/+1
|
* Preparing Spark release v1.2.1-rc1Patrick Wendell2015-01-271-1/+1
|
* Revert "Preparing Spark release v1.2.1-rc1"Patrick Wendell2015-01-261-1/+1
| | | | This reverts commit e87eb2b42f137c22194cfbca2abf06fecdf943da.
* Revert "Preparing development version 1.2.2-SNAPSHOT"Patrick Wendell2015-01-261-1/+1
| | | | This reverts commit adfed7086f10fa8db4eeac7996c84cf98f625e9a.
* Preparing development version 1.2.2-SNAPSHOTUbuntu2015-01-271-1/+1
|
* Preparing Spark release v1.2.1-rc1Ubuntu2015-01-271-1/+1
|
* Updating versions for Spark 1.2.1Patrick Wendell2015-01-261-1/+1
|
* SPARK-4147 [CORE] Reduce log4j dependencySean Owen2015-01-261-9/+11
| | | | | | | | | | | | | Defer use of log4j class until it's known that log4j 1.2 is being used. This may avoid dealing with log4j dependencies for callers that reroute slf4j to another logging framework. The only change is to push one half of the check in the original `if` condition inside. This is a trivial change, may or may not actually solve a problem, but I think it's all that makes sense to do for SPARK-4147. Author: Sean Owen <sowen@cloudera.com> Closes #4190 from srowen/SPARK-4147 and squashes the following commits: 4e99942 [Sean Owen] Defer use of log4j class until it's known that log4j 1.2 is being used. This may avoid dealing with log4j dependencies for callers that reroute slf4j to another logging framework. (cherry picked from commit 54e7b456dd56c9e52132154e699abca87563465b) Signed-off-by: Patrick Wendell <patrick@databricks.com>
* [SPARK-5355] use j.u.c.ConcurrentHashMap instead of TrieMapDavies Liu2015-01-263-21/+23
| | | | | | | | | | | | | | | | | | | | j.u.c.ConcurrentHashMap is more battle tested. cc rxin JoshRosen pwendell Author: Davies Liu <davies@databricks.com> Closes #4208 from davies/safe-conf and squashes the following commits: c2182dc [Davies Liu] address comments, fix tests 3a1d821 [Davies Liu] fix test da14ced [Davies Liu] Merge branch 'master' of github.com:apache/spark into safe-conf ae4d305 [Davies Liu] change to j.u.c.ConcurrentMap f8fa1cf [Davies Liu] change to TrieMap a1d769a [Davies Liu] make SparkConf thread-safe (cherry picked from commit 142093179a4c40bdd90744191034de7b94a963ff) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* Revert "[SPARK-5344][WebUI] HistoryServer cannot recognize that inprogress ↵Andrew Or2015-01-251-3/+1
| | | | | | file was renamed to completed file" This reverts commit 8f55beeb51e6ea72e63af3f276497f61dd24d09b.
* [SPARK-5344][WebUI] HistoryServer cannot recognize that inprogress file was ↵Kousuke Saruta2015-01-251-1/+3
| | | | | | | | | | | | | | | | | | | renamed to completed file `FsHistoryProvider` tries to update application status but if `checkForLogs` is called before `.inprogress` file is renamed to completed file, the file is not recognized as completed. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #4132 from sarutak/SPARK-5344 and squashes the following commits: 9658008 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5344 d2c72b6 [Kousuke Saruta] Fixed update issue of FsHistoryProvider (cherry picked from commit 8f5c827b01026bf45fc774ed7387f11a941abea8) Signed-off-by: Andrew Or <andrew@databricks.com> Conflicts: core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
* [SPARK-5402] log executor ID at executor-construction timeRyan Williams2015-01-251-5/+8
| | | | | | | | | | | | | | | | also rename "slaveHostname" to "executorHostname" Author: Ryan Williams <ryan.blake.williams@gmail.com> Closes #4195 from ryan-williams/exec and squashes the following commits: e60a7bb [Ryan Williams] log executor ID at executor-construction time (cherry picked from commit aea25482c370fbcf712a464501605bc16ee4ed5d) Signed-off-by: Andrew Or <andrew@databricks.com> Conflicts: core/src/main/scala/org/apache/spark/executor/Executor.scala
* [SPARK-5401] set executor ID before creating MetricsSystemRyan Williams2015-01-252-2/+6
| | | | | | | | Author: Ryan Williams <ryan.blake.williams@gmail.com> Closes #4194 from ryan-williams/metrics and squashes the following commits: 7c5a33f [Ryan Williams] set executor ID before creating MetricsSystem
* [SPARK-5063] More helpful error messages for several invalid operationsJosh Rosen2015-01-234-14/+119
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch adds more helpful error messages for invalid programs that define nested RDDs, broadcast RDDs, perform actions inside of transformations (e.g. calling `count()` from inside of `map()`), and call certain methods on stopped SparkContexts. Currently, these invalid programs lead to confusing NullPointerExceptions at runtime and have been a major source of questions on the mailing list and StackOverflow. In a few cases, I chose to log warnings instead of throwing exceptions in order to avoid any chance that this patch breaks programs that worked "by accident" in earlier Spark releases (e.g. programs that define nested RDDs but never run any jobs with them). In SparkContext, the new `assertNotStopped()` method is used to check whether methods are being invoked on a stopped SparkContext. In some cases, user programs will not crash in spite of calling methods on stopped SparkContexts, so I've only added `assertNotStopped()` calls to methods that always throw exceptions when called on stopped contexts (e.g. by dereferencing a null `dagScheduler` pointer). Author: Josh Rosen <joshrosen@databricks.com> Closes #3884 from JoshRosen/SPARK-5063 and squashes the following commits: a38774b [Josh Rosen] Fix spelling typo a943e00 [Josh Rosen] Convert two exceptions into warnings in order to avoid breaking user programs in some edge-cases. 2d0d7f7 [Josh Rosen] Fix test to reflect 1.2.1 compatibility 3f0ea0c [Josh Rosen] Revert two unintentional formatting changes 8e5da69 [Josh Rosen] Remove assertNotStopped() calls for methods that were sometimes safe to call on stopped SC's in Spark 1.2 8cff41a [Josh Rosen] IllegalStateException fix 6ef68d0 [Josh Rosen] Fix Python line length issues. 9f6a0b8 [Josh Rosen] Add improved error messages to PySpark. 13afd0f [Josh Rosen] SparkException -> IllegalStateException 8d404f3 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5063 b39e041 [Josh Rosen] Fix BroadcastSuite test which broadcasted an RDD 99cc09f [Josh Rosen] Guard against calling methods on stopped SparkContexts. 34833e8 [Josh Rosen] Add more descriptive error message. 57cc8a1 [Josh Rosen] Add error message when directly broadcasting RDD. 15b2e6b [Josh Rosen] [SPARK-5063] Useful error messages for nested RDDs and actions inside of transformations (cherry picked from commit cef1f092a628ac20709857b4388bb10e0b5143b0) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-5355] make SparkConf thread-safeDavies Liu2015-01-211-2/+3
| | | | | | | | | | | | | | | | The SparkConf is not thread-safe, but is accessed by many threads. The getAll() could return parts of the configs if another thread is access it. This PR changes SparkConf.settings to a thread-safe TrieMap. Author: Davies Liu <davies@databricks.com> Closes #4143 from davies/safe-conf and squashes the following commits: f8fa1cf [Davies Liu] change to TrieMap a1d769a [Davies Liu] make SparkConf thread-safe (cherry picked from commit 9bad062268676aaa66dcbddd1e0ab7f2d7742425) Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* Make sure only owner can read / write to directories created for the job.Marcelo Vanzin2015-01-215-53/+66
| | | | | | | Whenever a directory is created by the utility method, immediately restrict its permissions so that only the owner has access to its contents. Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-5006][Deploy]spark.port.maxRetries doesn't workWangTaoTheTonic2015-01-219-21/+25
| | | | | | | | | | | | | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-5006 I think the issue is produced in https://github.com/apache/spark/pull/1777. Not digging mesos's backend yet. Maybe should add same logic either. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits: 8cdf96d [WangTao] indent thing 2d86d65 [WangTaoTheTonic] fix line length 7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor 61a370d [WangTaoTheTonic] some minor fixes bc6e1ec [WangTaoTheTonic] rebase 67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg 29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable 396c226 [WangTaoTheTonic] make the grammar more like scala 191face [WangTaoTheTonic] invalid value name 62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work Conflicts: external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
* [SPARK-4569] Rename 'externalSorting' in AggregatorIlya Ganelin2015-01-211-4/+6
| | | | | | | | | | | | | Hi all - I've renamed the unhelpfully named variable and added a comment clarifying what's actually happening. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #3666 from ilganeli/SPARK-4569B and squashes the following commits: 1810394 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator e2d2092 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator d7cefec [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator 5b3f39c [Ilya Ganelin] [SPARK-4569] Rename in Aggregator
* [SPARK-4759] Fix driver hanging from coalescing partitionsAndrew Or2015-01-212-16/+22
| | | | | | | | | | | | | | | | | | The driver hangs sometimes when we coalesce RDD partitions. See JIRA for more details and reproduction. This is because our use of empty string as default preferred location in `CoalescedRDDPartition` causes the `TaskSetManager` to schedule the corresponding task on host `""` (empty string). The intended semantics here, however, is that the partition does not have a preferred location, and the TSM should schedule the corresponding task accordingly. Author: Andrew Or <andrew@databricks.com> Closes #3633 from andrewor14/coalesce-preferred-loc and squashes the following commits: e520d6b [Andrew Or] Oops 3ebf8bd [Andrew Or] A few comments f370a4e [Andrew Or] Fix tests 2f7dfb6 [Andrew Or] Avoid using empty string as default preferred location (cherry picked from commit 4f93d0cabe5d1fc7c0fd0a33d992fd85df1fecb4) Signed-off-by: Andrew Or <andrew@databricks.com>
* SPARK-4660: Use correct class loader in JavaSerializer (copy of PR #3840...Jacek Lewandowski2015-01-201-1/+1
| | | | | | | | | | | | | ... by Piotr Kolaczkowski) Author: Jacek Lewandowski <lewandowski.jacek@gmail.com> Closes #4113 from jacek-lewandowski/SPARK-4660-master and squashes the following commits: a5e84ca [Jacek Lewandowski] SPARK-4660: Use correct class loader in JavaSerializer (copy of PR #3840 by Piotr Kolaczkowski) (cherry picked from commit c93a57f0d6dc32b127aa68dbe4092ab0b22a9667) Signed-off-by: Patrick Wendell <patrick@databricks.com>
* [SPARK-5201][CORE] deal with int overflow in the ParallelCollectionRDD.slice ↵Ye Xianjin2015-01-163-16/+37
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | method There is an int overflow in the ParallelCollectionRDD.slice method. That's originally reported by SaintBacchus. ``` sc.makeRDD(1 to (Int.MaxValue)).count // result = 0 sc.makeRDD(1 to (Int.MaxValue - 1)).count // result = 2147483646 = Int.MaxValue - 1 sc.makeRDD(1 until (Int.MaxValue)).count // result = 2147483646 = Int.MaxValue - 1 ``` see https://github.com/apache/spark/pull/2874 for more details. This pr try to fix the overflow. However, There's another issue I don't address. ``` val largeRange = Int.MinValue to Int.MaxValue largeRange.length // throws java.lang.IllegalArgumentException: -2147483648 to 2147483647 by 1: seqs cannot contain more than Int.MaxValue elements. ``` So, the range we feed to sc.makeRDD cannot contain more than Int.MaxValue elements. This is the limitation of Scala. However I think we may want to support that kind of range. But the fix is beyond this pr. srowen andrewor14 would you mind take a look at this pr? Author: Ye Xianjin <advancedxy@gmail.com> Closes #4002 from advancedxy/SPARk-5201 and squashes the following commits: 96265a1 [Ye Xianjin] Update slice method comment and some responding docs. e143d7a [Ye Xianjin] Update inclusive range check for splitting inclusive range. b3f5577 [Ye Xianjin] We can include the last element in the last slice in general for inclusive range, hence eliminate the need to check Int.MaxValue or Int.MinValue. 7d39b9e [Ye Xianjin] Convert the two cases pattern matching to one case. 651c959 [Ye Xianjin] rename sign to needsInclusiveRange. add some comments 196f8a8 [Ye Xianjin] Add test cases for ranges end with Int.MaxValue or Int.MinValue e66e60a [Ye Xianjin] Deal with inclusive and exclusive ranges in one case. If the range is inclusive and the end of the range is (Int.MaxValue or Int.MinValue), we should use inclusive range instead of exclusive
* [SPARK-5078] Optionally read from SPARK_LOCAL_HOSTNAMEMichael Armbrust2015-01-121-1/+1
| | | | | | | | | | | | | Current spark lets you set the ip address using SPARK_LOCAL_IP, but then this is given to akka after doing a reverse DNS lookup. This makes it difficult to run spark in Docker. You can already change the hostname that is used programmatically, but it would be nice to be able to do this with an environment variable as well. Author: Michael Armbrust <michael@databricks.com> Closes #3893 from marmbrus/localHostnameEnv and squashes the following commits: 85045b6 [Michael Armbrust] Optionally read from SPARK_LOCAL_HOSTNAME (cherry picked from commit a3978f3e156e0ca67e978f1795b238ddd69ff9a6) Signed-off-by: Patrick Wendell <pwendell@gmail.com>
* [SPARK-5102][Core]subclass of MapStatus needs to be registered with Kryolianhuiwang2015-01-121-2/+3
| | | | | | | | | | | | | | CompressedMapStatus and HighlyCompressedMapStatus needs to be registered with Kryo, because they are subclass of MapStatus. Author: lianhuiwang <lianhuiwang09@gmail.com> Closes #4007 from lianhuiwang/SPARK-5102 and squashes the following commits: 9d2238a [lianhuiwang] remove register of MapStatus 05a285d [lianhuiwang] subclass of MapStatus needs to be registered with Kryo (cherry picked from commit ef9224e08010420b570c21a0b9208d22792a24fe) Signed-off-by: Patrick Wendell <pwendell@gmail.com>
* [SPARK-4951][Core] Fix the issue that a busy executor may be killedzsxwing2015-01-113-45/+144
| | | | | | | | | | | | | | | | | | | | | | | | | A few changes to fix this issue: 1. Handle the case that receiving `SparkListenerTaskStart` before `SparkListenerBlockManagerAdded`. 2. Don't add `executorId` to `removeTimes` when the executor is busy. 3. Use `HashMap.retain` to safely traverse the HashMap and remove items. 4. Use the same lock in ExecutorAllocationManager and ExecutorAllocationListener to fix the race condition in `totalPendingTasks`. 5. Move the blocking codes out of the message processing code in YarnSchedulerActor. Author: zsxwing <zsxwing@gmail.com> Closes #3783 from zsxwing/SPARK-4951 and squashes the following commits: d51fa0d [zsxwing] Add comments 2e365ce [zsxwing] Remove expired executors from 'removeTimes' and add idle executors back when a new executor joins 49f61a9 [zsxwing] Eliminate duplicate executor registered warnings d4c4e9a [zsxwing] Minor fixes for the code style 05f6238 [zsxwing] Move the blocking codes out of the message processing code 105ba3a [zsxwing] Fix the race condition in totalPendingTasks d5c615d [zsxwing] Fix the issue that a busy executor may be killed (cherry picked from commit 6942b974adad396cba2799eac1fa90448cea4da7) Signed-off-by: Andrew Or <andrew@databricks.com>
* [SPARK-4973][CORE] Local directory in the driver of client-mode continues ↵Kousuke Saruta2015-01-081-1/+1
| | | | | | | | | | | | | | | | | | | remaining even if application finished when external shuffle is enabled When we enables external shuffle service, local directories in the driver of client-mode continue remaining even if application has finished. I think local directories for drivers should be deleted. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #3811 from sarutak/SPARK-4973 and squashes the following commits: ad944ab [Kousuke Saruta] Fixed DiskBlockManager to cleanup local directory if it's the driver 43770da [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4973 88feecd [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4973 d99718e [Kousuke Saruta] Fixed SparkSubmit.scala and DiskBlockManager.scala in order to delete local directories of the driver of local-mode when external shuffle service is enabled (cherry picked from commit a00af6bec57b8df8b286aaa5897232475aef441c) Signed-off-by: Andrew Or <andrew@databricks.com>
* [SPARK-5132][Core]Correct stage Attempt Id key in stageInfofromJsonhushan[胡珊]2015-01-071-1/+1
| | | | | | | | | | | | | | | SPARK-5132: stageInfoToJson: Stage Attempt Id stageInfoFromJson: Attempt Id Author: hushan[胡珊] <hushan@xiaomi.com> Closes #3932 from suyanNone/json-stage and squashes the following commits: 41419ab [hushan[胡珊]] Correct stage Attempt Id key in stageInfofromJson (cherry picked from commit d345ebebd554ac3faa4e870bd7800ed02e89da58) Signed-off-by: Josh Rosen <joshrosen@databricks.com>