Commit graph

858 commits

Author SHA1 Message Date
jerryshao 468ad0ae87 [SPARK-11457][STREAMING][YARN] Fix incorrect AM proxy filter conf recovery from checkpoint
Currently Yarn AM proxy filter configuration is recovered from checkpoint file when Spark Streaming application is restarted, which will lead to some unwanted behaviors:

1. Wrong RM address if RM is redeployed from failure.
2. Wrong proxyBase, since app id is updated, old app id for proxyBase is wrong.

So instead of recovering from checkpoint file, these configurations should be reloaded each time when app started.

This problem only exists in Yarn cluster mode, for Yarn client mode, these configurations will be updated with RPC message `AddWebUIFilter`.

Please help to review tdas harishreedharan vanzin , thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #9412 from jerryshao/SPARK-11457.
2015-11-05 18:03:12 -08:00
Sean Owen 6f81eae24f [SPARK-11440][CORE][STREAMING][BUILD] Declare rest of @Experimental items non-experimental if they've existed since 1.2.0
Remove `Experimental` annotations in core, streaming for items that existed in 1.2.0 or before. The changes are:

* SparkContext
  * binary{Files,Records} : 1.2.0
  * submitJob : 1.0.0
* JavaSparkContext
  * binary{Files,Records} : 1.2.0
* DoubleRDDFunctions, JavaDoubleRDD
  * {mean,sum}Approx : 1.0.0
* PairRDDFunctions, JavaPairRDD
  * sampleByKeyExact : 1.2.0
  * countByKeyApprox : 1.0.0
* PairRDDFunctions
  * countApproxDistinctByKey : 1.1.0
* RDD
  * countApprox, countByValueApprox, countApproxDistinct : 1.0.0
* JavaRDDLike
  * countApprox : 1.0.0
* PythonHadoopUtil.Converter : 1.1.0
* PortableDataStream : 1.2.0 (related to binaryFiles)
* BoundedDouble : 1.0.0
* PartialResult : 1.0.0
* StreamingContext, JavaStreamingContext
  * binaryRecordsStream : 1.2.0
* HiveContext
  * analyze : 1.2.0

Author: Sean Owen <sowen@cloudera.com>

Closes #9396 from srowen/SPARK-11440.
2015-11-05 09:08:53 +00:00
zsxwing 9fbd75ab5d [SPARK-11212][CORE][STREAMING] Make preferred locations support ExecutorCacheTaskLocation and update…
… ReceiverTracker and ReceiverSchedulingPolicy to use it

This PR includes the following changes:

1. Add a new preferred location format, `executor_<host>_<executorID>` (e.g., "executor_localhost_2"), to support specifying the executor locations for RDD.
2. Use the new preferred location format in `ReceiverTracker` to optimize the starting time of Receivers when there are multiple executors in a host.

The goal of this PR is to enable the streaming scheduler to place receivers (which run as tasks) in specific executors. Basically, I want to have more control on the placement of the receivers such that they are evenly distributed among the executors. We tried to do this without changing the core scheduling logic. But it does not allow specifying particular executor as preferred location, only at the host level. So if there are two executors in the same host, and I want two receivers to run on them (one on each executor), I cannot specify that. Current code only specifies the host as preference, which may end up launching both receivers on the same executor. We try to work around it but restarting a receiver when it does not launch in the desired executor and hope that next time it will be started in the right one. But that cause lots of restarts, and delays in correctly launching the receiver.

So this change, would allow the streaming scheduler to specify the exact executor as the preferred location. Also this is not exposed to the user, only the streaming scheduler uses this.

Author: zsxwing <zsxwing@gmail.com>

Closes #9181 from zsxwing/executor-location.
2015-10-27 16:14:33 -07:00
Burak Yavuz 4f030b9e82 [SPARK-11324][STREAMING] Flag for closing Write Ahead Logs after a write
Currently the Write Ahead Log in Spark Streaming flushes data as writes need to be made. S3 does not support flushing of data, data is written once the stream is actually closed.
In case of failure, the data for the last minute (default rolling interval) will not be properly written. Therefore we need a flag to close the stream after the write, so that we achieve read after write consistency.

cc tdas zsxwing

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #9285 from brkyvz/caw-wal.
2015-10-27 16:01:26 -07:00
maxwell 17f4999207 [SPARK-5569][STREAMING] fix ObjectInputStreamWithLoader for supporting load array classes.
When use Kafka DirectStream API to create checkpoint and restore saved checkpoint when restart,
ClassNotFound exception would occur.

The reason for this error is that ObjectInputStreamWithLoader extends the ObjectInputStream class and override its resolveClass method. But Instead of Using Class.forName(desc,false,loader), Spark uses loader.loadClass(desc) to instance the class, which do not works with array class.

For example:
Class.forName("[Lorg.apache.spark.streaming.kafka.OffsetRange.",false,loader) works well while loader.loadClass("[Lorg.apache.spark.streaming.kafka.OffsetRange") would throw an class not found exception.

