Commit graph

6437 commits

Author SHA1 Message Date
Kay Ousterhout 18ad59e2c6 Merge pull request #321 from kayousterhout/ui_kill_fix. Closes #321.
Inform DAG scheduler about all started/finished tasks.

Previously, the DAG scheduler was not always informed
when tasks started and finished. The simplest example here
is for speculated tasks: the DAGScheduler was only told about
the first attempt of a task, meaning that SparkListeners were
also not told about multiple task attempts, so users can't see
what's going on with speculation in the UI.  The DAGScheduler
also wasn't always told about finished tasks, so in the UI, some
tasks will never be shown as finished (this occurs, for example,
if a task set gets killed).

The other problem is that the fairness accounting was wrong
-- the number of running tasks in a pool was decreased when a
task set was considered done, even if all of its tasks hadn't
yet finished.

Author: Kay Ousterhout <kayousterhout@gmail.com>

== Merge branch commits ==

commit c8d547d0f7a17f5a193bef05f5872b9f475675c5
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date:   Wed Jan 15 16:47:33 2014 -0800

    Addressed Reynold's review comments.

    Always use a TaskEndReason (remove the option), and explicitly
    signal when we don't know the reason. Also, always tell
    DAGScheduler (and associated listeners) about started tasks, even
    when they're speculated.

commit 3fee1e2e3c06b975ff7f95d595448f38cce97a04
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date:   Wed Jan 8 22:58:13 2014 -0800

    Fixed broken test and improved logging

commit ff12fcaa2567c5d02b75a1d5db35687225bcd46f
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date:   Sun Dec 29 21:08:20 2013 -0800

    Inform DAG scheduler about all finished tasks.

    Previously, the DAG scheduler was not always informed
    when tasks finished. For example, when a task set was
    aborted, the DAG scheduler was never told when the tasks
    in that task set finished. The DAG scheduler was also
    never told about the completion of speculated tasks.
    This led to confusion with SparkListeners because information
    about the completion of those tasks was never passed on to
    the listeners (so in the UI, for example, some tasks will never
    be shown as finished).

    The other problem is that the fairness accounting was wrong
    -- the number of running tasks in a pool was decreased when a
    task set was considered done, even if all of its tasks hadn't
    yet finished.
2014-02-06 16:10:48 -08:00
Sandy Ryza 446403b637 Merge pull request #554 from sryza/sandy-spark-1056. Closes #554.
SPARK-1056. Fix header comment in Executor to not imply that it's only u...

...sed for Mesos and Standalone.

Author: Sandy Ryza <sandy@cloudera.com>

== Merge branch commits ==

commit 1f2443d902a26365a5c23e4af9077e1539ed2eab
Author: Sandy Ryza <sandy@cloudera.com>
Date:   Thu Feb 6 15:03:50 2014 -0800

    SPARK-1056. Fix header comment in Executor to not imply that it's only used for Mesos and Standalone
2014-02-06 15:41:16 -08:00
Prashant Sharma 084839ba35 Merge pull request #498 from ScrapCodes/python-api. Closes #498.
Python api additions

Author: Prashant Sharma <prashant.s@imaginea.com>

== Merge branch commits ==

commit 8b51591f1a7a79a62c13ee66ff8d83040f7eccd8
Author: Prashant Sharma <prashant.s@imaginea.com>
Date:   Fri Jan 24 11:50:29 2014 +0530

    Josh's and Patricks review comments.

commit d37f9677838e43bef6c18ef61fbf08055ba6d1ca
Author: Prashant Sharma <prashant.s@imaginea.com>
Date:   Thu Jan 23 17:27:17 2014 +0530

    fixed doc tests

commit 27cb54bf5c99b1ea38a73858c291d0a1c43d8b7c
Author: Prashant Sharma <prashant.s@imaginea.com>
Date:   Thu Jan 23 16:48:43 2014 +0530

    Added keys and values methods for PairFunctions in python

commit 4ce76b396fbaefef2386d7a36d611572bdef9b5d
Author: Prashant Sharma <prashant.s@imaginea.com>
Date:   Thu Jan 23 13:51:26 2014 +0530

    Added foreachPartition

commit 05f05341a187cba829ac0e6c2bdf30be49948c89
Author: Prashant Sharma <prashant.s@imaginea.com>
Date:   Thu Jan 23 13:02:59 2014 +0530

    Added coalesce fucntion to python API

