aboutsummaryrefslogtreecommitdiff
path: root/docs
Commit message (Collapse)AuthorAgeFilesLines
* Merge pull request #533 from andrewor14/master. Closes #533.Andrew Or2014-02-061-3/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | External spilling - generalize batching logic The existing implementation consists of a hack for Kryo specifically and only works for LZF compression. Introducing an intermediate batch-level stream takes care of pre-fetching and other arbitrary behavior of higher level streams in a more general way. Author: Andrew Or <andrewor14@gmail.com> == Merge branch commits == commit 3ddeb7ef89a0af2b685fb5d071aa0f71c975cc82 Author: Andrew Or <andrewor14@gmail.com> Date: Wed Feb 5 12:09:32 2014 -0800 Also privatize fields commit 090544a87a0767effd0c835a53952f72fc8d24f0 Author: Andrew Or <andrewor14@gmail.com> Date: Wed Feb 5 10:58:23 2014 -0800 Privatize methods commit 13920c918efe22e66a1760b14beceb17a61fd8cc Author: Andrew Or <andrewor14@gmail.com> Date: Tue Feb 4 16:34:15 2014 -0800 Update docs commit bd5a1d7350467ed3dc19c2de9b2c9f531f0e6aa3 Author: Andrew Or <andrewor14@gmail.com> Date: Tue Feb 4 13:44:24 2014 -0800 Typo: phyiscal -> physical commit 287ef44e593ad72f7434b759be3170d9ee2723d2 Author: Andrew Or <andrewor14@gmail.com> Date: Tue Feb 4 13:38:32 2014 -0800 Avoid reading the entire batch into memory; also simplify streaming logic Additionally, address formatting comments. commit 3df700509955f7074821e9aab1e74cb53c58b5a5 Merge: a531d2e 164489d Author: Andrew Or <andrewor14@gmail.com> Date: Mon Feb 3 18:27:49 2014 -0800 Merge branch 'master' of github.com:andrewor14/incubator-spark commit a531d2e347acdcecf2d0ab72cd4f965ab5e145d8 Author: Andrew Or <andrewor14@gmail.com> Date: Mon Feb 3 18:18:04 2014 -0800 Relax assumptions on compressors and serializers when batching This commit introduces an intermediate layer of an input stream on the batch level. This guards against interference from higher level streams (i.e. compression and deserialization streams), especially pre-fetching, without specifically targeting particular libraries (Kryo) and forcing shuffle spill compression to use LZF. commit 164489d6f176bdecfa9dabec2dfce5504d1ee8af Author: Andrew Or <andrewor14@gmail.com> Date: Mon Feb 3 18:18:04 2014 -0800 Relax assumptions on compressors and serializers when batching This commit introduces an intermediate layer of an input stream on the batch level. This guards against interference from higher level streams (i.e. compression and deserialization streams), especially pre-fetching, without specifically targeting particular libraries (Kryo) and forcing shuffle spill compression to use LZF.
* Merge pull request #524 from rxin/docReynold Xin2014-01-301-0/+8
| | | | | | | | | | | | | | 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.
* Merge pull request #497 from tdas/docs-updateTathagata Das2014-01-2819-300/+1163
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* Merge pull request #466 from liyinan926/file-overwrite-newReynold Xin2014-01-271-0/+7
|\ | | | | | | | | | | 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.
| * Addressed comments from ReynoldYinan Li2014-01-181-1/+0
| | | | | | | | Signed-off-by: Yinan Li <liyinan926@gmail.com>
| * Allow files added through SparkContext.addFile() to be overwrittenYinan Li2014-01-181-0/+8
| | | | | | | | | | | | | | This is useful for the cases when a file needs to be refreshed and downloaded by the executors periodically. Signed-off-by: Yinan Li <liyinan926@gmail.com>
* | Deprecate mapPartitionsWithSplit in PySpark.Josh Rosen2014-01-231-2/+2
| | | | | | | | | | | | Also, replace the last reference to it in the docs. This fixes SPARK-1026.
* | Merge pull request #478 from sryza/sandy-spark-1033Patrick Wendell2014-01-221-1/+1
|\ \ | | | | | | | | | | | | | | | 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.
| * | Incorporate Tom's comments - update doc and code to reflect that core ↵Sandy Ryza2014-01-211-1/+1
| | | | | | | | | | | | requests may not always be honored
| * | SPARK-1033. Ask for cores in Yarn container requestsSandy Ryza2014-01-201-1/+1
| | |
* | | Merge pull request #315 from rezazadeh/sparsesvdMatei Zaharia2014-01-221-0/+51
|\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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
| * \ \ Merge remote-tracking branch 'upstream/master' into sparsesvdReza Zadeh2014-01-177-137/+228
| |\ \ \ | | | |/ | | |/|
| * | | 0index docsReza Zadeh2014-01-171-1/+1
| | | |
| * | | use 0-indexingReza Zadeh2014-01-171-2/+2
| | | |
| * | | changes from PRReza Zadeh2014-01-171-2/+3
| | | |
| * | | Merge remote-tracking branch 'upstream/master' into sparsesvdReza Zadeh2014-01-1324-22/+1054
| |\ \ \
| * \ \ \ Merge remote-tracking branch 'upstream/master' into sparsesvdReza Zadeh2014-01-112-5/+35
| |\ \ \ \
| * \ \ \ \ Merge remote-tracking branch 'upstream/master' into sparsesvdReza Zadeh2014-01-0915-149/+433
| |\ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | Conflicts: docs/mllib-guide.md
| * | | | | | documentation for sparsematrixReza Zadeh2014-01-071-3/+4
| | | | | | |
| * | | | | | fix docs to use SparseMatrixReza Zadeh2014-01-051-2/+5
| | | | | | |
| * | | | | | add k parameterReza Zadeh2014-01-041-2/+3
| | | | | | |
| * | | | | | Merge remote-tracking branch 'upstream/master' into sparsesvdReza Zadeh2014-01-0212-75/+154
| |\ \ \ \ \ \
| * | | | | | | doc tweaksReza Zadeh2014-01-011-1/+1
| | | | | | | |
| * | | | | | | doc tweakReza Zadeh2014-01-011-4/+5
| | | | | | | |
| * | | | | | | doc tweaksReza Zadeh2014-01-011-4/+4
| | | | | | | |
| * | | | | | | New documentationReza Zadeh2014-01-011-0/+52
| | | | | | | |
* | | | | | | | Clarify spark.default.parallelismAndrew Ash2014-01-211-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | It's the task count across the cluster, not per worker, per machine, per core, or anything else.
* | | | | | | | Force use of LZF when spilling dataPatrick Wendell2014-01-201-1/+3
| | | | | | | |
* | | | | | | | Removing docs on akka optionsPatrick Wendell2014-01-201-7/+0
| |_|_|_|_|_|/ |/| | | | | |
* | | | | | | Merge pull request #462 from mateiz/conf-file-fixPatrick Wendell2014-01-181-26/+2
| |_|_|_|_|/ |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Remove Typesafe Config usage and conf files to fix nested property names With Typesafe Config we had the subtle problem of no longer allowing nested property names, which are used for a few of our properties: http://apache-spark-developers-list.1001551.n3.nabble.com/Config-properties-broken-in-master-td208.html This PR is for branch 0.9 but should be added into master too. (cherry picked from commit 34e911ce9a9f91f3259189861779032069257852) Signed-off-by: Patrick Wendell <pwendell@gmail.com>
* | | | | | Merge pull request #439 from CrazyJvm/masterReynold Xin2014-01-151-2/+1
|\ \ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SPARK-1024 Remove "-XX:+UseCompressedStrings" option from tuning guide remove "-XX:+UseCompressedStrings" option from tuning guide since jdk7 no longer supports this.
| * | | | | | remove "-XX:+UseCompressedStrings" optionCrazyJvm2014-01-151-2/+1
| | | | | | | | | | | | | | | | | | | | | remove "-XX:+UseCompressedStrings" option from tuning guide since jdk7 no longer supports this.
* | | | | | | Clarify that Python 2.7 is only needed for MLlibMatei Zaharia2014-01-152-3/+4
| | | | | | |
* | | | | | | Merge pull request #433 from markhamstra/debFixPatrick Wendell2014-01-151-3/+3
|\ \ \ \ \ \ \ | |/ / / / / / |/| | | | | | | | | | | | | Updated Debian packaging
| * | | | | | Removed repl-bin and updated maven build doc.Mark Hamstra2014-01-141-3/+3
| | | | | | |
* | | | | | | Merge pull request #436 from ankurdave/VertexId-caseReynold Xin2014-01-141-35/+35
|\ \ \ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | Rename VertexID -> VertexId in GraphX
| * | | | | | | VertexID -> VertexIdAnkur Dave2014-01-141-35/+35
| | | | | | | |
* | | | | | | | Merge pull request #424 from jegonzal/GraphXProgrammingGuideReynold Xin2014-01-141-52/+121
|\ \ \ \ \ \ \ \ | |/ / / / / / / |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Additional edits for clarity in the graphx programming guide. Added an overview of the Graph and GraphOps functions and fixed numerous typos.
| * | | | | | | Additional edits for clarity in the graphx programming guide.Joseph E. Gonzalez2014-01-141-52/+121
| |/ / / / / /
* / / / / / / Describe GraphX caching and uncaching in guideAnkur Dave2014-01-141-1/+10
|/ / / / / /
* | | | | | Improving the graphx-programming-guide.Joseph E. Gonzalez2014-01-141-26/+37
| | | | | |
* | | | | | Merge pull request #416 from tdas/filestream-fixPatrick Wendell2014-01-141-15/+17
|\ \ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Removed unnecessary DStream operations and updated docs Removed StreamingContext.registerInputStream and registerOutputStream - they were useless. InputDStream has been made to register itself, and just registering a DStream as output stream cause RDD objects to be created but the RDDs will not be computed at all.. Also made DStream.register() private[streaming] for the same reasons. Updated docs, specially added package documentation for streaming package. Also, changed NetworkWordCount's input storage level to use MEMORY_ONLY, replication on the local machine causes warning messages (as replication fails) which is scary for a new user trying out his/her first example.
| * | | | | | Fixed loose ends in docs.Tathagata Das2014-01-141-2/+2
| | | | | | |
| * | | | | | Merge remote-tracking branch 'apache/master' into filestream-fixTathagata Das2014-01-1323-18/+1050
| |\ \ \ \ \ \ | | | |_|_|_|/ | | |/| | | | | | | | | | | | | | | | | | Conflicts: streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
| * | | | | | Removed StreamingContext.registerInputStream and registerOutputStream - they ↵Tathagata Das2014-01-131-15/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | were useless as InputDStream has been made to register itself. Also made DStream.register() private[streaming] - not useful to expose the confusing function. Updated a lot of documentation.
* | | | | | | Enable compression by default for spillsPatrick Wendell2014-01-131-1/+1
| |/ / / / / |/| | | | |
* | | | | | Merge pull request #380 from mateiz/py-bayesPatrick Wendell2014-01-133-7/+22
|\ \ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Add Naive Bayes to Python MLlib, and some API fixes - Added a Python wrapper for Naive Bayes - Updated the Scala Naive Bayes to match the style of our other algorithms better and in particular make it easier to call from Java (added builder pattern, removed default value in train method) - Updated Python MLlib functions to not require a SparkContext; we can get that from the RDD the user gives - Added a toString method in LabeledPoint - Made the Python MLlib tests run as part of run-tests as well (before they could only be run individually through each file)
| * | | | | | Fix Scala version in docs (it was printed as 2.1)Matei Zaharia2014-01-121-1/+1
| | | | | | |
| * | | | | | Update Python required version to 2.7, and mention MLlib supportMatei Zaharia2014-01-121-1/+7
| | | | | | |
| * | | | | | Update some Python MLlib parameters to use camelCase, and tweak docsMatei Zaharia2014-01-111-0/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | We've used camel case in other Spark methods so it felt reasonable to keep using it here and make the code match Scala/Java as much as possible. Note that parameter names matter in Python because it allows passing optional parameters by name.