details of the difference between Class.forName and loader.loadClass can be found here.
http://bugs.java.com/view_bug.do?bug_id=6446627

Author: maxwell <maxwellzdm@gmail.com>
Author: DEMING ZHU <deming.zhu@linecorp.com>

Closes #8955 from maxwellzdm/master.
2015-10-27 01:31:28 -07:00
Josh Rosen 85e654c5ec [SPARK-10984] Simplify *MemoryManager class structure
This patch refactors the MemoryManager class structure. After #9000, Spark had the following classes:

- MemoryManager
- StaticMemoryManager
- ExecutorMemoryManager
- TaskMemoryManager
- ShuffleMemoryManager

This is fairly confusing. To simplify things, this patch consolidates several of these classes:

- ShuffleMemoryManager and ExecutorMemoryManager were merged into MemoryManager.
- TaskMemoryManager is moved into Spark Core.

**Key changes and tasks**:

- [x] Merge ExecutorMemoryManager into MemoryManager.
  - [x] Move pooling logic into Allocator.
- [x] Move TaskMemoryManager from `spark-unsafe` to `spark-core`.
- [x] Refactor the existing Tungsten TaskMemoryManager interactions so Tungsten code use only this and not both this and ShuffleMemoryManager.
- [x] Refactor non-Tungsten code to use the TaskMemoryManager instead of ShuffleMemoryManager.
- [x] Merge ShuffleMemoryManager into MemoryManager.
  - [x] Move code
  - [x] ~~Simplify 1/n calculation.~~ **Will defer to followup, since this needs more work.**
- [x] Port ShuffleMemoryManagerSuite tests.
- [x] Move classes from `unsafe` package to `memory` package.
- [ ] Figure out how to handle the hacky use of the memory managers in HashedRelation's broadcast variable construction.
- [x] Test porting and cleanup: several tests relied on mock functionality (such as `TestShuffleMemoryManager.markAsOutOfMemory`) which has been changed or broken during the memory manager consolidation
  - [x] AbstractBytesToBytesMapSuite
  - [x] UnsafeExternalSorterSuite
  - [x] UnsafeFixedWidthAggregationMapSuite
  - [x] UnsafeKVExternalSorterSuite

**Compatiblity notes**:

- This patch introduces breaking changes in `ExternalAppendOnlyMap`, which is marked as `DevloperAPI` (likely for legacy reasons): this class now cannot be used outside of a task.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9127 from JoshRosen/SPARK-10984.
2015-10-25 21:19:52 -07:00
zsxwing 67582132bf [SPARK-11063] [STREAMING] Change preferredLocations of Receiver's RDD to hosts rather than hostports
The format of RDD's preferredLocations must be hostname but the format of Streaming Receiver's scheduling executors is hostport. So it doesn't work.

This PR converts `schedulerExecutors` to `hosts` before creating Receiver's RDD.

Author: zsxwing <zsxwing@gmail.com>