commit 6568d2c2fa14845dc56322c0f39ba2e13b3b26dd
Author: Prashant Sharma <prashant.s@imaginea.com>
Date:   Thu Jan 23 12:52:44 2014 +0530

    added repartition function to python API.
2014-02-06 14:58:35 -08:00
Kay Ousterhout 79c95527a7 Merge pull request #545 from kayousterhout/fix_progress. Closes #545.
Fix off-by-one error with task progress info log.

Author: Kay Ousterhout <kayousterhout@gmail.com>

== Merge branch commits ==

commit 29798fc685c4e7e3eb3bf91c75df7fa8ec94a235
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date:   Wed Feb 5 13:40:01 2014 -0800

    Fix off-by-one error with task progress info log.
2014-02-05 23:38:12 -08:00
Thomas Graves 38020961d1 Merge pull request #526 from tgravescs/yarn_client_stop_am_fix. Closes #526.
spark on yarn - yarn-client mode doesn't always exit immediately

https://spark-project.atlassian.net/browse/SPARK-1049

If you run in the yarn-client mode but you don't get all the workers you requested right away and then you exit your application, the application master stays around until it gets the number of workers you initially requested. This is a waste of resources.  The AM should exit immediately upon the client going away.

This fix simply checks to see if the driver closed while its waiting for the initial # of workers.

Author: Thomas Graves <tgraves@apache.org>

== Merge branch commits ==

commit 03f40a62584b6bdd094ba91670cd4aa6afe7cd81
Author: Thomas Graves <tgraves@apache.org>
Date:   Fri Jan 31 11:23:10 2014 -0600

    spark on yarn - yarn-client mode doesn't always exit immediately
2014-02-05 23:37:07 -08:00
CodingCat 18c4ee71e2 Merge pull request #549 from CodingCat/deadcode_master. Closes #549.
remove actorToWorker in master.scala, which is actually not used

actorToWorker is actually not used in the code....just remove it

Author: CodingCat <zhunansjtu@gmail.com>

== Merge branch commits ==

commit 52656c2d4bbf9abcd8bef65d454badb9cb14a32c
Author: CodingCat <zhunansjtu@gmail.com>
Date:   Thu Feb 6 00:28:26 2014 -0500

    remove actorToWorker in master.scala, which is actually not used
2014-02-05 22:08:47 -08:00
Kay Ousterhout cc14ba974c Merge pull request #544 from kayousterhout/fix_test_warnings. Closes #544.
Fixed warnings in test compilation.

This commit fixes two problems: a redundant import, and a
deprecated function.

Author: Kay Ousterhout <kayousterhout@gmail.com>

== Merge branch commits ==

commit da9d2e13ee4102bc58888df0559c65cb26232a82
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date:   Wed Feb 5 11:41:51 2014 -0800

    Fixed warnings in test compilation.

    This commit fixes two problems: a redundant import, and a
    deprecated function.
2014-02-05 12:44:24 -08:00
Stevo Slavić f7fd80d9a7 Merge pull request #540 from sslavic/patch-3. Closes #540.
Fix line end character stripping for Windows

LogQuery Spark example would produce unwanted result when run on Windows platform because of different, platform specific trailing line end characters (not only \n but \r too).

This fix makes use of Scala's standard library string functions to properly strip all trailing line end characters, letting Scala handle the platform specific stuff.

Author: Stevo Slavić <sslavic@gmail.com>

== Merge branch commits ==

commit 1e43ba0ea773cc005cf0aef78b6c1755f8e88b27
Author: Stevo Slavić <sslavic@gmail.com>
Date:   Wed Feb 5 14:48:29 2014 +0100

    Fix line end character stripping for Windows

    LogQuery Spark example would produce unwanted result when run on Windows platform because of different, platform specific trailing line end characters (not only \n but \r too).

    This fix makes use of Scala's standard library string functions to properly strip all trailing line end characters, letting Scala handle the platform specific stuff.
2014-02-05 10:29:45 -08:00
Stevo Slavić 92092879c3 Merge pull request #534 from sslavic/patch-1. Closes #534.
Fixed wrong path to compute-classpath.cmd

compute-classpath.cmd is in bin, not in sbin directory

