Commit graph

9451 commits

Author SHA1 Message Date
Fernando Otero (ZeoS) 72df5a301e SPARK-5148 [MLlib] Make usersOut/productsOut storagelevel in ALS configurable
Author: Fernando Otero (ZeoS) <fotero@gmail.com>

Closes #3953 from zeitos/storageLevel and squashes the following commits:

0f070b9 [Fernando Otero (ZeoS)] fix imports
6869e80 [Fernando Otero (ZeoS)] fix comment length
90c9f7e [Fernando Otero (ZeoS)] fix comment length
18a992e [Fernando Otero (ZeoS)] changing storage level
2015-01-08 12:42:54 -08:00
Eric Moyer 538f221627 Document that groupByKey will OOM for large keys
This pull request is my own work and I license it under Spark's open-source license.

This contribution is an improvement to the documentation. I documented that the maximum number of values per key for groupByKey is limited by available RAM (see [Datablox][datablox link] and [the spark mailing list][list link]).

Just saying that better performance is available is not sufficient. Sometimes you need to do a group-by - your operation needs all the items available in order to complete. This warning explains the problem.

[datablox link]: http://databricks.gitbooks.io/databricks-spark-knowledge-base/content/best_practices/prefer_reducebykey_over_groupbykey.html
[list link]: http://apache-spark-user-list.1001560.n3.nabble.com/Understanding-RDD-GroupBy-OutOfMemory-Exceptions-tp11427p11466.html

Author: Eric Moyer <eric_moyer@yahoo.com>

Closes #3936 from RadixSeven/better-group-by-docs and squashes the following commits:

5b6f4e9 [Eric Moyer] groupByKey docs naming updates
238e81b [Eric Moyer] Doc that groupByKey will OOM for large keys
2015-01-08 11:55:23 -08:00
WangTaoTheTonic 0760787da8 [SPARK-5130][Deploy]Take yarn-cluster as cluster mode in spark-submit
https://issues.apache.org/jira/browse/SPARK-5130

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3929 from WangTaoTheTonic/SPARK-5130 and squashes the following commits:

c490648 [WangTaoTheTonic] take yarn-cluster as cluster mode in spark-submit
2015-01-08 11:45:42 -08:00
Kousuke Saruta 0a597276db [Minor] Fix the value represented by spark.executor.id for consistency.
The property  `spark.executor.id` can represent both `driver` and `<driver>`  for one driver.
It's inconsistent.

This issue is minor so I didn't file this in JIRA.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3812 from sarutak/fix-driver-identifier and squashes the following commits:

d885498 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-driver-identifier
4275663 [Kousuke Saruta] Fixed the value represented by spark.executor.id of local mode
2015-01-08 11:35:56 -08:00
Zhang, Liye 06dc4b5206 [SPARK-4989][CORE] avoid wrong eventlog conf cause cluster down in standalone mode
when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers).
How to reproduce: just give an invalid value to "spark.eventLog.dir", for example: spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2. This will throw illegalArgumentException, which will cause the Master restart. And the whole cluster is not available.

Author: Zhang, Liye <liye.zhang@intel.com>

Closes #3824 from liyezhang556520/wrongConf4Cluster and squashes the following commits:

3c24d98 [Zhang, Liye] revert change with logwarning and excetption for FileNotFoundException
3c1ac2e [Zhang, Liye] change var to val
a49c52f [Zhang, Liye] revert wrong modification
12eee85 [Zhang, Liye] add more message in log and on webUI
5c1fa33 [Zhang, Liye] cache exceptions when eventlog with wrong conf
2015-01-08 10:40:26 -08:00
Takeshi Yamamuro f825e193f3 [SPARK-4917] Add a function to convert into a graph with canonical edges in GraphOps
Convert bi-directional edges into uni-directional ones instead of 'canonicalOrientation' in GraphLoader.edgeListFile.
This function is useful when a graph is loaded as it is and then is transformed into one with canonical edges.
It rewrites the vertex ids of edges so that srcIds are bigger than dstIds, and merges the duplicated edges.

Author: Takeshi Yamamuro <linguin.m.s@gmail.com>

Closes #3760 from maropu/ConvertToCanonicalEdgesSpike and squashes the following commits:

7f8b580 [Takeshi Yamamuro] Add a function to convert into a graph with canonical edges in GraphOps
2015-01-08 09:55:12 -08:00
Sandy Ryza 8d45834deb SPARK-5087. [YARN] Merge yarn.Client and yarn.ClientBase
Author: Sandy Ryza <sandy@cloudera.com>

Closes #3896 from sryza/sandy-spark-5087 and squashes the following commits:

65611d0 [Sandy Ryza] Review feedback
3294176 [Sandy Ryza] SPARK-5087. [YARN] Merge yarn.Client and yarn.ClientBase
2015-01-08 09:25:43 -08:00
Patrick Wendell c08238570c MAINTENANCE: Automated closing of pull requests.
This commit exists to close the following pull requests on Github:

Closes #3880 (close requested by 'ash211')
Closes #3649 (close requested by 'marmbrus')
Closes #3791 (close requested by 'mengxr')
Closes #3559 (close requested by 'andrewor14')
Closes #3879 (close requested by 'ash211')
2015-01-07 23:25:56 -08:00
Shuo Xiang c66a976300 [SPARK-5116][MLlib] Add extractor for SparseVector and DenseVector
Add extractor for SparseVector and DenseVector in MLlib to save some code while performing pattern matching on Vectors. For example, previously we may use:

     vec match {
          case dv: DenseVector =>
            val values = dv.values
            ...
          case sv: SparseVector =>
            val indices = sv.indices
            val values = sv.values
            val size = sv.size
            ...
      }

with extractor it is:

    vec match {
        case DenseVector(values) =>
          ...
        case SparseVector(size, indices, values) =>
          ...
    }

Author: Shuo Xiang <shuoxiangpub@gmail.com>

Closes #3919 from coderxiang/extractor and squashes the following commits:

359e8d5 [Shuo Xiang] merge master
ca5fc3e [Shuo Xiang] merge master
0b1e190 [Shuo Xiang] use extractor for vectors in RowMatrix.scala
e961805 [Shuo Xiang] use extractor for vectors in StandardScaler.scala
c2bbdaf [Shuo Xiang] use extractor for vectors in IDFscala
8433922 [Shuo Xiang] use extractor for vectors in NaiveBayes.scala and Normalizer.scala
d83c7ca [Shuo Xiang] use extractor for vectors in Vectors.scala
5523dad [Shuo Xiang] Add extractor for SparseVector and DenseVector
2015-01-07 23:22:37 -08:00
zsxwing 2b729d2250 [SPARK-5126][Core] Verify Spark urls before creating Actors so that invalid urls can crash the process.
Because `actorSelection` will return `deadLetters` for an invalid path,  Worker keeps quiet for an invalid master url. It's better to log an error so that people can find such problem quickly.

This PR will check the url before sending to `actorSelection`, throw and log a SparkException for an invalid url.

Author: zsxwing <zsxwing@gmail.com>

Closes #3927 from zsxwing/SPARK-5126 and squashes the following commits:

9d429ee [zsxwing] Create a utility method in Utils to parse Spark url; verify urls before creating Actors so that invalid urls can crash the process.
8286e51 [zsxwing] Check the url before sending to Akka and log the error if the url is invalid
2015-01-07 23:01:30 -08:00
hushan[胡珊] d345ebebd5 [SPARK-5132][Core]Correct stage Attempt Id key in stageInfofromJson
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
2015-01-07 12:09:12 -08:00
DB Tsai 60e2d9e290 [SPARK-5128][MLLib] Add common used log1pExp API in MLUtils
When `x` is positive and large, computing `math.log(1 + math.exp(x))` will lead to arithmetic
overflow. This will happen when `x > 709.78` which is not a very large number.
It can be addressed by rewriting the formula into `x + math.log1p(math.exp(-x))` when `x > 0`.

Author: DB Tsai <dbtsai@alpinenow.com>

Closes #3915 from dbtsai/mathutil and squashes the following commits:

bec6a84 [DB Tsai] remove empty line
3239541 [DB Tsai] revert part of patch into another PR
23144f3 [DB Tsai] doc
49f3658 [DB Tsai] temp
6c29ed3 [DB Tsai] formating
f8447f9 [DB Tsai] address another overflow issue in gradientMultiplier in LOR gradient code
64eefd0 [DB Tsai] first commit
2015-01-07 10:13:41 -08:00
Masayoshi TSUZUKI 6e74edeca3 [SPARK-2458] Make failed application log visible on History Server
Enabled HistoryServer to show incomplete applications.
We can see the log for incomplete applications by clicking the bottom link.

Author: Masayoshi TSUZUKI <tsudukim@oss.nttdata.co.jp>

Closes #3467 from tsudukim/feature/SPARK-2458-2 and squashes the following commits:

76205d2 [Masayoshi TSUZUKI] Fixed and added test code.
29a04a9 [Masayoshi TSUZUKI] Merge branch 'master' of github.com:tsudukim/spark into feature/SPARK-2458-2
f9ef854 [Masayoshi TSUZUKI] Added space between "if" and "(". Fixed "Incomplete" as capitalized in the web UI. Modified double negative variable name.
9b465b0 [Masayoshi TSUZUKI] Modified typo and better implementation.
3ed8a41 [Masayoshi TSUZUKI] Modified too long lines.
08ea14d [Masayoshi TSUZUKI] [SPARK-2458] Make failed application log visible on History Server
2015-01-07 07:32:53 -08:00
WangTaoTheTonic 8fdd48959c [SPARK-2165][YARN]add support for setting maxAppAttempts in the ApplicationSubmissionContext
...xt

https://issues.apache.org/jira/browse/SPARK-2165

I still have 2 questions:
* If this config is not set, we should use yarn's corresponding value or a default value(like 2) on spark side?
* Is the config name best? Or "spark.yarn.am.maxAttempts"?

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3878 from WangTaoTheTonic/SPARK-2165 and squashes the following commits:

1416c83 [WangTaoTheTonic] use the name spark.yarn.maxAppAttempts
202ac85 [WangTaoTheTonic] rephrase some
afdfc99 [WangTaoTheTonic] more detailed description
91562c6 [WangTaoTheTonic] add support for setting maxAppAttempts in the ApplicationSubmissionContext
2015-01-07 08:14:39 -06:00
huangzhaowei 5fde66163f [YARN][SPARK-4929] Bug fix: fix the yarn-client code to support HA
Nowadays, yarn-client will exit directly when the HA change happens no matter how many times the am should retry.
The reason may be that the default final status only considerred the sys.exit, and the yarn-client HA cann't benefit from this.
So we should distinct the default final status between client and cluster, because the SUCCEEDED status may cause the HA failed in client mode and UNDEFINED may cause the error reporter in cluster when using sys.exit.

Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #3771 from SaintBacchus/YarnHA and squashes the following commits:

c02bfcc [huangzhaowei] Improve the comment of the funciton 'getDefaultFinalStatus'
0e69924 [huangzhaowei] Bug fix: fix the yarn-client code to support HA
2015-01-07 08:10:42 -06:00
Liang-Chi Hsieh e21acc1978 [SPARK-5099][Mllib] Simplify logistic loss function
This is a minor pr where I think that we can simply take minus of `margin`, instead of subtracting  `margin`.

Mathematically, they are equal. But the modified equation is the common form of logistic loss function and so more readable. It also computes more accurate value as some quick tests show.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #3899 from viirya/logit_func and squashes the following commits:

91a3860 [Liang-Chi Hsieh] Modified for comment.
0aa51e4 [Liang-Chi Hsieh] Further simplified.
72a295e [Liang-Chi Hsieh] Revert LogLoss back and add more considerations in Logistic Loss.
a3f83ca [Liang-Chi Hsieh] Fix a bug.
2bc5712 [Liang-Chi Hsieh] Simplify loss function.
2015-01-06 21:23:31 -08:00
Liang-Chi Hsieh bb38ebb1ab [SPARK-5050][Mllib] Add unit test for sqdist
Related to #3643. Follow the previous suggestion to add unit test for `sqdist` in `VectorsSuite`.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #3869 from viirya/sqdist_test and squashes the following commits:

fb743da [Liang-Chi Hsieh] Modified for comment and fix bug.
90a08f3 [Liang-Chi Hsieh] Modified for comment.
39a3ca6 [Liang-Chi Hsieh] Take care of special case.
b789f42 [Liang-Chi Hsieh] More proper unit test with random sparsity pattern.
c36be68 [Liang-Chi Hsieh] Add unit test for sqdist.
2015-01-06 14:00:45 -08:00
Travis Galoppo 4108e5f36f SPARK-5017 [MLlib] - Use SVD to compute determinant and inverse of covariance matrix
MultivariateGaussian was calling both pinv() and det() on the covariance matrix, effectively performing two matrix decompositions.  Both values are now computed using the singular value decompositon. Both the pseudo-inverse and the pseudo-determinant are used to guard against singular matrices.

Author: Travis Galoppo <tjg2107@columbia.edu>

Closes #3871 from tgaloppo/spark-5017 and squashes the following commits:

383b5b3 [Travis Galoppo] MultivariateGaussian - minor optimization in density calculation
a5b8bc5 [Travis Galoppo] Added additional points to tests in test suite. Fixed comment in MultivariateGaussian
629d9d0 [Travis Galoppo] Moved some test values from var to val.
dc3d0f7 [Travis Galoppo] Catch potential exception calculating pseudo-determinant. Style improvements.
d448137 [Travis Galoppo] Added test suite for MultivariateGaussian, including test for degenerate case.
1989be0 [Travis Galoppo] SPARK-5017 - Fixed to use SVD to compute determinant and inverse of covariance matrix.  Previous code called both pinv() and det(), effectively performing two matrix decompositions. Additionally, the pinv() implementation in Breeze is known to fail for singular matrices.
b4415ea [Travis Galoppo] Merge branch 'spark-5017' of https://github.com/tgaloppo/spark into spark-5017
6f11b6d [Travis Galoppo] SPARK-5017 - Use SVD to compute determinant and inverse of covariance matrix. Code was calling both det() and pinv(), effectively performing two matrix decompositions. Futhermore, Breeze pinv() currently fails for singular matrices.
fd9784c [Travis Galoppo] SPARK-5017 - Use SVD to compute determinant and inverse of covariance matrix
2015-01-06 13:57:42 -08:00
Sean Owen 4cba6eb420 SPARK-4159 [CORE] Maven build doesn't run JUnit test suites
This PR:

- Reenables `surefire`, and copies config from `scalatest` (which is itself an old fork of `surefire`, so similar)
- Tells `surefire` to test only Java tests
- Enables `surefire` and `scalatest` for all children, and in turn eliminates some duplication.

For me this causes the Scala and Java tests to be run once each, it seems, as desired. It doesn't affect the SBT build but works for Maven. I still need to verify that all of the Scala tests and Java tests are being run.

Author: Sean Owen <sowen@cloudera.com>

Closes #3651 from srowen/SPARK-4159 and squashes the following commits:

2e8a0af [Sean Owen] Remove specialized SPARK_HOME setting for REPL, YARN tests as it appears to be obsolete
12e4558 [Sean Owen] Append to unit-test.log instead of overwriting, so that both surefire and scalatest output is preserved. Also standardize/correct comments a bit.
e6f8601 [Sean Owen] Reenable Java tests by reenabling surefire with config cloned from scalatest; centralize test config in the parent
2015-01-06 12:02:08 -08:00
kj-ki 5e3ec11104 [Minor] Fix comments for GraphX 2D partitioning strategy
The sum of vertices on matrix (v0 to v11) is 12. And, I think one same block overlaps in this strategy.

This is minor PR, so I didn't file in JIRA.

Author: kj-ki <kikushima.kenji@lab.ntt.co.jp>

Closes #3904 from kj-ki/fix-partitionstrategy-comments and squashes the following commits:

79829d9 [kj-ki] Fix comments for 2D partitioning.
2015-01-06 09:49:37 -08:00
Josh Rosen a6394bc2c0 [SPARK-1600] Refactor FileInputStream tests to remove Thread.sleep() calls and SystemClock usage
This patch refactors Spark Streaming's FileInputStream tests to remove uses of Thread.sleep() and SystemClock, which should hopefully resolve some longstanding flakiness in these tests (see SPARK-1600).

Key changes:

- Modify FileInputDStream to use the scheduler's Clock instead of System.currentTimeMillis(); this allows it to be tested using ManualClock.
- Fix a synchronization issue in ManualClock's `currentTime` method.
- Add a StreamingTestWaiter class which allows callers to block until a certain number of batches have finished.
- Change the FileInputStream tests so that files' modification times are manually set based off of ManualClock; this eliminates many Thread.sleep calls.
- Update these tests to use the withStreamingContext fixture.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3801 from JoshRosen/SPARK-1600 and squashes the following commits:

e4494f4 [Josh Rosen] Address a potential race when setting file modification times
8340bd0 [Josh Rosen] Use set comparisons for output.
0b9c252 [Josh Rosen] Fix some ManualClock usage problems.
1cc689f [Josh Rosen] ConcurrentHashMap -> SynchronizedMap
db26c3a [Josh Rosen] Use standard timeout in ScalaTest `eventually` blocks.
3939432 [Josh Rosen] Rename StreamingTestWaiter to BatchCounter
0b9c3a1 [Josh Rosen] Wait for checkpoint to complete
863d71a [Josh Rosen] Remove Thread.sleep that was used to make task run slowly
b4442c3 [Josh Rosen] batchTimeToSelectedFiles should be thread-safe
15b48ee [Josh Rosen] Replace several TestWaiter methods w/ ScalaTest eventually.
fffc51c [Josh Rosen] Revert "Remove last remaining sleep() call"
dbb8247 [Josh Rosen] Remove last remaining sleep() call
566a63f [Josh Rosen] Fix log message and comment typos
da32f3f [Josh Rosen] Fix log message and comment typos
3689214 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-1600
c8f06b1 [Josh Rosen] Remove Thread.sleep calls in FileInputStream CheckpointSuite test.
d4f2d87 [Josh Rosen] Refactor file input stream tests to not rely on SystemClock.
dda1403 [Josh Rosen] Add StreamingTestWaiter class.
3c3efc3 [Josh Rosen] Synchronize `currentTime` in ManualClock
a95ddc4 [Josh Rosen] Modify FileInputDStream to use Clock class.
2015-01-06 00:31:19 -08:00
Kostas Sakellis 451546aa6d SPARK-4843 [YARN] Squash ExecutorRunnableUtil and ExecutorRunnable
ExecutorRunnableUtil is a parent of ExecutorRunnable because of the yarn-alpha and yarn-stable split. Now that yarn-alpha is gone, this commit squashes the unnecessary hierarchy. The methods from ExecutorRunnableUtil are added as private.

Author: Kostas Sakellis <kostas@cloudera.com>

Closes #3696 from ksakellis/kostas-spark-4843 and squashes the following commits:

486716f [Kostas Sakellis] Moved prepareEnvironment call to after yarnConf declaration
470e22e [Kostas Sakellis] Fixed indentation and renamed sparkConf variable
9b1b1c9 [Kostas Sakellis] SPARK-4843 [YARN] Squash ExecutorRunnableUtil and ExecutorRunnable
2015-01-05 23:26:33 -08:00
Reynold Xin 04d55d8e8e [SPARK-5040][SQL] Support expressing unresolved attributes using $"attribute name" notation in SQL DSL.
Author: Reynold Xin <rxin@databricks.com>

Closes #3862 from rxin/stringcontext-attr and squashes the following commits:

9b10f57 [Reynold Xin] Rename StrongToAttributeConversionHelper
72121af [Reynold Xin] [SPARK-5040][SQL] Support expressing unresolved attributes using $"attribute name" notation in SQL DSL.
2015-01-05 15:34:22 -08:00
Reynold Xin bbcba3a943 [SPARK-5093] Set spark.network.timeout to 120s consistently.
Author: Reynold Xin <rxin@databricks.com>

Closes #3903 from rxin/timeout-120 and squashes the following commits:

7c2138e [Reynold Xin] [SPARK-5093] Set spark.network.timeout to 120s consistently.
2015-01-05 15:19:53 -08:00
freeman 6c6f325740 [SPARK-5089][PYSPARK][MLLIB] Fix vector convert
This is a small change addressing a potentially significant bug in how PySpark + MLlib handles non-float64 numpy arrays. The automatic conversion to `DenseVector` that occurs when passing RDDs to MLlib algorithms in PySpark should automatically upcast to float64s, but currently this wasn't actually happening. As a result, non-float64 would be silently parsed inappropriately during SerDe, yielding erroneous results when running, for example, KMeans.

The PR includes the fix, as well as a new test for the correct conversion behavior.

davies

Author: freeman <the.freeman.lab@gmail.com>

Closes #3902 from freeman-lab/fix-vector-convert and squashes the following commits:

764db47 [freeman] Add a test for proper conversion behavior
704f97e [freeman] Return array after changing type
2015-01-05 13:10:59 -08:00
Jongyoul Lee 1c0e7ce056 [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environme...
...nt at all.

- fixed a scope of runAsSparkUser from MesosExecutorDriver.run to MesosExecutorBackend.launchTask
- See the Jira Issue for more details.

Author: Jongyoul Lee <jongyoul@gmail.com>

Closes #3741 from jongyoul/SPARK-4465 and squashes the following commits:

46ad71e [Jongyoul Lee] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environment at all. - Removed unused import
3d6631f [Jongyoul Lee] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environment at all. - Removed comments and adjusted indentations
2343f13 [Jongyoul Lee] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environment at all. - fixed a scope of runAsSparkUser from MesosExecutorDriver.run to MesosExecutorBackend.launchTask
2015-01-05 12:05:09 -08:00
WangTao ce39b34404 [SPARK-5057] Log message in failed askWithReply attempts
https://issues.apache.org/jira/browse/SPARK-5057

Author: WangTao <barneystinson@aliyun.com>
Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3875 from WangTaoTheTonic/SPARK-5057 and squashes the following commits:

1503487 [WangTao] use string interpolation
706c8a7 [WangTaoTheTonic] log more messages
2015-01-05 12:00:02 -08:00
Varun Saxena d3f07fd23c [SPARK-4688] Have a single shared network timeout in Spark
[SPARK-4688] Have a single shared network timeout in Spark

Author: Varun Saxena <vsaxena.varun@gmail.com>
Author: varunsaxena <vsaxena.varun@gmail.com>

Closes #3562 from varunsaxena/SPARK-4688 and squashes the following commits:

6e97f72 [Varun Saxena] [SPARK-4688] Single shared network timeout
cd783a2 [Varun Saxena] SPARK-4688
d6f8c29 [Varun Saxena] SCALA-4688
9562b15 [Varun Saxena] SPARK-4688
a75f014 [varunsaxena] SPARK-4688
594226c [varunsaxena] SPARK-4688
2015-01-05 10:32:37 -08:00
zsxwing 5c506cecb9 [SPARK-5074][Core] Fix a non-deterministic test failure
Add `assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))` to make sure `sparkListener` receive the message.

Author: zsxwing <zsxwing@gmail.com>

Closes #3889 from zsxwing/SPARK-5074 and squashes the following commits:

e61c198 [zsxwing] Fix a non-deterministic test failure
2015-01-04 21:18:33 -08:00
zsxwing 27e7f5a723 [SPARK-5083][Core] Fix a flaky test in TaskResultGetterSuite
Because `sparkEnv.blockManager.master.removeBlock` is asynchronous, we need to make sure the block has already been removed before calling `super.enqueueSuccessfulTask`.

Author: zsxwing <zsxwing@gmail.com>

Closes #3894 from zsxwing/SPARK-5083 and squashes the following commits:

d97c03d [zsxwing] Fix a flaky test in TaskResultGetterSuite
2015-01-04 21:09:21 -08:00
zsxwing 6c726a3fbd [SPARK-5069][Core] Fix the race condition of TaskSchedulerImpl.dagScheduler
It's not necessary to set `TaskSchedulerImpl.dagScheduler` in preStart. It's safe to set it after `initializeEventProcessActor()`.

Author: zsxwing <zsxwing@gmail.com>

Closes #3887 from zsxwing/SPARK-5069 and squashes the following commits:

d95894f [zsxwing] Fix the race condition of TaskSchedulerImpl.dagScheduler
2015-01-04 21:06:04 -08:00
zsxwing 72396522bc [SPARK-5067][Core] Use '===' to compare well-defined case class
A simple fix would be adding `assert(e1.appId == e2.appId)` for `SparkListenerApplicationStart`. But actually we can use `===` for well-defined case class directly. Therefore, instead of fixing this issue, I use `===` to compare those well-defined case classes (all fields have implemented a correct `equals` method, such as primitive types)

Author: zsxwing <zsxwing@gmail.com>

Closes #3886 from zsxwing/SPARK-5067 and squashes the following commits:

0a51711 [zsxwing] Use '===' to compare well-defined case class
2015-01-04 21:03:17 -08:00
Josh Rosen 939ba1f8f6 [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs
This patch disables output spec. validation for jobs launched through Spark Streaming, since this interferes with checkpoint recovery.

Hadoop OutputFormats have a `checkOutputSpecs` method which performs certain checks prior to writing output, such as checking whether the output directory already exists.  SPARK-1100 added checks for FileOutputFormat, SPARK-1677 (#947) added a SparkConf configuration to disable these checks, and SPARK-2309 (#1088) extended these checks to run for all OutputFormats, not just FileOutputFormat.

In Spark Streaming, we might have to re-process a batch during checkpoint recovery, so `save` actions may be called multiple times.  In addition to `DStream`'s own save actions, users might use `transform` or `foreachRDD` and call the `RDD` and `PairRDD` save actions.  When output spec. validation is enabled, the second calls to these actions will fail due to existing output.

This patch automatically disables output spec. validation for jobs submitted by the Spark Streaming scheduler.  This is done by using Scala's `DynamicVariable` to propagate the bypass setting without having to mutate SparkConf or introduce a global variable.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3832 from JoshRosen/SPARK-4835 and squashes the following commits:

36eaf35 [Josh Rosen] Add comment explaining use of transform() in test.
6485cf8 [Josh Rosen] Add test case in Streaming; fix bug for transform()
7b3e06a [Josh Rosen] Remove Streaming-specific setting to undo this change; update conf. guide
bf9094d [Josh Rosen] Revise disableOutputSpecValidation() comment to not refer to Spark Streaming.
e581d17 [Josh Rosen] Deduplicate isOutputSpecValidationEnabled logic.
762e473 [Josh Rosen] [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs.
2015-01-04 20:26:18 -08:00
bilna e767d7ddac [SPARK-4631] unit test for MQTT
Please review the unit test for MQTT

Author: bilna <bilnap@am.amrita.edu>
Author: Bilna P <bilna.p@gmail.com>

Closes #3844 from Bilna/master and squashes the following commits:

acea3a3 [bilna] Adding dependency with scope test
28681fa [bilna] Merge remote-tracking branch 'upstream/master'
fac3904 [bilna] Correction in Indentation and coding style
ed9db4c [bilna] Merge remote-tracking branch 'upstream/master'
4b34ee7 [Bilna P] Update MQTTStreamSuite.scala
04503cf [bilna] Added embedded broker service for mqtt test
89d804e [bilna] Merge remote-tracking branch 'upstream/master'
fc8eb28 [bilna] Merge remote-tracking branch 'upstream/master'
4b58094 [Bilna P] Update MQTTStreamSuite.scala
b1ac4ad [bilna] Added BeforeAndAfter
5f6bfd2 [bilna] Added BeforeAndAfter
e8b6623 [Bilna P] Update MQTTStreamSuite.scala
5ca6691 [Bilna P] Update MQTTStreamSuite.scala
8616495 [bilna] [SPARK-4631] unit test for MQTT
2015-01-04 19:37:48 -08:00
Dale 3fddc9468f [SPARK-4787] Stop SparkContext if a DAGScheduler init error occurs
Author: Dale <tigerquoll@outlook.com>

Closes #3809 from tigerquoll/SPARK-4787 and squashes the following commits:

5661e01 [Dale] [SPARK-4787] Ensure that call to stop() doesn't lose the exception by using a finally block.
2172578 [Dale] [SPARK-4787] Stop context properly if an exception occurs during DAGScheduler initialization.
2015-01-04 13:29:13 -08:00
Brennon York b96008d552 [SPARK-794][Core] Remove sleep() in ClusterScheduler.stop
Removed `sleep()` from the `stop()` method of the `TaskSchedulerImpl` class which, from the JIRA ticket, is believed to be a legacy artifact slowing down testing originally introduced in the `ClusterScheduler` class.

Author: Brennon York <brennon.york@capitalone.com>

Closes #3851 from brennonyork/SPARK-794 and squashes the following commits:

04c3e64 [Brennon York] Removed sleep() from the stop() method
2015-01-04 12:40:39 -08:00
sigmoidanalytics 342612b65f [SPARK-5058] Updated broken links
Updated the broken link pointing to the KafkaWordCount example to the correct one.

Author: sigmoidanalytics <mayur@sigmoidanalytics.com>

Closes #3877 from sigmoidanalytics/patch-1 and squashes the following commits:

3e19b31 [sigmoidanalytics] Updated broken links
2015-01-03 19:46:08 -08:00
Akhil Das cdccc263b2 Fixed typos in streaming-kafka-integration.md
Changed projrect to project :)

Author: Akhil Das <akhld@darktech.ca>

Closes #3876 from akhld/patch-1 and squashes the following commits:

e0cf9ef [Akhil Das] Fixed typos in streaming-kafka-integration.md
2015-01-02 15:12:27 -08:00
Yadong Qi bd88b71853 [SPARK-3325][Streaming] Add a parameter to the method print in class DStream
This PR is a fixed version of the original PR #3237 by watermen and scwf.
This adds the ability to specify how many elements to print in `DStream.print`.

Author: Yadong Qi <qiyadong2010@gmail.com>
Author: q00251598 <qiyadong@huawei.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: wangfei <wangfei1@huawei.com>

Closes #3865 from tdas/print-num and squashes the following commits:

cd34e9e [Tathagata Das] Fix bug
7c09f16 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into HEAD
bb35d1a [Yadong Qi] Update MimaExcludes.scala
f8098ca [Yadong Qi] Update MimaExcludes.scala
f6ac3cb [Yadong Qi] Update MimaExcludes.scala
e4ed897 [Yadong Qi] Update MimaExcludes.scala
3b9d5cf [wangfei] fix conflicts
ec8a3af [q00251598] move to  Spark 1.3
26a70c0 [q00251598] extend the Python DStream's print
b589a4b [q00251598] add another print function
2015-01-02 15:09:41 -08:00
Josh Rosen 012839807c [HOTFIX] Bind web UI to ephemeral port in DriverSuite
The job launched by DriverSuite should bind the web UI to an ephemeral port, since it looks like port contention in this test has caused a large number of Jenkins failures when many builds are started simultaneously.  Our tests already disable the web UI, but this doesn't affect subprocesses launched by our tests.  In this case, I've opted to bind to an ephemeral port instead of disabling the UI because disabling features in this test may mask its ability to catch certain bugs.

See also: e24d3a9

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3873 from JoshRosen/driversuite-webui-port and squashes the following commits:

48cd05c [Josh Rosen] [HOTFIX] Bind web UI to ephemeral port in DriverSuite.
2015-01-01 15:03:54 -08:00
Reynold Xin 7749dd6c36 [SPARK-5038] Add explicit return type for implicit functions.
As we learned in #3580, not explicitly typing implicit functions can lead to compiler bugs and potentially unexpected runtime behavior.

This is a follow up PR for rest of Spark (outside Spark SQL). The original PR for Spark SQL can be found at https://github.com/apache/spark/pull/3859

Author: Reynold Xin <rxin@databricks.com>

Closes #3860 from rxin/implicit and squashes the following commits:

73702f9 [Reynold Xin] [SPARK-5038] Add explicit return type for implicit functions.
2014-12-31 17:07:47 -08:00
Sean Owen 4bb12488d5 SPARK-2757 [BUILD] [STREAMING] Add Mima test for Spark Sink after 1.10 is released
Re-enable MiMa for Streaming Flume Sink module, now that 1.1.0 is released, per the JIRA TO-DO. That's pretty much all there is to this.

Author: Sean Owen <sowen@cloudera.com>

Closes #3842 from srowen/SPARK-2757 and squashes the following commits:

50ff80e [Sean Owen] Exclude apparent false positive turned up by re-enabling MiMa checks for Streaming Flume Sink
0e5ba5c [Sean Owen] Re-enable MiMa for Streaming Flume Sink module
2014-12-31 16:59:17 -08:00
Josh Rosen fe6efacc0b [SPARK-5035] [Streaming] ReceiverMessage trait should extend Serializable
Spark Streaming's ReceiverMessage trait should extend Serializable in order to fix a subtle bug that only occurs when running on a real cluster:

If you attempt to send a fire-and-forget message to a remote Akka actor and that message cannot be serialized, then this seems to lead to more-or-less silent failures. As an optimization, Akka skips message serialization for messages sent within the same JVM. As a result, Spark's unit tests will never fail due to non-serializable Akka messages, but these will cause mostly-silent failures when running on a real cluster.

Before this patch, here was the code for ReceiverMessage:

```
/** Messages sent to the NetworkReceiver. */
private[streaming] sealed trait ReceiverMessage
private[streaming] object StopReceiver extends ReceiverMessage
```

Since ReceiverMessage does not extend Serializable and StopReceiver is a regular `object`, not a `case object`, StopReceiver will throw serialization errors. As a result, graceful receiver shutdown is broken on real clusters (and local-cluster mode) but works in local modes. If you want to reproduce this, try running the word count example from the Streaming Programming Guide in the Spark shell:

```
import org.apache.spark._
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
val ssc = new StreamingContext(sc, Seconds(10))
// Create a DStream that will connect to hostname:port, like localhost:9999
val lines = ssc.socketTextStream("localhost", 9999)
// Split each line into words
val words = lines.flatMap(_.split(" "))
import org.apache.spark.streaming.StreamingContext._
// Count each word in each batch
val pairs = words.map(word => (word, 1))
val wordCounts = pairs.reduceByKey(_ + _)
// Print the first ten elements of each RDD generated in this DStream to the console
wordCounts.print()
ssc.start()
Thread.sleep(10000)
ssc.stop(true, true)
```

Prior to this patch, this would work correctly in local mode but fail when running against a real cluster (it would report that some receivers were not shut down).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3857 from JoshRosen/SPARK-5035 and squashes the following commits:

71d0eae [Josh Rosen] [SPARK-5035] ReceiverMessage trait should extend Serializable.
2014-12-31 16:02:47 -08:00
Travis Galoppo c4f0b4f334 SPARK-5020 [MLlib] GaussianMixtureModel.predictMembership() should take an RDD only
Removed unnecessary parameters to predictMembership()

CC: jkbradley

Author: Travis Galoppo <tjg2107@columbia.edu>

Closes #3854 from tgaloppo/spark-5020 and squashes the following commits:

1bf4669 [Travis Galoppo] renamed predictMembership() to predictSoft()
0f1d96e [Travis Galoppo] SPARK-5020 - Removed superfluous parameters from predictMembership()
2014-12-31 15:39:58 -08:00
jerryshao fdc2aa4918 [SPARK-5028][Streaming]Add total received and processed records metrics to Streaming UI
This is a follow-up work of [SPARK-4537](https://issues.apache.org/jira/browse/SPARK-4537). Adding total received records and processed records metrics back to UI.

![screenshot](https://dl.dropboxusercontent.com/u/19230832/screenshot.png)

Author: jerryshao <saisai.shao@intel.com>

Closes #3852 from jerryshao/SPARK-5028 and squashes the following commits:

c8c4877 [jerryshao] Add total received and processed metrics to Streaming UI
2014-12-31 14:45:31 -08:00
Hari Shreedharan 3610d3c615 [SPARK-4790][STREAMING] Fix ReceivedBlockTrackerSuite waits for old file...
...s to get deleted before continuing.

Since the deletes are happening asynchronously, the getFileStatus call might throw an exception in older HDFS
versions, if the delete happens between the time listFiles is called on the directory and getFileStatus is called
on the file in the getFileStatus method.

This PR addresses this by adding an option to delete the files synchronously and then waiting for the deletion to
complete before proceeding.

Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #3726 from harishreedharan/spark-4790 and squashes the following commits:

bbbacd1 [Hari Shreedharan] Call cleanUpOldLogs only once in the tests.
3255f17 [Hari Shreedharan] Add test for async deletion. Remove method from ReceiverTracker that does not take waitForCompletion.
e4c83ec [Hari Shreedharan] Making waitForCompletion a mandatory param. Remove eventually from WALSuite since the cleanup method returns only after all files are deleted.
af00fd1 [Hari Shreedharan] [SPARK-4790][STREAMING] Fix ReceivedBlockTrackerSuite waits for old files to get deleted before continuing.
2014-12-31 14:35:07 -08:00
Reynold Xin c88a3d7fca [SPARK-5038][SQL] Add explicit return type for implicit functions in Spark SQL
As we learned in https://github.com/apache/spark/pull/3580, not explicitly typing implicit functions can lead to compiler bugs and potentially unexpected runtime behavior.

Author: Reynold Xin <rxin@databricks.com>

Closes #3859 from rxin/sql-implicits and squashes the following commits:

30c2c24 [Reynold Xin] [SPARK-5038] Add explicit return type for implicit functions in Spark SQL.
2014-12-31 14:25:03 -08:00
Josh Rosen e24d3a9a29 [HOTFIX] Disable Spark UI in SparkSubmitSuite tests
This should fix a major cause of build breaks when running many parallel tests.
2014-12-31 14:13:09 -08:00
Sean Owen 3d194cc757 SPARK-4547 [MLLIB] OOM when making bins in BinaryClassificationMetrics
Now that I've implemented the basics here, I'm less convinced there is a need for this change, somehow. Callers can downsample before or after. Really the OOM is not in the ROC curve code, but in code that might `collect()` it for local analysis. Still, might be useful to down-sample since the ROC curve probably never needs millions of points.

This is a first pass. Since the `(score,label)` are already grouped and sorted, I think it's sufficient to just take every Nth such pair, in order to downsample by a factor of N? this is just like retaining every Nth point on the curve, which I think is the goal. All of the data is still used to build the curve of course.

What do you think about the API, and usefulness?

Author: Sean Owen <sowen@cloudera.com>

Closes #3702 from srowen/SPARK-4547 and squashes the following commits:

1d34d05 [Sean Owen] Indent and reorganize numBins scaladoc
692d825 [Sean Owen] Change handling of large numBins, make 2nd consturctor instead of optional param, style change
a03610e [Sean Owen] Add downsamplingFactor to BinaryClassificationMetrics
2014-12-31 13:37:04 -08:00
Brennon York 8e14c5eb55 [SPARK-4298][Core] - The spark-submit cannot read Main-Class from Manifest.
Resolves a bug where the `Main-Class` from a .jar file wasn't being read in properly. This was caused by the fact that the `primaryResource` object was a URI and needed to be normalized through a call to `.getPath` before it could be passed into the `JarFile` object.

Author: Brennon York <brennon.york@capitalone.com>

Closes #3561 from brennonyork/SPARK-4298 and squashes the following commits:

5e0fce1 [Brennon York] Use string interpolation for error messages, moved comment line from original code to above its necessary code segment
14daa20 [Brennon York] pushed mainClass assignment into match statement, removed spurious spaces, removed { } from case statements, removed return values
c6dad68 [Brennon York] Set case statement to support multiple jar URI's and enabled the 'file' URI to load the main-class
8d20936 [Brennon York] updated to reset the error message back to the default
a043039 [Brennon York] updated to split the uri and jar vals
8da7cbf [Brennon York] fixes SPARK-4298
2014-12-31 11:54:10 -08:00