Closes #9075 from zsxwing/SPARK-11063.
2015-10-19 15:35:14 -07:00
zsxwing e1eef248f1 [SPARK-11104] [STREAMING] Fix a deadlock in StreamingContex.stop
The following deadlock may happen if shutdownHook and StreamingContext.stop are running at the same time.
```
Java stack information for the threads listed above:
===================================================
"Thread-2":
	at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:699)
	- waiting to lock <0x00000005405a1680> (a org.apache.spark.streaming.StreamingContext)
	at org.apache.spark.streaming.StreamingContext.org$apache$spark$streaming$StreamingContext$$stopOnShutdown(StreamingContext.scala:729)
	at org.apache.spark.streaming.StreamingContext$$anonfun$start$1.apply$mcV$sp(StreamingContext.scala:625)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:266)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ShutdownHookManager.scala:236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(ShutdownHookManager.scala:236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(ShutdownHookManager.scala:236)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1697)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(ShutdownHookManager.scala:236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(ShutdownHookManager.scala:236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(ShutdownHookManager.scala:236)
	at scala.util.Try$.apply(Try.scala:161)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:236)
	- locked <0x00000005405b6a00> (a org.apache.spark.util.SparkShutdownHookManager)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:216)
	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)
"main":
	at org.apache.spark.util.SparkShutdownHookManager.remove(ShutdownHookManager.scala:248)
	- waiting to lock <0x00000005405b6a00> (a org.apache.spark.util.SparkShutdownHookManager)
	at org.apache.spark.util.ShutdownHookManager$.removeShutdownHook(ShutdownHookManager.scala:199)
	at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:712)
	- locked <0x00000005405a1680> (a org.apache.spark.streaming.StreamingContext)
	at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:684)
	- locked <0x00000005405a1680> (a org.apache.spark.streaming.StreamingContext)
	at org.apache.spark.streaming.SessionByKeyBenchmark$.main(SessionByKeyBenchmark.scala:108)
	at org.apache.spark.streaming.SessionByKeyBenchmark.main(SessionByKeyBenchmark.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:497)
	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:680)
	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

This PR just moved `ShutdownHookManager.removeShutdownHook` out of `synchronized` to avoid deadlock.

Author: zsxwing <zsxwing@gmail.com>

Closes #9116 from zsxwing/stop-deadlock.
2015-10-16 13:56:51 -07:00
zsxwing 369d786f58 [SPARK-10974] [STREAMING] Add progress bar for output operation column and use red dots for failed batches
Screenshot:
<img width="1363" alt="1" src="https://cloud.githubusercontent.com/assets/1000778/10342571/385d9340-6d4c-11e5-8e79-1fa4c3c98f81.png">

Also fixed the description and duration for output operations that don't have spark jobs.
<img width="1354" alt="2" src="https://cloud.githubusercontent.com/assets/1000778/10342775/4bd52a0e-6d4d-11e5-99bc-26265a9fc792.png">

Author: zsxwing <zsxwing@gmail.com>

Closes #9010 from zsxwing/output-op-progress-bar.
2015-10-16 13:53:06 -07:00
jerryshao 43f5d1f326 [SPARK-11060] [STREAMING] Fix some potential NPE in DStream transformation
This patch fixes:

1. Guard out against NPEs in `TransformedDStream` when parent DStream returns None instead of empty RDD.
2. Verify some input streams which will potentially return None.
3. Add unit test to verify the behavior when input stream returns None.

cc tdas , please help to review, thanks a lot :).

Author: jerryshao <sshao@hortonworks.com>

Closes #9070 from jerryshao/SPARK-11060.
2015-10-16 11:53:47 +01:00
Tom Graves 135a2ce5b0 [SPARK-10619] Can't sort columns on Executor Page
should pick into spark 1.5.2 also.

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

looks like this was broken by commit: fb1d06fc24 (diff-b8adb646ef90f616c34eb5c98d1ebd16)
It looks like somethings were change to use the UIUtils.listingTable but executor page wasn't converted so when it removed sortable from the UIUtils. TABLE_CLASS_NOT_STRIPED it broke this page.

Simply add the sortable tag back in and it fixes both active UI and the history server UI.

Author: Tom Graves <tgraves@yahoo-inc.com>

Closes #9101 from tgravescs/SPARK-10619.
2015-10-14 10:12:25 -07:00
Jacker Hu a16396df76 [SPARK-10772] [STREAMING] [SCALA] NullPointerException when transform function in DStream returns NULL
Currently, the ```TransformedDStream``` will using ```Some(transformFunc(parentRDDs, validTime))``` as compute return value, when the ```transformFunc``` somehow returns null as return value, the followed operator will have NullPointerExeception.

This fix uses the ```Option()``` instead of ```Some()``` to deal with the possible null value. When   ```transformFunc``` returns ```null```, the option will transform null to ```None```, the downstream can handle ```None``` correctly.

NOTE (2015-09-25): The latest fix will check the return value of transform function, if it is ```NULL```, a spark exception will be thrown out

Author: Jacker Hu <gt.hu.chang@gmail.com>
Author: jhu-chang <gt.hu.chang@gmail.com>

Closes #8881 from jhu-chang/Fix_Transform.
2015-10-10 11:36:18 +01:00
Andrew Or 67fbecbf32 [SPARK-10956] Common MemoryManager interface for storage and execution
This patch introduces a `MemoryManager` that is the central arbiter of how much memory to grant to storage and execution. This patch is primarily concerned only with refactoring while preserving the existing behavior as much as possible.

This is the first step away from the existing rigid separation of storage and execution memory, which has several major drawbacks discussed on the [issue](https://issues.apache.org/jira/browse/SPARK-10956). It is the precursor of a series of patches that will attempt to address those drawbacks.

Author: Andrew Or <andrew@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: andrewor14 <andrew@databricks.com>

Closes #9000 from andrewor14/memory-manager.
2015-10-08 21:44:59 -07:00
Hari Shreedharan 0984129005 [SPARK-10955] [STREAMING] Add a warning if dynamic allocation for Streaming applications
Dynamic allocation can be painful for streaming apps and can lose data. Log a warning for streaming applications if dynamic allocation is enabled.

Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #8998 from harishreedharan/ss-log-error and squashes the following commits:

462b264 [Hari Shreedharan] Improve log message.
2733d94 [Hari Shreedharan] Minor change to warning message.
eaa48cc [Hari Shreedharan] Log a warning instead of failing the application if dynamic allocation is enabled.
725f090 [Hari Shreedharan] Add config parameter to allow dynamic allocation if the user explicitly sets it.
b3f9a95 [Hari Shreedharan] Disable dynamic allocation and kill app if it is enabled.
a4a5212 [Hari Shreedharan] [streaming] SPARK-10955. Disable dynamic allocation for Streaming applications.
2015-10-08 18:53:38 -07:00
Marcelo Vanzin 94fc57afdf [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8775 from vanzin/SPARK-10300.
2015-10-07 14:11:21 -07:00
zsxwing ffe6831e49 [SPARK-10885] [STREAMING] Display the failed output op in Streaming UI
This PR implements the following features for both `master` and `branch-1.5`.
1. Display the failed output op count in the batch list
2. Display the failure reason of output op in the batch detail page

Screenshots:
<img width="1356" alt="1" src="https://cloud.githubusercontent.com/assets/1000778/10198387/5b2b97ec-67ce-11e5-81c2-f818b9d2f3ad.png">
<img width="1356" alt="2" src="https://cloud.githubusercontent.com/assets/1000778/10198388/5b76ac14-67ce-11e5-8c8b-de2683c5b485.png">

There are still two remaining problems in the UI.
1. If an output operation doesn't run any spark job, we cannot get the its duration since now it's the sum of all jobs' durations.
2. If an output operation doesn't run any spark job, we cannot get the description since it's the latest job's call site.

We need to add new `StreamingListenerEvent` about output operations to fix them. So I'd like to fix them only for `master` in another PR.

Author: zsxwing <zsxwing@gmail.com>

Closes #8950 from zsxwing/batch-failure.
2015-10-06 16:51:03 -07:00
zsxwing be7c5ff1ad [SPARK-10900] [STREAMING] Add output operation events to StreamingListener
Add output operation events to StreamingListener so as to implement the following UI features:

1. Progress bar of a batch in the batch list.
2. Be able to display output operation `description` and `duration` when there is no spark job in a Streaming job.

Author: zsxwing <zsxwing@gmail.com>

Closes #8958 from zsxwing/output-operation-events.
2015-10-05 19:23:41 -07:00
Guillaume Poulin be0dcd6eb1 FIX: rememberDuration reassignment error message
I was reading throught the scheduler and found this small mistake.

Author: Guillaume Poulin <guillaume@hopper.com>

Closes #8966 from gpoulin/remember_duration_typo.
2015-10-03 12:14:00 +01:00
zsxwing 107320c9bb [SPARK-6028] [CORE] Remerge #6457: new RPC implemetation and also pick #8905
This PR just reverted 02144d6745 to remerge #6457 and also included the commits in #8905.

Author: zsxwing <zsxwing@gmail.com>

Closes #8944 from zsxwing/SPARK-6028.
2015-10-03 01:04:35 -07:00
Xiangrui Meng 02144d6745 Revert "[SPARK-6028][Core]A new RPC implemetation based on the network module"
This reverts commit 084e4e1262.
2015-09-24 08:25:44 -07:00
zsxwing 758c9d25e9 [SPARK-10692] [STREAMING] Expose failureReasons in BatchInfo for streaming UI to clear failed batches
Slightly modified version of #8818, all credit goes to zsxwing

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

Closes #8892 from tdas/SPARK-10692.
2015-09-23 19:52:02 -07:00
zsxwing 084e4e1262 [SPARK-6028][Core]A new RPC implemetation based on the network module
Design doc: https://docs.google.com/document/d/1CF5G6rGVQMKSyV_QKo4D2M-x6rxz5x1Ew7aK3Uq6u8c/edit?usp=sharing

Author: zsxwing <zsxwing@gmail.com>

Closes #6457 from zsxwing/new-rpc.
2015-09-23 18:59:49 -07:00
zsxwing 50e4634236 [SPARK-10769] [STREAMING] [TESTS] Fix o.a.s.streaming.CheckpointSuite.maintains rate controller
Fixed the following failure in https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1787/testReport/junit/org.apache.spark.streaming/CheckpointSuite/recovery_maintains_rate_controller/
```
sbt.ForkMain$ForkError: The code passed to eventually never returned normally. Attempted 660 times over 10.000044392000001 seconds. Last failure message: 9223372036854775807 did not equal 200.
	at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
	at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:336)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	at org.apache.spark.streaming.CheckpointSuite$$anonfun$15.apply$mcV$sp(CheckpointSuite.scala:413)
	at org.apache.spark.streaming.CheckpointSuite$$anonfun$15.apply(CheckpointSuite.scala:396)
	at org.apache.spark.streaming.CheckpointSuite$$anonfun$15.apply(CheckpointSuite.scala:396)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