Author: Stevo Slavić <sslavic@gmail.com>

== Merge branch commits ==

commit 23deca32b69e9429b33ad31d35b7e1bfc9459f59
Author: Stevo Slavić <sslavic@gmail.com>
Date:   Tue Feb 4 15:01:47 2014 +0100

    Fixed wrong path to compute-classpath.cmd

    compute-classpath.cmd is in bin, not in sbin directory
2014-02-04 09:47:11 -08:00
Stevo Slavić 0c05cd374d Merge pull request #535 from sslavic/patch-2. Closes #535.
Fixed typo in scaladoc

Author: Stevo Slavić <sslavic@gmail.com>

== Merge branch commits ==

commit 0a77f789e281930f4168543cc0d3b3ffbf5b3764
Author: Stevo Slavić <sslavic@gmail.com>
Date:   Tue Feb 4 15:30:27 2014 +0100

    Fixed typo in scaladoc
2014-02-04 09:45:46 -08:00
Xiangrui Meng 23af00f9e0 Merge pull request #528 from mengxr/sample. Closes #528.
Refactor RDD sampling and add randomSplit to RDD (update)

Replace SampledRDD by PartitionwiseSampledRDD, which accepts a RandomSampler instance as input. The current sample with/without replacement can be easily integrated via BernoulliSampler and PoissonSampler. The benefits are:

1) RDD.randomSplit is implemented in the same way, related to https://github.com/apache/incubator-spark/pull/513
2) Stratified sampling and importance sampling can be implemented in the same manner as well.

Unit tests are included for samplers and RDD.randomSplit.

This should performance better than my previous request where the BernoulliSampler creates many Iterator instances:
https://github.com/apache/incubator-spark/pull/513

Author: Xiangrui Meng <meng@databricks.com>

== Merge branch commits ==

commit e8ce957e5f0a600f2dec057924f4a2ca6adba373
Author: Xiangrui Meng <meng@databricks.com>
Date:   Mon Feb 3 12:21:08 2014 -0800

    more docs to PartitionwiseSampledRDD

commit fbb4586d0478ff638b24bce95f75ff06f713d43b
Author: Xiangrui Meng <meng@databricks.com>
Date:   Mon Feb 3 00:44:23 2014 -0800

    move XORShiftRandom to util.random and use it in BernoulliSampler

commit 987456b0ee8612fd4f73cb8c40967112dc3c4c2d
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sat Feb 1 11:06:59 2014 -0800

    relax assertions in SortingSuite because the RangePartitioner has large variance in this case

commit 3690aae416b2dc9b2f9ba32efa465ba7948477f4
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sat Feb 1 09:56:28 2014 -0800

    test split ratio of RDD.randomSplit

commit 8a410bc933a60c4d63852606f8bbc812e416d6ae
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sat Feb 1 09:25:22 2014 -0800

    add a test to ensure seed distribution and minor style update

commit ce7e866f674c30ab48a9ceb09da846d5362ab4b6
Author: Xiangrui Meng <meng@databricks.com>
Date:   Fri Jan 31 18:06:22 2014 -0800

    minor style change

commit 750912b4d77596ed807d361347bd2b7e3b9b7a74
Author: Xiangrui Meng <meng@databricks.com>
Date:   Fri Jan 31 18:04:54 2014 -0800

    fix some long lines

commit c446a25c38d81db02821f7f194b0ce5ab4ed7ff5
Author: Xiangrui Meng <meng@databricks.com>
Date:   Fri Jan 31 17:59:59 2014 -0800

    add complement to BernoulliSampler and minor style changes

commit dbe2bc2bd888a7bdccb127ee6595840274499403
Author: Xiangrui Meng <meng@databricks.com>
Date:   Fri Jan 31 17:45:08 2014 -0800

    switch to partition-wise sampling for better performance

commit a1fca5232308feb369339eac67864c787455bb23
Merge: ac712e4 cf6128f
Author: Xiangrui Meng <meng@databricks.com>
Date:   Fri Jan 31 16:33:09 2014 -0800

    Merge branch 'sample' of github.com:mengxr/incubator-spark into sample

commit cf6128fb672e8c589615adbd3eaa3cbdb72bd461
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sun Jan 26 14:40:07 2014 -0800

    set SampledRDD deprecated in 1.0

