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
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
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.
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.
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.
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.
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.
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.
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.
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
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.
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
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.
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.
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
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.
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.
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.
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.
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.
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.
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
Fix graphx Commons Math dependency
`graphx` depends on Commons Math (2.x) in `SVDPlusPlus.scala`. However the module doesn't declare this dependency. It happens to work because it is included by Hadoop artifacts. But, I can tell you this isn't true as of a month or so ago. Building versus recent Hadoop would fail. (That's how we noticed.)
The simple fix is to declare the dependency, as it should be. But it's also worth noting that `commons-math` is the old-ish 2.x line, while `commons-math3` is where newer 3.x releases are. Drop-in replacement, but different artifact and package name. Changing this only usage to `commons-math3` works, tests pass, and isn't surprising that it does, so is probably also worth changing. (A comment in some test code also references `commons-math3`, FWIW.)
It does raise another question though: `mllib` looks like it uses the `jblas` `DoubleMatrix` for general purpose vector/matrix stuff. Should `graphx` really use Commons Math for this? Beyond the tiny scope here but worth asking.
SPARK-1033. Ask for cores in Yarn container requests
Tested on a pseudo-distributed cluster against the Fair Scheduler and observed a worker taking more than a single core.
Fixed bug where task set managers are added to queue twice
@mateiz can you verify that this is a bug and wasn't intentional? (90a04dab8d (diff-7fa4f84a961750c374f2120ca70e96edR551))
This bug leads to a small performance hit because task
set managers will get offered each rejected resource
offer twice, but doesn't lead to any incorrect functionality.
Thanks to @hdc1112 for pointing this out.
Sparse SVD
# Singular Value Decomposition
Given an *m x n* matrix *A*, compute matrices *U, S, V* such that
*A = U * S * V^T*
There is no restriction on m, but we require n^2 doubles to fit in memory.
Further, n should be less than m.
The decomposition is computed by first computing *A^TA = V S^2 V^T*,
computing svd locally on that (since n x n is small),
from which we recover S and V.
Then we compute U via easy matrix multiplication
as *U = A * V * S^-1*
Only singular vectors associated with the largest k singular values
If there are k such values, then the dimensions of the return will be:
* *S* is *k x k* and diagonal, holding the singular values on diagonal.
* *U* is *m x k* and satisfies U^T*U = eye(k).
* *V* is *n x k* and satisfies V^TV = eye(k).
All input and output is expected in sparse matrix format, 0-indexed
as tuples of the form ((i,j),value) all in RDDs.
# Testing
Tests included. They test:
- Decomposition promise (A = USV^T)
- For small matrices, output is compared to that of jblas
- Rank 1 matrix test included
- Full Rank matrix test included
- Middle-rank matrix forced via k included
# Example Usage
import org.apache.spark.SparkContext
import org.apache.spark.mllib.linalg.SVD
import org.apache.spark.mllib.linalg.SparseMatrix
import org.apache.spark.mllib.linalg.MatrixyEntry
// Load and parse the data file
val data = sc.textFile("mllib/data/als/test.data").map { line =>
val parts = line.split(',')
MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble)
}
val m = 4
val n = 4
// recover top 1 singular vector
val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), 1)
println("singular values = " + decomposed.S.data.toArray.mkString)
# Documentation
Added to docs/mllib-guide.md