```

In this test, it calls `advanceTimeWithRealDelay(ssc, 2)` to run two batch jobs. However, one race condition is these two jobs can finish before the receiver is registered. Then `UpdateRateLimit` won't be sent to the receiver and `getDefaultBlockGeneratorRateLimit` cannot be updated.

Here are the logs related to this issue:
```
15/09/22 19:28:26.154 pool-1-thread-1-ScalaTest-running-CheckpointSuite INFO CheckpointSuite: Manual clock before advancing = 2500

15/09/22 19:28:26.869 JobScheduler INFO JobScheduler: Finished job streaming job 3000 ms.0 from job set of time 3000 ms
15/09/22 19:28:26.869 JobScheduler INFO JobScheduler: Total delay: 1442975303.869 s for time 3000 ms (execution: 0.711 s)

15/09/22 19:28:26.873 JobScheduler INFO JobScheduler: Finished job streaming job 3500 ms.0 from job set of time 3500 ms
15/09/22 19:28:26.873 JobScheduler INFO JobScheduler: Total delay: 1442975303.373 s for time 3500 ms (execution: 0.004 s)

15/09/22 19:28:26.879 sparkDriver-akka.actor.default-dispatcher-3 INFO ReceiverTracker: Registered receiver for stream 0 from localhost:57749