commit f430f847c3df91a3894687c513f23f823f77c255
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sun Jan 26 14:38:59 2014 -0800

    update code style

commit a8b5e2021a9204e318c80a44d00c5c495f1befb6
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sun Jan 26 12:56:27 2014 -0800

    move package random to util.random

commit ab0fa2c4965033737a9e3a9bf0a59cbb0df6a6f5
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sun Jan 26 12:50:35 2014 -0800

    add Apache headers and update code style

commit 985609fe1a55655ad11966e05a93c18c138a403d
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sun Jan 26 11:49:25 2014 -0800

    add new lines

commit b21bddf29850a2c006a868869b8f91960a029322
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sun Jan 26 11:46:35 2014 -0800

    move samplers to random.IndependentRandomSampler and add tests

commit c02dacb4a941618e434cefc129c002915db08be6
Author: Xiangrui Meng <meng@databricks.com>
Date:   Sat Jan 25 15:20:24 2014 -0800

    add RandomSampler

commit 8ff7ba3c5cf1fc338c29ae8b5fa06c222640e89c
Author: Xiangrui Meng <meng@databricks.com>
Date:   Fri Jan 24 13:23:22 2014 -0800

    init impl of IndependentlySampledRDD
2014-02-03 13:02:09 -08:00
Aaron Davidson 1625d8c446 Merge pull request #530 from aarondav/cleanup. Closes #530.
Remove explicit conversion to PairRDDFunctions in cogroup()

As SparkContext._ is already imported, using the implicit conversion appears to make the code much cleaner. Perhaps there was some sinister reason for doing the conversion explicitly, however.

Author: Aaron Davidson <aaron@databricks.com>

== Merge branch commits ==

commit aa4a63f1bfd5b5178fe67364dd7ce4d84c357996
Author: Aaron Davidson <aaron@databricks.com>
Date:   Sun Feb 2 23:48:04 2014 -0800

    Remove explicit conversion to PairRDDFunctions in cogroup()

    As SparkContext._ is already imported, using the implicit conversion
    appears to make the code much cleaner. Perhaps there was some sinister
    reason for doing the converion explicitly, however.
2014-02-03 11:25:39 -08:00
Henry Saputra 0386f42e38 Merge pull request #529 from hsaputra/cleanup_right_arrowop_scala
Change the ⇒ character (maybe from scalariform) to => in Scala code for style consistency

Looks like there are some ⇒ Unicode character (maybe from scalariform) in Scala code.
This PR is to change it to => to get some consistency on the Scala code.

If we want to use ⇒ as default we could use sbt plugin scalariform to make sure all Scala code has ⇒ instead of =>

And remove unused imports found in TwitterInputDStream.scala while I was there =)

Author: Henry Saputra <hsaputra@apache.org>

== Merge branch commits ==

commit 29c1771d346dff901b0b778f764e6b4409900234
Author: Henry Saputra <hsaputra@apache.org>
Date:   Sat Feb 1 22:05:16 2014 -0800

    Change the ⇒ character (maybe from scalariform) to => in Scala code for style consistency.
2014-02-02 21:51:17 -08:00
Ankur Dave a8cf3ec157 Merge pull request #527 from ankurdave/graphx-assembly-pom
Add GraphX to assembly/pom.xml

Author: Ankur Dave <ankurdave@gmail.com>

== Merge branch commits ==

commit bb0b33ef9eb1b3d4a4fc283d9abb2ece4abcac23
Author: Ankur Dave <ankurdave@gmail.com>
Date:   Fri Jan 31 15:24:52 2014 -0800

    Add GraphX to assembly/pom.xml
2014-01-31 16:52:02 -08:00
Reynold Xin ac712e48af Merge pull request #524 from rxin/doc
Added spark.shuffle.file.buffer.kb to configuration doc.

Author: Reynold Xin <rxin@apache.org>

== Merge branch commits ==

commit 0eea1d761ff772ff89be234e1e28035d54e5a7de
Author: Reynold Xin <rxin@apache.org>
Date:   Wed Jan 29 14:40:48 2014 -0800

    Added spark.shuffle.file.buffer.kb to configuration doc.
2014-01-30 09:33:18 -08:00
Erik Selin 0ff38c2220 Merge pull request #494 from tyro89/worker_registration_issue
Issue with failed worker registrations

