aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/main
Commit message (Collapse)AuthorAgeFilesLines
* FIX: rememberDuration reassignment error messageGuillaume Poulin2015-10-031-11/+5
| | | | | | | | I was reading throught the scheduler and found this small mistake. Author: Guillaume Poulin <guillaume@hopper.com> Closes #8966 from gpoulin/remember_duration_typo.
* [SPARK-6028] [CORE] Remerge #6457: new RPC implemetation and also pick #8905zsxwing2015-10-031-1/+1
| | | | | | | | This PR just reverted https://github.com/apache/spark/commit/02144d6745ec0a6d8877d969feb82139bd22437f to remerge #6457 and also included the commits in #8905. Author: zsxwing <zsxwing@gmail.com> Closes #8944 from zsxwing/SPARK-6028.
* Revert "[SPARK-6028][Core]A new RPC implemetation based on the network module"Xiangrui Meng2015-09-241-1/+1
| | | | This reverts commit 084e4e126211d74a79e8dbd2d0e604dd3c650822.
* [SPARK-10692] [STREAMING] Expose failureReasons in BatchInfo for streaming ↵zsxwing2015-09-233-16/+39
| | | | | | | | | | | 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.
* [SPARK-6028][Core]A new RPC implemetation based on the network modulezsxwing2015-09-231-1/+1
| | | | | | | | 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.
* [SPARK-10224] [STREAMING] Fix the issue that blockIntervalTimer won't call ↵zsxwing2015-09-231-6/+13
| | | | | | | | | | | | | | | | | 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 (https://github.com/apache/spark/blob/69c9c177160e32a2fbc9b36ecc52156077fca6fc/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.
* [SPARK-10652] [SPARK-10742] [STREAMING] Set meaningful job descriptions for ↵Tathagata Das2015-09-224-16/+41
| | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-10649] [STREAMING] Prevent inheriting job group and irrelevant job ↵Tathagata Das2015-09-211-3/+12
| | | | | | | | | | | | | | | | 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.
* [SPARK-10576] [BUILD] Move .java files out of src/main/scalaSean Owen2015-09-141-0/+0
| | | | | | | | 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.
* [SPARK-10227] fatal warnings with sbt on Scala 2.11Luc Bourlier2015-09-0910-26/+26
| | | | | | | | | | | 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.
* [SPARK-10071] [STREAMING] Output a warning when writing QueueInputDStream ↵zsxwing2015-09-082-4/+11
| | | | | | | | | | | | 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
* [SPARK-10311] [STREAMING] Reload appId and attemptId when app starts with ↵xutingjun2015-09-041-0/+2
| | | | | | | | checkpoint file in cluster mode Author: xutingjun <xutingjun@huawei.com> Closes #8477 from XuTingjun/streaming-attempt.
* [SPARK-10369] [STREAMING] Don't remove ReceiverTrackingInfo when ↵zsxwing2015-08-311-2/+2
| | | | | | | | | | 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.
* [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses ↵Sean Owen2015-08-258-49/+46
| | | | | | | | | | | | 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.
* [SPARK-10210] [STREAMING] Filter out non-existent blocks before creating ↵Tathagata Das2015-08-252-2/+10
| | | | | | | | | | | | | | 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.
* [SPARK-10137] [STREAMING] Avoid to restart receivers if scheduleReceivers ↵zsxwing2015-08-242-51/+113
| | | | | | | | | | | | | | | | | | | 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.
* [SPARK-9791] [PACKAGE] Change private class to private class to prevent ↵Tathagata Das2015-08-245-3/+7
| | | | | | | | | | | | 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
* [SPARK-10142] [STREAMING] Made python checkpoint recovery handle non-local ↵Tathagata Das2015-08-231-0/+9
| | | | | | | | | | | | | | | | | | 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
* [SPARK-10148] [STREAMING] Display active and inactive receiver numbers in ↵zsxwing2015-08-232-0/+14
| | | | | | | | | | | | 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.
* [SPARK-10128] [STREAMING] Used correct classloader to deserialize WAL dataTathagata Das2015-08-191-2/+3
| | | | | | | | | | 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
* [SPARK-10125] [STREAMING] Fix a potential deadlock in JobGenerator.stopzsxwing2015-08-191-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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: https://github.com/zsxwing/spark/commit/8a88f28d1331003a65fabef48ae3d22a7c21f05f 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.
* [SPARK-9967] [SPARK-10099] [STREAMING] Renamed conf ↵Tathagata Das2015-08-182-6/+6
| | | | | | | | | | | | 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.
* [SPARK-10102] [STREAMING] Fix a race condition that startReceiver may happen ↵zsxwing2015-08-181-3/+8
| | | | | | | | | | | | | | 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.
* [SPARK-10072] [STREAMING] BlockGenerator can deadlock when the queue of ↵Tathagata Das2015-08-181-10/+19
| | | | | | | | | | | | 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.
* [HOTFIX] fix duplicated bracesDavies Liu2015-08-142-2/+2
| | | | | | Author: Davies Liu <davies@databricks.com> Closes #8219 from davies/fix_typo.
* [SPARK-9968] [STREAMING] Reduced time spent within synchronized block to ↵Tathagata Das2015-08-141-8/+32
| | | | | | | | | | | | | prevent lock starvation When the rate limiter is actually limiting the rate at which data is inserted into the buffer, the synchronized block of BlockGenerator.addData stays blocked for long time. This causes the thread switching the buffer and generating blocks (synchronized with addData) to starve and not generate blocks for seconds. The correct solution is to not block on the rate limiter within the synchronized block for adding data to the buffer. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8204 from tdas/SPARK-9968 and squashes the following commits: 8cbcc1b [Tathagata Das] Removed unused val a73b645 [Tathagata Das] Reduced time spent within synchronized block
* [SPARK-9966] [STREAMING] Handle couple of corner cases in PIDRateEstimatorTathagata Das2015-08-142-14/+36
| | | | | | | | | | | | | 1. The rate estimator should not estimate any rate when there are no records in the batch, as there is no data to estimate the rate. In the current state, it estimates and set the rate to zero. That is incorrect. 2. The rate estimator should not never set the rate to zero under any circumstances. Otherwise the system will stop receiving data, and stop generating useful estimates (see reason 1). So the fix is to define a parameters that sets a lower bound on the estimated rate, so that the system always receives some data. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8199 from tdas/SPARK-9966 and squashes the following commits: 829f793 [Tathagata Das] Fixed unit test and added comments 3a994db [Tathagata Das] Added min rate and updated tests in PIDRateEstimator
* [SPARK-9826] [CORE] Fix cannot use custom classes in log4j.propertiesMichel Lemay2015-08-121-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Refactor Utils class and create ShutdownHookManager. NOTE: Wasn't able to run /dev/run-tests on windows machine. Manual tests were conducted locally using custom log4j.properties file with Redis appender and logstash formatter (bundled in the fat-jar submitted to spark) ex: log4j.rootCategory=WARN,console,redis log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.apache.spark.graphx.Pregel=INFO log4j.appender.redis=com.ryantenney.log4j.FailoverRedisAppender log4j.appender.redis.endpoints=hostname:port log4j.appender.redis.key=mykey log4j.appender.redis.alwaysBatch=false log4j.appender.redis.layout=net.logstash.log4j.JSONEventLayoutV1 Author: michellemay <mlemay@gmail.com> Closes #8109 from michellemay/SPARK-9826.
* [SPARK-9801] [STREAMING] Check if file exists before deleting temporary files.Hao Zhu2015-08-101-2/+6
| | | | | | | | | | | | Spark streaming deletes the temp file and backup files without checking if they exist or not Author: Hao Zhu <viadeazhu@gmail.com> Closes #8082 from viadea/master and squashes the following commits: 242d05f [Hao Zhu] [SPARK-9801][Streaming]No need to check the existence of those files fd143f2 [Hao Zhu] [SPARK-9801][Streaming]Check if backupFile exists before deleting backupFile files. 087daf0 [Hao Zhu] SPARK-9801
* [SPARK-9639] [STREAMING] Fix a potential NPE in Streaming JobSchedulerzsxwing2015-08-061-10/+22
| | | | | | | | | | | | Because `JobScheduler.stop(false)` may set `eventLoop` to null when `JobHandler` is running, then it's possible that when `post` is called, `eventLoop` happens to null. This PR fixed this bug and also set threads in `jobExecutor` to `daemon`. Author: zsxwing <zsxwing@gmail.com> Closes #7960 from zsxwing/fix-npe and squashes the following commits: b0864c4 [zsxwing] Fix a potential NPE in Streaming JobScheduler
* [SPARK-9556] [SPARK-9619] [SPARK-9624] [STREAMING] Make BlockGenerator more ↵Tathagata Das2015-08-066-78/+176
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | robust and make all BlockGenerators subscribe to rate limit updates In some receivers, instead of using the default `BlockGenerator` in `ReceiverSupervisorImpl`, custom generator with their custom listeners are used for reliability (see [`ReliableKafkaReceiver`](https://github.com/apache/spark/blob/master/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala#L99) and [updated `KinesisReceiver`](https://github.com/apache/spark/pull/7825/files)). These custom generators do not receive rate updates. This PR modifies the code to allow custom `BlockGenerator`s to be created through the `ReceiverSupervisorImpl` so that they can be kept track and rate updates can be applied. In the process, I did some simplification, and de-flaki-fication of some rate controller related tests. In particular. - Renamed `Receiver.executor` to `Receiver.supervisor` (to match `ReceiverSupervisor`) - Made `RateControllerSuite` faster (by increasing batch interval) and less flaky - Changed a few internal API to return the current rate of block generators as Long instead of Option\[Long\] (was inconsistent at places). - Updated existing `ReceiverTrackerSuite` to test that custom block generators get rate updates as well. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #7913 from tdas/SPARK-9556 and squashes the following commits: 41d4461 [Tathagata Das] fix scala style eb9fd59 [Tathagata Das] Updated kinesis receiver d24994d [Tathagata Das] Updated BlockGeneratorSuite to use manual clock in BlockGenerator d70608b [Tathagata Das] Updated BlockGenerator with states and proper synchronization f6bd47e [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9556 31da173 [Tathagata Das] Fix bug 12116df [Tathagata Das] Add BlockGeneratorSuite 74bd069 [Tathagata Das] Fix style 989bb5c [Tathagata Das] Made BlockGenerator fail is used after stop, and added better unit tests for it 3ff618c [Tathagata Das] Fix test b40eff8 [Tathagata Das] slight refactoring f0df0f1 [Tathagata Das] Scala style fixes 51759cb [Tathagata Das] Refactored rate controller tests and added the ability to update rate of any custom block generator
* [SPARK-9217] [STREAMING] Make the kinesis receiver reliable by recording ↵Tathagata Das2015-08-051-32/+39
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | sequence numbers This PR is the second one in the larger issue of making the Kinesis integration reliable and provide WAL-free at-least once guarantee. It is based on the design doc - https://docs.google.com/document/d/1k0dl270EnK7uExrsCE7jYw7PYx0YC935uBcxn3p0f58/edit In this PR, I have updated the Kinesis Receiver to do the following. - Control the block generation, by creating its own BlockGenerator with own callback methods and using it to keep track of the ranges of sequence numbers that go into each block. - More specifically, as the KinesisRecordProcessor provides small batches of records, the records are atomically inserted into the block (that is, either the whole batch is in the block, or not). Accordingly the sequence number range of the batch is recorded. Since there may be many batches added to a block, the receiver tracks all the range of sequence numbers that is added to a block. - When the block is ready to be pushed, the block is pushed and the ranges are reported as metadata of the block. In addition, the ranges are used to find out the latest sequence number for each shard that can be checkpointed through the DynamoDB. - Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence number for it own shard. - The array of ranges in the block metadata is used to create KinesisBackedBlockRDDs. The ReceiverInputDStream has been slightly refactored to allow the creation of KinesisBackedBlockRDDs instead of the WALBackedBlockRDDs. Things to be done - [x] Add new test to verify that the sequence numbers are recovered. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #7825 from tdas/kinesis-receiver and squashes the following commits: 2159be9 [Tathagata Das] Fixed bug 569be83 [Tathagata Das] Fix scala style issue bf31e22 [Tathagata Das] Added more documentation to make the kinesis test endpoint more configurable 3ad8361 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into kinesis-receiver c693a63 [Tathagata Das] Removed unnecessary constructor params from KinesisTestUtils e1f1d0a [Tathagata Das] Addressed PR comments b9fa6bf [Tathagata Das] Fix serialization issues f8b7680 [Tathagata Das] Updated doc 33fe43a [Tathagata Das] Added more tests 7997138 [Tathagata Das] Fix style errors a806710 [Tathagata Das] Fixed unit test and use KinesisInputDStream 40a1709 [Tathagata Das] Fixed KinesisReceiverSuite tests 7e44df6 [Tathagata Das] Added documentation and fixed checkpointing 096383f [Tathagata Das] Added test, and addressed some of the comments. 84a7892 [Tathagata Das] fixed scala style issue e19e37d [Tathagata Das] Added license 1cd7b66 [Tathagata Das] Updated kinesis receiver
* [SPARK-9534] [BUILD] Enable javac lint for scalac parity; fix a lot of build ↵Sean Owen2015-08-041-1/+3
| | | | | | | | | | | | | | warnings, 1.5.0 edition Enable most javac lint warnings; fix a lot of build warnings. In a few cases, touch up surrounding code in the process. I'll explain several of the changes inline in comments. Author: Sean Owen <sowen@cloudera.com> Closes #7862 from srowen/SPARK-9534 and squashes the following commits: ea51618 [Sean Owen] Enable most javac lint warnings; fix a lot of build warnings. In a few cases, touch up surrounding code in the process.
* [SPARK-9056] [STREAMING] Rename configuration ↵Sameer Abhyankar2015-07-311-2/+4
| | | | | | | | | | | | | | | | `spark.streaming.minRememberDuration` to `spark.streaming.fileStream.minRememberDuration` Rename configuration `spark.streaming.minRememberDuration` to `spark.streaming.fileStream.minRememberDuration` Author: Sameer Abhyankar <sabhyankar@sabhyankar-MBP.local> Author: Sameer Abhyankar <sabhyankar@sabhyankar-MBP.Samavihome> Closes #7740 from sabhyankar/spark_branch_9056 and squashes the following commits: d5b2f1f [Sameer Abhyankar] Correct deprecated version to 1.5 1268133 [Sameer Abhyankar] Add {} and indentation ddf9844 [Sameer Abhyankar] Change 4 space indentation to 2 space indentation 1819b5f [Sameer Abhyankar] Use spark.streaming.fileStream.minRememberDuration property in lieu of spark.streaming.minRememberDuration
* [SPARK-8979] Add a PID based rate estimatorIulian Dragos2015-07-313-5/+139
| | | | | | | | | | | | | | | | | | Based on #7600 /cc tdas Author: Iulian Dragos <jaguarul@gmail.com> Author: François Garillot <francois@garillot.net> Closes #7648 from dragos/topic/streaming-bp/pid and squashes the following commits: aa5b097 [Iulian Dragos] Add more comments, made all PID constant parameters positive, a couple more tests. 93b74f8 [Iulian Dragos] Better explanation of historicalError. 7975b0c [Iulian Dragos] Add configuration for PID. 26cfd78 [Iulian Dragos] A couple of variable renames. d0bdf7c [Iulian Dragos] Update to latest version of the code, various style and name improvements. d58b845 [François Garillot] [SPARK-8979][Streaming] Implements a PIDRateEstimator
* [SPARK-9472] [STREAMING] consistent hadoop configuration, streaming onlycody koeninger2015-07-304-6/+9
| | | | | | | | Author: cody koeninger <cody@koeninger.org> Closes #7772 from koeninger/streaming-hadoop-config and squashes the following commits: 5267284 [cody koeninger] [SPARK-4229][Streaming] consistent hadoop configuration, streaming only
* [SPARK-9479] [STREAMING] [TESTS] Fix ReceiverTrackerSuite failure for maven ↵zsxwing2015-07-301-3/+9
| | | | | | | | | | | | | | | | | | | build and other potential test failures in Streaming See https://issues.apache.org/jira/browse/SPARK-9479 for the failure cause. The PR includes the following changes: 1. Make ReceiverTrackerSuite create StreamingContext in the test body. 2. Fix places that don't stop StreamingContext. I verified no SparkContext was stopped in the shutdown hook locally after this fix. 3. Fix an issue that `ReceiverTracker.endpoint` may be null. 4. Make sure stopping SparkContext in non-main thread won't fail other tests. Author: zsxwing <zsxwing@gmail.com> Closes #7797 from zsxwing/fix-ReceiverTrackerSuite and squashes the following commits: 3a4bb98 [zsxwing] Fix another potential NPE d7497df [zsxwing] Fix ReceiverTrackerSuite; make sure StreamingContext in tests is closed
* [SPARK-8977] [STREAMING] Defines the RateEstimator interface, and impements ↵Iulian Dragos2015-07-295-3/+185
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | the RateController Based on #7471. - [x] add a test that exercises the publish path from driver to receiver - [ ] remove Serializable from `RateController` and `RateEstimator` Author: Iulian Dragos <jaguarul@gmail.com> Author: François Garillot <francois@garillot.net> Closes #7600 from dragos/topic/streaming-bp/rate-controller and squashes the following commits: f168c94 [Iulian Dragos] Latest review round. 5125e60 [Iulian Dragos] Fix style. a2eb3b9 [Iulian Dragos] Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller 475e346 [Iulian Dragos] Latest round of reviews. e9fb45e [Iulian Dragos] - Add a test for checkpointing - fixed serialization for RateController.executionContext 715437a [Iulian Dragos] Review comments and added a `reset` call in ReceiverTrackerTest. e57c66b [Iulian Dragos] Added a couple of tests for the full scenario from driver to receivers, with several rate updates. b425d32 [Iulian Dragos] Removed DeveloperAPI, removed rateEstimator field, removed Noop rate estimator, changed logic for initialising rate estimator. 238cfc6 [Iulian Dragos] Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller 34a389d [Iulian Dragos] Various style changes and a first test for the rate controller. d32ca36 [François Garillot] [SPARK-8977][Streaming] Defines the RateEstimator interface, and implements the ReceiverRateController 8941cf9 [Iulian Dragos] Renames and other nitpicks. 162d9e5 [Iulian Dragos] Use Reflection for accessing truly private `executor` method and use the listener bus to know when receivers have registered (`onStart` is called before receivers have registered, leading to flaky behavior). 210f495 [Iulian Dragos] Revert "Added a few tests that measure the receiver’s rate." 0c51959 [Iulian Dragos] Added a few tests that measure the receiver’s rate. 261a051 [Iulian Dragos] - removed field to hold the current rate limit in rate limiter - made rate limit a Long and default to Long.MaxValue (consequence of the above) - removed custom `waitUntil` and replaced it by `eventually` cd1397d [Iulian Dragos] Add a test for the propagation of a new rate limit from driver to receivers. 6369b30 [Iulian Dragos] Merge pull request #15 from huitseeker/SPARK-8975 d15de42 [François Garillot] [SPARK-8975][Streaming] Adds Ratelimiter unit tests w.r.t. spark.streaming.receiver.maxRate 4721c7d [François Garillot] [SPARK-8975][Streaming] Add a mechanism to send a new rate from the driver to the block generator
* [SPARK-8882] [STREAMING] Add a new Receiver scheduling mechanismzsxwing2015-07-276-173/+532
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | The design doc: https://docs.google.com/document/d/1ZsoRvHjpISPrDmSjsGzuSu8UjwgbtmoCTzmhgTurHJw/edit?usp=sharing Author: zsxwing <zsxwing@gmail.com> Closes #7276 from zsxwing/receiver-scheduling and squashes the following commits: 137b257 [zsxwing] Add preferredNumExecutors to rescheduleReceiver 61a6c3f [zsxwing] Set state to ReceiverState.INACTIVE in deregisterReceiver 5e1fa48 [zsxwing] Fix the code style 7451498 [zsxwing] Move DummyReceiver back to ReceiverTrackerSuite 715ef9c [zsxwing] Rename: scheduledLocations -> scheduledExecutors; locations -> executors 05daf9c [zsxwing] Use receiverTrackingInfo.toReceiverInfo 1d6d7c8 [zsxwing] Merge branch 'master' into receiver-scheduling 8f93c8d [zsxwing] Use hostPort as the receiver location rather than host; fix comments and unit tests 59f8887 [zsxwing] Schedule all receivers at the same time when launching them 075e0a3 [zsxwing] Add receiver RDD name; use '!isTrackerStarted' instead 276a4ac [zsxwing] Remove "ReceiverLauncher" and move codes to "launchReceivers" fab9a01 [zsxwing] Move methods back to the outer class 4e639c4 [zsxwing] Fix unintentional changes f60d021 [zsxwing] Reorganize ReceiverTracker to use an event loop for lock free 105037e [zsxwing] Merge branch 'master' into receiver-scheduling 5fee132 [zsxwing] Update tha scheduling algorithm to avoid to keep restarting Receiver 9e242c8 [zsxwing] Remove the ScheduleReceiver message because we can refuse it when receiving RegisterReceiver a9acfbf [zsxwing] Merge branch 'squash-pr-6294' into receiver-scheduling 881edb9 [zsxwing] ReceiverScheduler -> ReceiverSchedulingPolicy e530bcc [zsxwing] [SPARK-5681][Streaming] Use a lock to eliminate the race condition when stopping receivers and registering receivers happen at the same time #6294 3b87e4a [zsxwing] Revert SparkContext.scala a86850c [zsxwing] Remove submitAsyncJob and revert JobWaiter f549595 [zsxwing] Add comments for the scheduling approach 9ecc08e [zsxwing] Fix comments and code style 28d1bee [zsxwing] Make 'host' protected; rescheduleReceiver -> getAllowedLocations 2c86a9e [zsxwing] Use tryFailure to support calling jobFailed multiple times ca6fe35 [zsxwing] Add a test for Receiver.restart 27acd45 [zsxwing] Add unit tests for LoadBalanceReceiverSchedulerImplSuite cc76142 [zsxwing] Add JobWaiter.toFuture to avoid blocking threads d9a3e72 [zsxwing] Add a new Receiver scheduling mechanism
* [SPARK-9261] [STREAMING] Avoid calling APIs that expose shaded classes.Marcelo Vanzin2015-07-241-9/+3
| | | | | | | | | | | | Doing this may cause weird errors when tests are run on maven, depending on the flags used. Instead, expose the needed functionality through methods that do not expose shaded classes. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #7601 from vanzin/SPARK-9261 and squashes the following commits: 4f64a16 [Marcelo Vanzin] [SPARK-9261] [streaming] Avoid calling APIs that expose shaded classes.
* [SPARK-8975] [STREAMING] Adds a mechanism to send a new rate from the driver ↵Iulian Dragos2015-07-226-8/+45
| | | | | | | | | | | | | | | | | | | | | | | to the block generator First step for [SPARK-7398](https://issues.apache.org/jira/browse/SPARK-7398). tdas huitseeker Author: Iulian Dragos <jaguarul@gmail.com> Author: François Garillot <francois@garillot.net> Closes #7471 from dragos/topic/streaming-bp/dynamic-rate and squashes the following commits: 8941cf9 [Iulian Dragos] Renames and other nitpicks. 162d9e5 [Iulian Dragos] Use Reflection for accessing truly private `executor` method and use the listener bus to know when receivers have registered (`onStart` is called before receivers have registered, leading to flaky behavior). 210f495 [Iulian Dragos] Revert "Added a few tests that measure the receiver’s rate." 0c51959 [Iulian Dragos] Added a few tests that measure the receiver’s rate. 261a051 [Iulian Dragos] - removed field to hold the current rate limit in rate limiter - made rate limit a Long and default to Long.MaxValue (consequence of the above) - removed custom `waitUntil` and replaced it by `eventually` cd1397d [Iulian Dragos] Add a test for the propagation of a new rate limit from driver to receivers. 6369b30 [Iulian Dragos] Merge pull request #15 from huitseeker/SPARK-8975 d15de42 [François Garillot] [SPARK-8975][Streaming] Adds Ratelimiter unit tests w.r.t. spark.streaming.receiver.maxRate 4721c7d [François Garillot] [SPARK-8975][Streaming] Add a mechanism to send a new rate from the driver to the block generator
* [SPARK-5681] [STREAMING] Move 'stopReceivers' to the event loop to resolve ↵zsxwing2015-07-173-62/+121
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | the race condition This is an alternative way to fix `SPARK-5681`. It minimizes the changes. Closes #4467 Author: zsxwing <zsxwing@gmail.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6294 from zsxwing/pr4467 and squashes the following commits: 709ac1f [zsxwing] Fix the comment e103e8a [zsxwing] Move ReceiverTracker.stop into ReceiverTracker.stop f637142 [zsxwing] Address minor code style comments a178d37 [zsxwing] Move 'stopReceivers' to the event looop to resolve the race condition 51fb07e [zsxwing] Fix the code style 3cb19a3 [zsxwing] Merge branch 'master' into pr4467 b4c29e7 [zsxwing] Stop receiver only if we start it c41ee94 [zsxwing] Make stopReceivers private 7c73c1f [zsxwing] Use trackerStateLock to protect trackerState a8120c0 [zsxwing] Merge branch 'master' into pr4467 7b1d9af [zsxwing] "case Throwable" => "case NonFatal" 15ed4a1 [zsxwing] Register before starting the receiver fff63f9 [zsxwing] Use a lock to eliminate the race condition when stopping receivers and registering receivers happen at the same time. e0ef72a [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into tracker_status_timeout 19b76d9 [Liang-Chi Hsieh] Remove timeout. 34c18dc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into tracker_status_timeout c419677 [Liang-Chi Hsieh] Fix style. 9e1a760 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into tracker_status_timeout 355f9ce [Liang-Chi Hsieh] Separate register and start events for receivers. 3d568e8 [Liang-Chi Hsieh] Let receivers get registered first before going started. ae0d9fd [Liang-Chi Hsieh] Merge branch 'master' into tracker_status_timeout 77983f3 [Liang-Chi Hsieh] Add tracker status and stop to receive messages when stopping tracker.
* [SPARK-6304] [STREAMING] Fix checkpointing doesn't retain driver port issue.jerryshao2015-07-161-0/+2
| | | | | | | | | | | | | | Author: jerryshao <saisai.shao@intel.com> Author: Saisai Shao <saisai.shao@intel.com> Closes #5060 from jerryshao/SPARK-6304 and squashes the following commits: 89b01f5 [jerryshao] Update the unit test to add more cases 275d252 [jerryshao] Address the comments 7cc146d [jerryshao] Address the comments 2624723 [jerryshao] Fix rebase conflict 45befaa [Saisai Shao] Update the unit test bbc1c9c [Saisai Shao] Fix checkpointing doesn't retain driver port issue
* [SPARK-8820] [STREAMING] Add a configuration to set checkpoint dir.huangzhaowei2015-07-141-0/+2
| | | | | | | | | | | | | Add a configuration to set checkpoint directory for convenience to user. [Jira Address](https://issues.apache.org/jira/browse/SPARK-8820) Author: huangzhaowei <carlmartinmax@gmail.com> Closes #7218 from SaintBacchus/SPARK-8820 and squashes the following commits: d49fe4b [huangzhaowei] Rename the configuration name 66ea47c [huangzhaowei] Add the unit test. dd0acc1 [huangzhaowei] [SPARK-8820][Streaming] Add a configuration to set checkpoint dir.
* [SPARK-8962] Add Scalastyle rule to ban direct use of Class.forName; fix ↵Josh Rosen2015-07-141-3/+3
| | | | | | | | | | | | | | | | | | | | | existing uses This pull request adds a Scalastyle regex rule which fails the style check if `Class.forName` is used directly. `Class.forName` always loads classes from the default / system classloader, but in a majority of cases, we should be using Spark's own `Utils.classForName` instead, which tries to load classes from the current thread's context classloader and falls back to the classloader which loaded Spark when the context classloader is not defined. <!-- Reviewable:start --> [<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7350) <!-- Reviewable:end --> Author: Josh Rosen <joshrosen@databricks.com> Closes #7350 from JoshRosen/ban-Class.forName and squashes the following commits: e3e96f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName c0b7885 [Josh Rosen] Hopefully fix the last two cases d707ba7 [Josh Rosen] Fix uses of Class.forName that I missed in my first cleanup pass 046470d [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName 62882ee [Josh Rosen] Fix uses of Class.forName or add exclusion. d9abade [Josh Rosen] Add stylechecker rule to ban uses of Class.forName
* [SPARK-8743] [STREAMING] Deregister Codahale metrics for streaming when ↵Neelesh Srinivas Salian2015-07-131-4/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | StreamingContext is closed The issue link: https://issues.apache.org/jira/browse/SPARK-8743 Deregister Codahale metrics for streaming when StreamingContext is closed Design: Adding the method calls in the appropriate start() and stop () methods for the StreamingContext Actions in the PullRequest: 1) Added the registerSource method call to the start method for the Streaming Context. 2) Added the removeSource method to the stop method. 3) Added comments for both 1 and 2 and comment to show initialization of the StreamingSource 4) Added a test case to check for both registration and de-registration of metrics Previous closed PR for reference: https://github.com/apache/spark/pull/7250 Author: Neelesh Srinivas Salian <nsalian@cloudera.com> Closes #7362 from nssalian/branch-SPARK-8743 and squashes the following commits: 7d998a3 [Neelesh Srinivas Salian] Removed the Thread.sleep() call 8b26397 [Neelesh Srinivas Salian] Moved the scalatest.{} import 0e8007a [Neelesh Srinivas Salian] moved import org.apache.spark{} to correct place daedaa5 [Neelesh Srinivas Salian] Corrected Ordering of imports 8873180 [Neelesh Srinivas Salian] Removed redundancy in imports 59227a4 [Neelesh Srinivas Salian] Changed the ordering of the imports to classify scala and spark imports d8cb577 [Neelesh Srinivas Salian] Added registerSource to start() and removeSource to stop(). Wrote a test to check the registration and de-registration
* [SPARK-7977] [BUILD] Disallowing printlnJonathan Alter2015-07-103-2/+6
| | | | | | | | | | | | | | | | | | | | | | | Author: Jonathan Alter <jonalter@users.noreply.github.com> Closes #7093 from jonalter/SPARK-7977 and squashes the following commits: ccd44cc [Jonathan Alter] Changed println to log in ThreadingSuite 7fcac3e [Jonathan Alter] Reverting to println in ThreadingSuite 10724b6 [Jonathan Alter] Changing some printlns to logs in tests eeec1e7 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977 0b1dcb4 [Jonathan Alter] More println cleanup aedaf80 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977 925fd98 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977 0c16fa3 [Jonathan Alter] Replacing some printlns with logs 45c7e05 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977 5c8e283 [Jonathan Alter] Allowing println in audit-release examples 5b50da1 [Jonathan Alter] Allowing printlns in example files ca4b477 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977 83ab635 [Jonathan Alter] Fixing new printlns 54b131f [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977 1cd8a81 [Jonathan Alter] Removing some unnecessary comments and printlns b837c3a [Jonathan Alter] Disallowing println
* [SPARK-8701] [STREAMING] [WEBUI] Add input metadata in the batch pagezsxwing2015-07-099-26/+98
| | | | | | | | | | | | | | | | | | | | | | This PR adds `metadata` to `InputInfo`. `InputDStream` can report its metadata for a batch and it will be shown in the batch page. For example, ![screen shot](https://cloud.githubusercontent.com/assets/1000778/8403741/d6ffc7e2-1e79-11e5-9888-c78c1575123a.png) FileInputDStream will display the new files for a batch, and DirectKafkaInputDStream will display its offset ranges. Author: zsxwing <zsxwing@gmail.com> Closes #7081 from zsxwing/input-metadata and squashes the following commits: f7abd9b [zsxwing] Revert the space changes in project/MimaExcludes.scala d906209 [zsxwing] Merge branch 'master' into input-metadata 74762da [zsxwing] Fix MiMa tests 7903e33 [zsxwing] Merge branch 'master' into input-metadata 450a46c [zsxwing] Address comments 1d94582 [zsxwing] Raname InputInfo to StreamInputInfo and change "metadata" to Map[String, Any] d496ae9 [zsxwing] Add input metadata in the batch page
* [MINOR] [STREAMING] Fix log statements in ReceiverSupervisorImplTathagata Das2015-07-091-3/+3
| | | | | | | | | | Log statements incorrectly showed that the executor was being stopped when receiver was being stopped. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #7328 from tdas/fix-log and squashes the following commits: 9cc6e99 [Tathagata Das] Fix log statements.
* [SPARK-8619] [STREAMING] Don't recover keytab and principal configuration ↵huangzhaowei2015-06-301-2/+14
| | | | | | | | | | | | | | | | | | | | within Streaming checkpoint [Client.scala](https://github.com/apache/spark/blob/master/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L786) will change these configurations, so this would cause the problem that the Streaming recover logic can't find the local keytab file(since configuration was changed) ```scala sparkConf.set("spark.yarn.keytab", keytabFileName) sparkConf.set("spark.yarn.principal", args.principal) ``` Problem described at [Jira](https://issues.apache.org/jira/browse/SPARK-8619) Author: huangzhaowei <carlmartinmax@gmail.com> Closes #7008 from SaintBacchus/SPARK-8619 and squashes the following commits: d50dbdf [huangzhaowei] Delect one blank space 9b8e92c [huangzhaowei] Fix code style and add a short comment. 0d8f800 [huangzhaowei] Don't recover keytab and principal configuration within Streaming checkpoint.