15/09/22 19:28:27.154 pool-1-thread-1-ScalaTest-running-CheckpointSuite INFO CheckpointSuite: Manual clock after advancing = 3500
```
`advanceTimeWithRealDelay(ssc, 2)` triggered job 3000ms and 3500ms but the receiver was registered after job 3000ms and 3500ms finished.

So we should make sure the receiver online before running `advanceTimeWithRealDelay(ssc, 2)`.

Author: zsxwing <zsxwing@gmail.com>

Closes #8877 from zsxwing/SPARK-10769.
2015-09-23 01:29:30 -07:00
zsxwing 44c28abf12 [SPARK-10224] [STREAMING] Fix the issue that blockIntervalTimer won't call updateCurrentBuffer when stopping
`blockIntervalTimer.stop(interruptTimer = false)` doesn't guarantee calling `updateCurrentBuffer`. So it's possible that `blockIntervalTimer` will exit when `updateCurrentBuffer` is not empty. Then the data in `currentBuffer` will be lost.

To reproduce it, you can add `Thread.sleep(200)` in this line (69c9c17716/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala (L100)) and run `StreamingContexSuite`.
I cannot write a unit test to reproduce it because I cannot find an approach to force `RecurringTimer` suspend at this line for a few milliseconds.

There was a failure in Jenkins here: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41455/console

This PR updates RecurringTimer to make sure `stop(interruptTimer = false)` will call `callback` at least once after the `stop` method is called.

Author: zsxwing <zsxwing@gmail.com>

Closes #8417 from zsxwing/SPARK-10224.
2015-09-23 01:28:02 -07:00
Tathagata Das 5548a25475 [SPARK-10652] [SPARK-10742] [STREAMING] Set meaningful job descriptions for all streaming jobs
Here is the screenshot after adding the job descriptions to threads that run receivers and the scheduler thread running the batch jobs.

## All jobs page
* Added job descriptions with links to relevant batch details page
![image](https://cloud.githubusercontent.com/assets/663212/9924165/cda4a372-5cb1-11e5-91ca-d43a32c699e9.png)

## All stages page
* Added stage descriptions with links to relevant batch details page
![image](https://cloud.githubusercontent.com/assets/663212/9923814/2cce266a-5cae-11e5-8a3f-dad84d06c50e.png)

## Streaming batch details page
* Added the +details link
![image](https://cloud.githubusercontent.com/assets/663212/9921977/24014a32-5c98-11e5-958e-457b6c38065b.png)

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

Closes #8791 from tdas/SPARK-10652.
2015-09-22 22:44:09 -07:00
Tathagata Das 72869883f1 [SPARK-10649] [STREAMING] Prevent inheriting job group and irrelevant job description in streaming jobs
The job group, and job descriptions information is passed through thread local properties, and get inherited by child threads. In case of spark streaming, the streaming jobs inherit these properties from the thread that called streamingContext.start(). This may not make sense.

1. Job group: This is mainly used for cancelling a group of jobs together. It does not make sense to cancel streaming jobs like this, as the effect will be unpredictable. And its not a valid usecase any way, to cancel a streaming context, call streamingContext.stop()

2. Job description: This is used to pass on nice text descriptions for jobs to show up in the UI. The job description of the thread that calls streamingContext.start() is not useful for all the streaming jobs, as it does not make sense for all of the streaming jobs to have the same description, and the description may or may not be related to streaming.

The solution in this PR is meant for the Spark master branch, where local properties are inherited by cloning the properties. The job group and job description in the thread that starts the streaming scheduler are explicitly removed, so that all the subsequent child threads does not inherit them. Also, the starting is done in a new child thread, so that setting the job group and description for streaming, does not change those properties in the thread that called streamingContext.start().

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

Closes #8781 from tdas/SPARK-10649.
2015-09-21 16:47:52 -07:00
Marcelo Vanzin b42059d2ef Revert "[SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py."
This reverts commit 8abef21dac.
2015-09-15 13:03:38 -07:00
Marcelo Vanzin 8abef21dac [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.
This change does two things:

- tag a few tests and adds the mechanism in the build to be able to disable those tags,
  both in maven and sbt, for both junit and scalatest suites.
- add some logic to run-tests.py to disable some tags depending on what files have
  changed; that's used to disable expensive tests when a module hasn't explicitly
  been changed, to speed up testing for changes that don't directly affect those
  modules.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8437 from vanzin/test-tags.
2015-09-15 10:45:02 -07:00
Reynold Xin 09b7e7c198 Update version to 1.6.0-SNAPSHOT.
Author: Reynold Xin <rxin@databricks.com>

Closes #8350 from rxin/1.6.
2015-09-15 00:54:20 -07:00
Sean Owen 4e2242bb41 [SPARK-10576] [BUILD] Move .java files out of src/main/scala
Move .java files in `src/main/scala` to `src/main/java` root, except for `package-info.java` (to stay next to package.scala)

Author: Sean Owen <sowen@cloudera.com>

Closes #8736 from srowen/SPARK-10576.
2015-09-14 15:03:51 -07:00
Sean Owen 22730ad54d [SPARK-10547] [TEST] Streamline / improve style of Java API tests
Fix a few Java API test style issues: unused generic types, exceptions, wrong assert argument order

Author: Sean Owen <sowen@cloudera.com>

Closes #8706 from srowen/SPARK-10547.
2015-09-12 10:40:10 +01:00
Luc Bourlier c1bc4f439f [SPARK-10227] fatal warnings with sbt on Scala 2.11
The bulk of the changes are on `transient` annotation on class parameter. Often the compiler doesn't generate a field for this parameters, so the the transient annotation would be unnecessary.
But if the class parameter are used in methods, then fields are created. So it is safer to keep the annotations.

The remainder are some potential bugs, and deprecated syntax.

Author: Luc Bourlier <luc.bourlier@typesafe.com>

Closes #8433 from skyluc/issue/sbt-2.11.
2015-09-09 09:57:58 +01:00
zsxwing 820913f554 [SPARK-10071] [STREAMING] Output a warning when writing QueueInputDStream and throw a better exception when reading QueueInputDStream
Output a warning when serializing QueueInputDStream rather than throwing an exception to allow unit tests use it. Moreover, this PR also throws an better exception when deserializing QueueInputDStream to make the user find out the problem easily. The previous exception is hard to understand: https://issues.apache.org/jira/browse/SPARK-8553

Author: zsxwing <zsxwing@gmail.com>

Closes #8624 from zsxwing/SPARK-10071 and squashes the following commits:

847cfa8 [zsxwing] Output a warning when writing QueueInputDStream and throw a better exception when reading QueueInputDStream
2015-09-08 20:39:15 -07:00
Reynold Xin 5ffe752b59 [SPARK-9767] Remove ConnectionManager.
We introduced the Netty network module for shuffle in Spark 1.2, and has turned it on by default for 3 releases. The old ConnectionManager is difficult to maintain. If we merge the patch now, by the time it is released, it would be 1 yr for which ConnectionManager is off by default. It's time to remove it.

Author: Reynold Xin <rxin@databricks.com>

Closes #8161 from rxin/SPARK-9767.
2015-09-07 10:42:30 -10:00
xutingjun eafe37236c [SPARK-10311] [STREAMING] Reload appId and attemptId when app starts with checkpoint file in cluster mode
Author: xutingjun <xutingjun@huawei.com>

Closes #8477 from XuTingjun/streaming-attempt.
2015-09-04 15:40:02 -07:00
robbins 754f853b02 [SPARK-9869] [STREAMING] Wait for all event notifications before asserting results
Author: robbins <robbins@uk.ibm.com>

Closes #8589 from robbinspg/InputStreamSuite-fix.
2015-09-03 13:48:35 -07:00
zsxwing 4a5fe09165 [SPARK-10369] [STREAMING] Don't remove ReceiverTrackingInfo when deregisterReceivering since we may reuse it later
`deregisterReceiver` should not remove `ReceiverTrackingInfo`. Otherwise, it will throw `java.util.NoSuchElementException: key not found` when restarting it.

Author: zsxwing <zsxwing@gmail.com>

Closes #8538 from zsxwing/SPARK-10369.
2015-08-31 12:19:11 -07:00
Sean Owen 69c9c17716 [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses to JavaConverters
Replace `JavaConversions` implicits with `JavaConverters`

Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet.

Author: Sean Owen <sowen@cloudera.com>

Closes #8033 from srowen/SPARK-9613.
2015-08-25 12:33:13 +01:00
Tathagata Das 1fc37581a5 [SPARK-10210] [STREAMING] Filter out non-existent blocks before creating BlockRDD
When write ahead log is not enabled, a recovered streaming driver still tries to run jobs using pre-failure block ids, and fails as the block do not exists in-memory any more (and cannot be recovered as receiver WAL is not enabled).

This occurs because the driver-side WAL of ReceivedBlockTracker is recovers that past block information, and ReceiveInputDStream creates BlockRDDs even if those blocks do not exist.

The solution in this PR is to filter out block ids that do not exist before creating the BlockRDD. In addition, it adds unit tests to verify other logic in ReceiverInputDStream.

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

Closes #8405 from tdas/SPARK-10210.
2015-08-25 00:35:51 -07:00
zsxwing f023aa2fcc [SPARK-10137] [STREAMING] Avoid to restart receivers if scheduleReceivers returns balanced results
This PR fixes the following cases for `ReceiverSchedulingPolicy`.

1) Assume there are 4 executors: host1, host2, host3, host4, and 5 receivers: r1, r2, r3, r4, r5. Then `ReceiverSchedulingPolicy.scheduleReceivers` will return (r1 -> host1, r2 -> host2, r3 -> host3, r4 -> host4, r5 -> host1).
Let's assume r1 starts at first on `host1` as `scheduleReceivers` suggested,  and try to register with ReceiverTracker. But the previous `ReceiverSchedulingPolicy.rescheduleReceiver` will return (host2, host3, host4) according to the current executor weights (host1 -> 1.0, host2 -> 0.5, host3 -> 0.5, host4 -> 0.5), so ReceiverTracker will reject `r1`. This is unexpected since r1 is starting exactly where `scheduleReceivers` suggested.

This case can be fixed by ignoring the information of the receiver that is rescheduling in `receiverTrackingInfoMap`.

2) Assume there are 3 executors (host1, host2, host3) and each executors has 3 cores, and 3 receivers: r1, r2, r3. Assume r1 is running on host1. Now r2 is restarting, the previous `ReceiverSchedulingPolicy.rescheduleReceiver` will always return (host1, host2, host3). So it's possible that r2 will be scheduled to host1 by TaskScheduler. r3 is similar. Then at last, it's possible that there are 3 receivers running on host1, while host2 and host3 are idle.

This issue can be fixed by returning only executors that have the minimum wight rather than returning at least 3 executors.

Author: zsxwing <zsxwing@gmail.com>

Closes #8340 from zsxwing/fix-receiver-scheduling.
2015-08-24 23:34:50 -07:00
Tathagata Das 7478c8b66d [SPARK-9791] [PACKAGE] Change private class to private class to prevent unnecessary classes from showing up in the docs
In addition, some random cleanup of import ordering

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

Closes #8387 from tdas/SPARK-9791 and squashes the following commits:

67f3ee9 [Tathagata Das] Change private class to private[package] class to prevent them from showing up in the docs
2015-08-24 12:40:09 -07:00
Tathagata Das 053d94fcf3 [SPARK-10142] [STREAMING] Made python checkpoint recovery handle non-local checkpoint paths and existing SparkContexts
The current code only checks checkpoint files in local filesystem, and always tries to create a new Python SparkContext (even if one already exists). The solution is to do the following:
1. Use the same code path as Java to check whether a valid checkpoint exists
2. Create a new Python SparkContext only if there no active one.

There is not test for the path as its hard to test with distributed filesystem paths in a local unit test. I am going to test it with a distributed file system manually to verify that this patch works.

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

Closes #8366 from tdas/SPARK-10142 and squashes the following commits:

3afa666 [Tathagata Das] Added tests
2dd4ae5 [Tathagata Das] Added the check to not create a context if one already exists
9bf151b [Tathagata Das] Made python checkpoint recovery use java to find the checkpoint files
2015-08-23 19:24:32 -07:00
zsxwing c6df5f66d9 [SPARK-10148] [STREAMING] Display active and inactive receiver numbers in Streaming page
Added the active and inactive receiver numbers in the summary section of Streaming page.

<img width="1074" alt="screen shot 2015-08-21 at 2 08 54 pm" src="https://cloud.githubusercontent.com/assets/1000778/9402437/ff2806a2-480f-11e5-8f8e-efdf8e5d514d.png">

Author: zsxwing <zsxwing@gmail.com>

Closes #8351 from zsxwing/receiver-number.
2015-08-23 17:41:49 -07:00
Tathagata Das b762f9920f [SPARK-10128] [STREAMING] Used correct classloader to deserialize WAL data
Recovering Kinesis sequence numbers from WAL leads to classnotfoundexception because the ObjectInputStream does not use the correct classloader and the SequenceNumberRanges class (in streaming-kinesis-asl package) cannot be found (added through spark-submit) while deserializing. The solution is to use `Thread.currentThread().getContextClassLoader` while deserializing.

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

Closes #8328 from tdas/SPARK-10128 and squashes the following commits:

f19b1c2 [Tathagata Das] Used correct classloader to deserialize WAL data
2015-08-19 21:15:58 -07:00
zsxwing affc8a887e [SPARK-10125] [STREAMING] Fix a potential deadlock in JobGenerator.stop
Because `lazy val` uses `this` lock, if JobGenerator.stop and JobGenerator.doCheckpoint (JobGenerator.shouldCheckpoint has not yet been initialized) run at the same time, it may hang.

Here are the stack traces for the deadlock:

```Java
"pool-1-thread-1-ScalaTest-running-StreamingListenerSuite" #11 prio=5 os_prio=31 tid=0x00007fd35d094800 nid=0x5703 in Object.wait() [0x000000012ecaf000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Thread.join(Thread.java:1245)
        - locked <0x00000007b5d8d7f8> (a org.apache.spark.util.EventLoop$$anon$1)
        at java.lang.Thread.join(Thread.java:1319)
        at org.apache.spark.util.EventLoop.stop(EventLoop.scala:81)
        at org.apache.spark.streaming.scheduler.JobGenerator.stop(JobGenerator.scala:155)
        - locked <0x00000007b5d8cea0> (a org.apache.spark.streaming.scheduler.JobGenerator)
        at org.apache.spark.streaming.scheduler.JobScheduler.stop(JobScheduler.scala:95)
        - locked <0x00000007b5d8ced8> (a org.apache.spark.streaming.scheduler.JobScheduler)
        at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:687)