I've been going through the spark source after having some odd issues with workers dying and not coming back. After some digging (I'm very new to scala and spark) I believe I've found a worker registration issue. It looks to me like a failed registration follows the same code path as a successful registration which end up with workers believing they are connected (since they received a `RegisteredWorker` event) even tho they are not registered on the Master.

This is a quick fix that I hope addresses this issue (assuming I didn't completely miss-read the code and I'm about to look like a silly person :P)

I'm opening this pr now to start a chat with you guys while I do some more testing on my side :)

Author: Erik Selin <erik.selin@jadedpixel.com>

== Merge branch commits ==

commit 973012f8a2dcf1ac1e68a69a2086a1b9a50f401b
Author: Erik Selin <erik.selin@jadedpixel.com>
Date:   Tue Jan 28 23:36:12 2014 -0500

    break logwarning into two lines to respect line character limit.

commit e3754dc5b94730f37e9806974340e6dd93400f85
Author: Erik Selin <erik.selin@jadedpixel.com>
Date:   Tue Jan 28 21:16:21 2014 -0500

    add log warning when worker registration fails due to attempt to re-register on same address.

commit 14baca241fa7823e1213cfc12a3ff2a9b865b1ed
Author: Erik Selin <erik.selin@jadedpixel.com>
Date:   Wed Jan 22 21:23:26 2014 -0500

    address code style comment

commit 71c0d7e6f59cd378d4e24994c21140ab893954ee
Author: Erik Selin <erik.selin@jadedpixel.com>
Date:   Wed Jan 22 16:01:42 2014 -0500

    Make a failed registration not persist, not send a `RegisteredWordker` event and not run `schedule` but rather send a `RegisterWorkerFailed` message to the worker attempting to register.
2014-01-29 12:44:54 -08:00
Tathagata Das 7930209614 Merge pull request #497 from tdas/docs-update
Updated Spark Streaming Programming Guide

Here is the updated version of the Spark Streaming Programming Guide. This is still a work in progress, but the major changes are in place. So feedback is most welcome.

In general, I have tried to make the guide to easier to understand even if the reader does not know much about Spark. The updated website is hosted here -

http://www.eecs.berkeley.edu/~tdas/spark_docs/streaming-programming-guide.html

The major changes are:
- Overview illustrates the usecases of Spark Streaming - various input sources and various output sources
- An example right after overview to quickly give an idea of what Spark Streaming program looks like
- Made Java API and examples a first class citizen like Scala by using tabs to show both Scala and Java examples (similar to AMPCamp tutorial's code tabs)
- Highlighted the DStream operations updateStateByKey and transform because of their powerful nature
- Updated driver node failure recovery text to highlight automatic recovery in Spark standalone mode
- Added information about linking and using the external input sources like Kafka and Flume
- In general, reorganized the sections to better show the Basic section and the more advanced sections like Tuning and Recovery.

Todos:
- Links to the docs of external Kafka, Flume, etc
- Illustrate window operation with figure as well as example.

Author: Tathagata Das <tathagata.das1565@gmail.com>

== Merge branch commits ==

commit 18ff10556570b39d672beeb0a32075215cfcc944
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Tue Jan 28 21:49:30 2014 -0800

    Fixed a lot of broken links.

commit 34a5a6008dac2e107624c7ff0db0824ee5bae45f
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Tue Jan 28 18:02:28 2014 -0800

    Updated github url to use SPARK_GITHUB_URL variable.

commit f338a60ae8069e0a382d2cb170227e5757cc0b7a
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Mon Jan 27 22:42:42 2014 -0800

    More updates based on Patrick and Harvey's comments.

commit 89a81ff25726bf6d26163e0dd938290a79582c0f
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Mon Jan 27 13:08:34 2014 -0800

    Updated docs based on Patricks PR comments.

commit d5b6196b532b5746e019b959a79ea0cc013a8fc3
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Sun Jan 26 20:15:58 2014 -0800

    Added spark.streaming.unpersist config and info on StreamingListener interface.

commit e3dcb46ab83d7071f611d9b5008ba6bc16c9f951
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Sun Jan 26 18:41:12 2014 -0800

    Fixed docs on StreamingContext.getOrCreate.

commit 6c29524639463f11eec721e4d17a9d7159f2944b
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Thu Jan 23 18:49:39 2014 -0800

    Added example and figure for window operations, and links to Kafka and Flume API docs.

commit f06b964a51bb3b21cde2ff8bdea7d9785f6ce3a9
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Wed Jan 22 22:49:12 2014 -0800

    Fixed missing endhighlight tag in the MLlib guide.

commit 036a7d46187ea3f2a0fb8349ef78f10d6c0b43a9
Merge: eab351d a1cd185
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Wed Jan 22 22:17:42 2014 -0800

    Merge remote-tracking branch 'apache/master' into docs-update

commit eab351d05c0baef1d4b549e1581310087158d78d
Author: Tathagata Das <tathagata.das1565@gmail.com>
Date:   Wed Jan 22 22:17:15 2014 -0800

    Update Spark Streaming Programming Guide.
2014-01-28 21:51:05 -08:00
Josh Rosen f8c742ce27 Merge pull request #523 from JoshRosen/SPARK-1043
Switch from MUTF8 to UTF8 in PySpark serializers.

This fixes SPARK-1043, a bug introduced in 0.9.0 where PySpark couldn't serialize strings > 64kB.

This fix was written by @tyro89 and @bouk in #512. This commit squashes and rebases their pull request in order to fix some merge conflicts.
2014-01-28 21:30:20 -08:00
Josh Rosen 1381fc72f7 Switch from MUTF8 to UTF8 in PySpark serializers.
This fixes SPARK-1043, a bug introduced in 0.9.0
where PySpark couldn't serialize strings > 64kB.

This fix was written by @tyro89 and @bouk in #512.
This commit squashes and rebases their pull request
in order to fix some merge conflicts.
2014-01-28 20:20:08 -08:00
Reynold Xin 84670f2715 Merge pull request #466 from liyinan926/file-overwrite-new
Allow files added through SparkContext.addFile() to be overwritten

This is useful for the cases when a file needs to be refreshed and downloaded by the executors periodically. For example, a possible use case is: the driver periodically renews a Hadoop delegation token and writes it to a token file. The token file needs to be downloaded by the executors whenever it gets renewed. However, the current implementation throws an exception when the target file exists and its contents do not match those of the new source. This PR adds an option to allow files to be overwritten to support use cases similar to the above.
2014-01-27 17:08:35 -08:00
Reynold Xin 3d5c03e230 Merge pull request #516 from sarutak/master
modified SparkPluginBuild.scala to use https protocol for accessing gith...

We cannot build Spark behind a proxy although we execute sbt with -Dhttp(s).proxyHost -Dhttp(s).proxyPort -Dhttp(s).proxyUser -Dhttp(s).proxyPassword options.
It's because of using git protocol to clone junit_xml_listener.git.
I could build after modifying SparkPluginBuild.scala.

I reported this issue to JIRA.
https://spark-project.atlassian.net/browse/SPARK-1046
2014-01-27 16:27:01 -08:00
Reynold Xin f16c21e22f Merge pull request #490 from hsaputra/modify_checkoption_with_isdefined
Replace the check for None Option with isDefined and isEmpty in Scala code

Propose to replace the Scala check for Option "!= None" with Option.isDefined and "=== None" with Option.isEmpty.

I think this, using method call if possible then operator function plus argument, will make the Scala code easier to read and understand.

Pass compile and tests.
2014-01-27 14:24:06 -08:00
Sean Owen f67ce3e229 Merge pull request #460 from srowen/RandomInitialALSVectors
Choose initial user/item vectors uniformly on the unit sphere

...rather than within the unit square to possibly avoid bias in the initial state and improve convergence.

The current implementation picks the N vector elements uniformly at random from [0,1). This means they all point into one quadrant of the vector space. As N gets just a little large, the vector tend strongly to point into the "corner", towards (1,1,1...,1). The vectors are not unit vectors either.

I suggest choosing the elements as Gaussian ~ N(0,1) and normalizing. This gets you uniform random choices on the unit sphere which is more what's of interest here. It has worked a little better for me in the past.

This is pretty minor but wanted to warm up suggesting a few tweaks to ALS.
Please excuse my Scala, pretty new to it.

Author: Sean Owen <sowen@cloudera.com>

== Merge branch commits ==

commit 492b13a7469e5a4ed7591ee8e56d8bd7570dfab6
Author: Sean Owen <sowen@cloudera.com>
Date:   Mon Jan 27 08:05:25 2014 +0000

    Style: spaces around binary operators

commit ce2b5b5a4fefa0356875701f668f01f02ba4d87e
Author: Sean Owen <sowen@cloudera.com>
Date:   Sun Jan 19 22:50:03 2014 +0000

    Generate factors with all positive components, per discussion in https://github.com/apache/incubator-spark/pull/460

commit b6f7a8a61643a8209e8bc662e8e81f2d15c710c7
Author: Sean Owen <sowen@cloudera.com>
Date:   Sat Jan 18 15:54:42 2014 +0000

    Choose initial user/item vectors uniformly on the unit sphere rather than within the unit square to possibly avoid bias in the initial state and improve convergence
2014-01-27 11:15:51 -08:00
sarutak 6a5af7b7d8 modified SparkPluginBuild.scala to use https protocol for accessing github. 2014-01-27 17:00:26 +09:00
Reynold Xin c40619d487 Merge pull request #504 from JoshRosen/SPARK-1025
Fix PySpark hang when input files are deleted (SPARK-1025)

This pull request addresses [SPARK-1025](https://spark-project.atlassian.net/browse/SPARK-1025), an issue where PySpark could hang if its input files were deleted.
2014-01-25 22:41:30 -08:00
Reynold Xin c66a2ef1c2 Merge pull request #511 from JoshRosen/SPARK-1040
Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040)

This fixes [SPARK-1040](https://spark-project.atlassian.net/browse/SPARK-1040), an issue where JavaPairRDD.collectAsMap() could sometimes fail with ClassCastException.  I applied the same fix to the Spark Streaming Java APIs.  The commit message describes the fix in more detail.

I also increased the verbosity of JUnit test output under SBT to make it easier to verify that the Java tests are actually running.
2014-01-25 22:36:07 -08:00
Josh Rosen 740e865f40 Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040)
This fixes an issue where collectAsMap() could
fail when called on a JavaPairRDD that was derived
by transforming a non-JavaPairRDD.

The root problem was that we were creating the
JavaPairRDD's ClassTag by casting a
ClassTag[AnyRef] to a ClassTag[Tuple2[K2, V2]].
To fix this, I cast a ClassTag[Tuple2[_, _]]
instead, since this actually produces a ClassTag
of the appropriate type because ClassTags don't
capture type parameters:

scala> implicitly[ClassTag[Tuple2[_, _]]] == implicitly[ClassTag[Tuple2[Int, Int]]]
res8: Boolean = true

scala> implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[Int, Int]]] == implicitly[ClassTag[Tuple2[Int, Int]]]
res9: Boolean = false
2014-01-25 16:41:12 -08:00
Josh Rosen 531d9d7576 Increase JUnit test verbosity under SBT.
Upgrade junit-interface plugin from 0.9 to 0.10.