"JobGenerator" #67 daemon prio=5 os_prio=31 tid=0x00007fd35c3b9800 nid=0x9f03 waiting for monitor entry [0x0000000139e4a000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.spark.streaming.scheduler.JobGenerator.shouldCheckpoint$lzycompute(JobGenerator.scala:63)
        - waiting to lock <0x00000007b5d8cea0> (a org.apache.spark.streaming.scheduler.JobGenerator)
        at org.apache.spark.streaming.scheduler.JobGenerator.shouldCheckpoint(JobGenerator.scala:63)
        at org.apache.spark.streaming.scheduler.JobGenerator.doCheckpoint(JobGenerator.scala:290)
        at org.apache.spark.streaming.scheduler.JobGenerator.org$apache$spark$streaming$scheduler$JobGenerator$$processEvent(JobGenerator.scala:182)
        at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:83)
        at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:82)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
```

I can use this patch to produce this deadlock: 8a88f28d13

And a timeout build in Jenkins due to this deadlock: https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1654/

This PR initializes `checkpointWriter` before `eventLoop` uses it to avoid this deadlock.

Author: zsxwing <zsxwing@gmail.com>

Closes #8326 from zsxwing/SPARK-10125.
2015-08-19 19:43:09 -07:00
Tathagata Das bc9a0e0323 [SPARK-9967] [SPARK-10099] [STREAMING] Renamed conf spark.streaming.backpressure.{enable-->enabled} and fixed deprecated annotations
Small changes
- Renamed conf spark.streaming.backpressure.{enable --> enabled}
- Change Java Deprecated annotations to Scala deprecated annotation with more information.

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

Closes #8299 from tdas/SPARK-9967.
2015-08-18 23:37:57 -07:00
zsxwing 90273eff96 [SPARK-10102] [STREAMING] Fix a race condition that startReceiver may happen before setting trackerState to Started
Test failure: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/3305/testReport/junit/org.apache.spark.streaming/StreamingContextSuite/stop_gracefully/

There is a race condition that setting `trackerState` to `Started` could happen after calling `startReceiver`. Then `startReceiver` won't start the receivers because it uses `! isTrackerStarted` to check if ReceiverTracker is stopping or stopped. But actually, `trackerState` is `Initialized` and will be changed to `Started` soon.

Therefore, we should use `isTrackerStopping || isTrackerStopped`.

Author: zsxwing <zsxwing@gmail.com>

Closes #8294 from zsxwing/SPARK-9504.
2015-08-18 20:15:54 -07:00
Tathagata Das 1aeae05bb2 [SPARK-10072] [STREAMING] BlockGenerator can deadlock when the queue of generate blocks fills up to capacity
Generated blocks are inserted into an ArrayBlockingQueue, and another thread pulls stuff from the ArrayBlockingQueue and pushes it into BlockManager. Now if that queue fills up to capacity (default is 10 blocks), then the inserting into queue (done in the function updateCurrentBuffer) get blocked inside a synchronized block. However, the thread that is pulling blocks from the queue uses the same lock to check the current (active or stopped) while pulling from the queue. Since the block generating threads is blocked (as the queue is full) on the lock, this thread that is supposed to drain the queue gets blocked. Ergo, deadlock.

Solution: Moved blocking call to ArrayBlockingQueue outside the synchronized to prevent deadlock.

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

Closes #8257 from tdas/SPARK-10072.
2015-08-18 19:26:38 -07:00
Tathagata Das 9108eff74a [SPARK-10098] [STREAMING] [TEST] Cleanup active context after test in FailureSuite
Failures in streaming.FailureSuite can leak StreamingContext and SparkContext which fails all subsequent tests

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

Closes #8289 from tdas/SPARK-10098.
2015-08-18 17:00:13 -07:00
Davies Liu 37586e5449 [HOTFIX] fix duplicated braces
Author: Davies Liu <davies@databricks.com>

Closes #8219 from davies/fix_typo.
2015-08-14 20:56:55 -07:00