I noticed that the JavaAPISuite tests didn't
appear to display any output locally or under
Jenkins, making it difficult to know whether they
were running.  This change increases the verbosity
to more closely match the ScalaTest tests.
2014-01-25 16:32:44 -08:00
Patrick Wendell 05be704774 Merge pull request #505 from JoshRosen/SPARK-1026
Deprecate mapPartitionsWithSplit in PySpark (SPARK-1026)

This commit deprecates `mapPartitionsWithSplit` in PySpark (see [SPARK-1026](https://spark-project.atlassian.net/browse/SPARK-1026) and removes the remaining references to it from the docs.
2014-01-23 20:53:18 -08:00
Josh Rosen 4cebb79c9f Deprecate mapPartitionsWithSplit in PySpark.
Also, replace the last reference to it in the docs.

This fixes SPARK-1026.
2014-01-23 20:01:36 -08:00
Patrick Wendell 3d6e754193 Merge pull request #503 from pwendell/master
Fix bug on read-side of external sort when using Snappy.

This case wasn't handled correctly and this patch fixes it.
2014-01-23 19:47:00 -08:00
Patrick Wendell ff44732171 Minor fix 2014-01-23 19:23:12 -08:00
Patrick Wendell c3196171f3 Merge pull request #502 from pwendell/clone-1
Remove Hadoop object cloning and warn users making Hadoop RDD's.

The code introduced in #359 used Hadoop's WritableUtils.clone() to
duplicate objects when reading from Hadoop files. Some users have
reported exceptions when cloning data in various file formats,
including Avro and another custom format.

This patch removes that functionality to ensure stability for the
0.9 release. Instead, it puts a clear warning in the documentation
that copying may be necessary for Hadoop data sets.
2014-01-23 19:11:59 -08:00
Patrick Wendell cad3002fea Merge pull request #501 from JoshRosen/cartesian-rdd-fixes
Fix two bugs in PySpark cartesian(): SPARK-978 and SPARK-1034

This pull request fixes two bugs in PySpark's `cartesian()` method:

- [SPARK-978](https://spark-project.atlassian.net/browse/SPARK-978): PySpark's cartesian method throws ClassCastException exception
- [SPARK-1034](https://spark-project.atlassian.net/browse/SPARK-1034): Py4JException on PySpark Cartesian Result

The JIRAs have more details describing the fixes.
2014-01-23 19:08:34 -08:00
Patrick Wendell 268ecbd231 Minor changes after auditing diff from earlier version 2014-01-23 18:30:11 -08:00
Josh Rosen f83068497b Fix for SPARK-1025: PySpark hang on missing files. 2014-01-23 18:24:51 -08:00
Patrick Wendell c58d4ea3d4 Response to Matei's review 2014-01-23 18:12:40 -08:00
Patrick Wendell 0213b4032a Fix bug on read-side of external sort when using Snappy.
This case wasn't handled correctly and this patch fixes it.
2014-01-23 18:04:55 -08:00
Patrick Wendell 7101017803 Remove Hadoop object cloning and warn users making Hadoop RDD's.
The code introduced in #359 used Hadoop's WritableUtils.clone() to
duplicate objects when reading from Hadoop files. Some users have
reported exceptions when cloning data in verious file formats,
including Avro and another custom format.

This patch removes that functionality to ensure stability for the
0.9 release. Instead, it puts a clear warning in the documentation
that copying may be necessary for Hadoop data sets.
2014-01-23 17:39:23 -08:00
Josh Rosen 61569906cc Fix SPARK-978: ClassCastException in PySpark cartesian. 2014-01-23 15:09:19 -08:00
Josh Rosen 0035dbbc81 Fix SPARK-1034: Py4JException on PySpark Cartesian Result 2014-01-23 13:05:59 -08:00
Josh Rosen fad6aacfb0 Merge pull request #406 from eklavya/master
Extending Java API coverage

Hi,

I have added three new methods to JavaRDD.

Please review and merge.
2014-01-23 11:14:15 -08:00
Reynold Xin a2b47dae66 Merge pull request #499 from jianpingjwang/dev1
Replace commons-math with jblas in SVDPlusPlus
2014-01-23 10:48:26 -08:00
eklavya 60e7457266 fixed ClassTag in mapPartitions 2014-01-23 17:40:36 +05:30
Jianping J Wang 19a01c1b1d Add jblas dependency 2014-01-23 19:54:01 +08:00
Jianping J Wang a5a513e25e Add jblas dependency 2014-01-23 19:48:39 +08:00
Jianping J Wang cc0fd33177 Replace commons-math with jblas 2014-01-23 19:44:30 +08:00
Patrick Wendell a1cd185122 Merge pull request #496 from pwendell/master
Fix bug in worker clean-up in UI

Introduced in d5a96fec (/cc @aarondav).

This should be picked into 0.8 and 0.9 as well. The bug causes old (zombie) workers on a node to not disappear immediately from the UI when a new one registers.
2014-01-22 19:37:29 -08:00
Patrick Wendell 034dce2a7e Merge pull request #447 from CodingCat/SPARK-1027
fix for SPARK-1027

fix for SPARK-1027  (https://spark-project.atlassian.net/browse/SPARK-1027)

FIXES

1. change sparkhome from String to Option(String) in ApplicationDesc

2. remove sparkhome parameter in LaunchExecutor message

3. adjust involved files
2014-01-22 18:58:02 -08:00
Patrick Wendell 6285513147 Fix bug in worker clean-up in UI
Introduced in d5a96fec. This should be picked into 0.8 and 0.9 as well.
2014-01-22 18:19:52 -08:00