aboutsummaryrefslogtreecommitdiff
path: root/streaming
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-7282] [STREAMING] Fix the race conditions in StreamingListenerSuitezsxwing2015-04-301-7/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Fixed the following flaky test ```Scala [info] StreamingListenerSuite: [info] - batch info reporting (782 milliseconds) [info] - receiver info reporting *** FAILED *** (3 seconds, 911 milliseconds) [info] The code passed to eventually never returned normally. Attempted 10 times over 3.4735783689999997 seconds. Last failure message: 0 did not equal 1. (StreamingListenerSuite.scala:104) [info] org.scalatest.exceptions.TestFailedDueToTimeoutException: [info] at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) [info] at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) [info] at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) [info] at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307) [info] at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply$mcV$sp(StreamingListenerSuite.scala:104) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94) [info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) [info] at org.scalatest.Transformer.apply(Transformer.scala:22) [info] at org.scalatest.Transformer.apply(Transformer.scala:20) [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) [info] at org.scalatest.Suite$class.withFixture(Suite.scala:1122) [info] at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) [info] at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) [info] at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingListenerSuite.scala:34) [info] at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) [info] at org.apache.spark.streaming.StreamingListenerSuite.runTest(StreamingListenerSuite.scala:34) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) [info] at scala.collection.immutable.List.foreach(List.scala:318) [info] at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) [info] at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) [info] at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) [info] at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) [info] at org.scalatest.Suite$class.run(Suite.scala:1424) [info] at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) [info] at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) [info] at org.scalatest.SuperEngine.runImpl(Engine.scala:545) [info] at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) [info] at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$run(StreamingListenerSuite.scala:34) [info] at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) [info] at org.apache.spark.streaming.StreamingListenerSuite.run(StreamingListenerSuite.scala:34) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:294) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:284) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:262) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [info] at java.lang.Thread.run(Thread.java:745) [info] Cause: org.scalatest.exceptions.TestFailedException: 0 did not equal 1 [info] at org.scalatest.MatchersHelper$.newTestFailedException(MatchersHelper.scala:160) [info] at org.scalatest.Matchers$ShouldMethodHelper$.shouldMatcher(Matchers.scala:6231) [info] at org.scalatest.Matchers$AnyShouldWrapper.should(Matchers.scala:6277) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2$$anonfun$apply$mcV$sp$1.apply$mcV$sp(StreamingListenerSuite.scala:105) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2$$anonfun$apply$mcV$sp$1.apply(StreamingListenerSuite.scala:104) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2$$anonfun$apply$mcV$sp$1.apply(StreamingListenerSuite.scala:104) [info] at org.scalatest.concurrent.Eventually$class.makeAValiantAttempt$1(Eventually.scala:394) [info] at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:408) [info] at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) [info] at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) [info] at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307) [info] at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply$mcV$sp(StreamingListenerSuite.scala:104) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94) [info] at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94) [info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) [info] at org.scalatest.Transformer.apply(Transformer.scala:22) [info] at org.scalatest.Transformer.apply(Transformer.scala:20) [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) [info] at org.scalatest.Suite$class.withFixture(Suite.scala:1122) [info] at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) [info] at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) [info] at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingListenerSuite.scala:34) [info] at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) [info] at org.apache.spark.streaming.StreamingListenerSuite.runTest(StreamingListenerSuite.scala:34) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) [info] at scala.collection.immutable.List.foreach(List.scala:318) [info] at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) [info] at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) [info] at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) [info] at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) [info] at org.scalatest.Suite$class.run(Suite.scala:1424) [info] at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) [info] at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) [info] at org.scalatest.SuperEngine.runImpl(Engine.scala:545) [info] at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) [info] at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$run(StreamingListenerSuite.scala:34) [info] at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) [info] at org.apache.spark.streaming.StreamingListenerSuite.run(StreamingListenerSuite.scala:34) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:294) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:284) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:262) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [info] at java.lang.Thread.run(Thread.java:745) ``` The original codes didn't have a memory barrier in the `eventually` closure, which might fail the test, because JVM doesn't guarantee the memory consistency between different threads without a memory barrier. This PR used `ConcurrentLinkedQueue` to set up the memory barrier. Author: zsxwing <zsxwing@gmail.com> Closes #5812 from zsxwing/SPARK-7282 and squashes the following commits: 59115ef [zsxwing] Use SynchronizedBuffer 014dd2b [zsxwing] Fix the race conditions in StreamingListenerSuite
* [SPARK-6862] [STREAMING] [WEBUI] Add BatchPage to display details of a batchzsxwing2015-04-2911-80/+709
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This is an initial commit for SPARK-6862. Once SPARK-6796 is merged, I will add the links to StreamingPage so that the user can jump to BatchPage. Screenshots: ![success](https://cloud.githubusercontent.com/assets/1000778/7102439/bbe75406-e0b3-11e4-84fe-3e6de629a49a.png) ![failure](https://cloud.githubusercontent.com/assets/1000778/7102440/bc124454-e0b3-11e4-921a-c8b39d6b61bc.png) Author: zsxwing <zsxwing@gmail.com> Closes #5473 from zsxwing/SPARK-6862 and squashes the following commits: 0727d35 [zsxwing] Change BatchUIData to a case class b380cfb [zsxwing] Add createJobStart to eliminate duplicate codes 9a3083d [zsxwing] Rename XxxDatas -> XxxData 087ba98 [zsxwing] Refactor BatchInfo to store only necessary fields cb62e4f [zsxwing] Use Seq[(OutputOpId, SparkJobId)] to store the id relations 72f8e7e [zsxwing] Add unit tests for BatchPage 1282b10 [zsxwing] Handle some corner cases and add tests for StreamingJobProgressListener 77a69ae [zsxwing] Refactor codes as per TD's comments 35ffd80 [zsxwing] Merge branch 'master' into SPARK-6862 15bdf9b [zsxwing] Add batch links and unit tests 4bf66b6 [zsxwing] Merge branch 'master' into SPARK-6862 7168807 [zsxwing] Limit the max width of the error message and fix nits in the UI 0b226f9 [zsxwing] Change 'Last Error' to 'Error' fc98a43 [zsxwing] Put clearing local properties to finally and remove redundant private[streaming] 0c7b2eb [zsxwing] Add BatchPage to display details of a batch
* [SPARK-6752] [STREAMING] [REOPENED] Allow StreamingContext to be recreated ↵Tathagata Das2015-04-296-61/+476
| | | | | | | | | | | | | | | | | | | | from checkpoint and existing SparkContext Original PR #5428 got reverted due to issues between MutableBoolean and Hadoop 1.0.4 (see JIRA). This replaces MutableBoolean with AtomicBoolean. srowen pwendell Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #5773 from tdas/SPARK-6752 and squashes the following commits: a0c0ead [Tathagata Das] Fix for hadoop 1.0.4 70ae85b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-6752 94db63c [Tathagata Das] Fix long line. 524f519 [Tathagata Das] Many changes based on PR comments. eabd092 [Tathagata Das] Added Function0, Java API and unit tests for StreamingContext.getOrCreate 36a7823 [Tathagata Das] Minor changes. 204814e [Tathagata Das] Added StreamingContext.getOrCreate with existing SparkContext
* [SPARK-7056] [STREAMING] Make the Write Ahead Log pluggableTathagata Das2015-04-2921-206/+684
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Users may want the WAL data to be written to non-HDFS data storage systems. To allow that, we have to make the WAL pluggable. The following design doc outlines the plan. https://docs.google.com/a/databricks.com/document/d/1A2XaOLRFzvIZSi18i_luNw5Rmm9j2j4AigktXxIYxmY/edit?usp=sharing Things to add. * Unit tests for WriteAheadLogUtils Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #5645 from tdas/wal-pluggable and squashes the following commits: 2c431fd [Tathagata Das] Minor fixes. c2bc7384 [Tathagata Das] More changes based on PR comments. 569a416 [Tathagata Das] fixed long line bde26b1 [Tathagata Das] Renamed segment to record handle everywhere b65e155 [Tathagata Das] More changes based on PR comments. d7cd15b [Tathagata Das] Fixed test 1a32a4b [Tathagata Das] Fixed test e0d19fb [Tathagata Das] Fixed defaults 9310cbf [Tathagata Das] style fix. 86abcb1 [Tathagata Das] Refactored WriteAheadLogUtils, and consolidated all WAL related configuration into it. 84ce469 [Tathagata Das] Added unit test and fixed compilation error. bce5e75 [Tathagata Das] Fixed long lines. 837c4f5 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into wal-pluggable 754fbf8 [Tathagata Das] Added license and docs. 09bc6fe [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into wal-pluggable 7dd2d4b [Tathagata Das] Added pluggable WriteAheadLog interface, and refactored all code along with it
* [SPARK-7223] Rename RPC askWithReply -> askWithReply, sendWithReply -> ask.Reynold Xin2015-04-291-3/+3
| | | | | | | | | | The old naming scheme was very confusing between askWithReply and sendWithReply. I also divided RpcEnv.scala into multiple files. Author: Reynold Xin <rxin@databricks.com> Closes #5768 from rxin/rpc-rename and squashes the following commits: a84058e [Reynold Xin] [SPARK-7223] Rename RPC askWithReply -> askWithReply, sendWithReply -> ask.
* [SPARK-7138] [STREAMING] Add method to BlockGenerator to add multiple ↵Tathagata Das2015-04-281-0/+14
| | | | | | | | | | | | | records to BlockGenerator with single callback This is to ensure that receivers that receive data in small batches (like Kinesis) and want to add them but want the callback function to be called only once. This is for internal use only for improvement to Kinesis Receiver that we are planning to do. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #5695 from tdas/SPARK-7138 and squashes the following commits: a35cf7d [Tathagata Das] Fixed style. a7a4cb9 [Tathagata Das] Added extra method to BlockGenerator.
* Revert "[SPARK-6752][Streaming] Allow StreamingContext to be recreated from ↵Patrick Wendell2015-04-256-476/+61
| | | | | | checkpoint and existing SparkContext" This reverts commit 534f2a43625fbf1a3a65d09550a19875cd1dce43.
* [SPARK-6752][Streaming] Allow StreamingContext to be recreated from ↵Tathagata Das2015-04-236-61/+476
| | | | | | | | | | | | | | | | | | | | | | | | checkpoint and existing SparkContext Currently if you want to create a StreamingContext from checkpoint information, the system will create a new SparkContext. This prevent StreamingContext to be recreated from checkpoints in managed environments where SparkContext is precreated. The solution in this PR: Introduce the following methods on StreamingContext 1. `new StreamingContext(checkpointDirectory, sparkContext)` Recreate StreamingContext from checkpoint using the provided SparkContext 2. `StreamingContext.getOrCreate(checkpointDirectory, sparkContext, createFunction: SparkContext => StreamingContext)` If checkpoint file exists, then recreate StreamingContext using the provided SparkContext (that is, call 1.), else create StreamingContext using the provided createFunction TODO: the corresponding Java and Python API has to be added as well. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #5428 from tdas/SPARK-6752 and squashes the following commits: 94db63c [Tathagata Das] Fix long line. 524f519 [Tathagata Das] Many changes based on PR comments. eabd092 [Tathagata Das] Added Function0, Java API and unit tests for StreamingContext.getOrCreate 36a7823 [Tathagata Das] Minor changes. 204814e [Tathagata Das] Added StreamingContext.getOrCreate with existing SparkContext
* Disable flaky test: ReceiverSuite "block generator throttling".Reynold Xin2015-04-221-1/+1
|
* [SPARK-7052][Core] Add ThreadUtils and move thread methods from Utils to ↵zsxwing2015-04-222-4/+4
| | | | | | | | | | | | | | ThreadUtils As per rxin 's suggestion in https://github.com/apache/spark/pull/5392/files#r28757176 What's more, there is a race condition in the global shared `daemonThreadFactoryBuilder`. `daemonThreadFactoryBuilder` may be modified by multiple threads. This PR removed the global `daemonThreadFactoryBuilder` and created a new `ThreadFactoryBuilder` every time. Author: zsxwing <zsxwing@gmail.com> Closes #5631 from zsxwing/thread-utils and squashes the following commits: 9fe5b0e [zsxwing] Add ThreadUtils and move thread methods from Utils to ThreadUtils
* SPARK-3276 Added a new configuration spark.streaming.minRememberDurationemres2015-04-211-13/+17
| | | | | | | | | | | | | | | | | | SPARK-3276 Added a new configuration parameter ``spark.streaming.minRememberDuration``, with a default value of 1 minute. So that when a Spark Streaming application is started, an arbitrary number of minutes can be taken as threshold for remembering. Author: emres <emre.sevinc@gmail.com> Closes #5438 from emres/SPARK-3276 and squashes the following commits: 766f938 [emres] SPARK-3276 Switched to using newly added getTimeAsSeconds method. affee1d [emres] SPARK-3276 Changed the property name and variable name for minRememberDuration c9d58ca [emres] SPARK-3276 Minor code re-formatting. 1c53ba9 [emres] SPARK-3276 Started to use ssc.conf rather than ssc.sparkContext.getConf, and also getLong method directly. bfe0acb [emres] SPARK-3276 Moved the minRememberDurationMin to the class daccc82 [emres] SPARK-3276 Changed the property name to reflect the unit of value and reduced number of fields. 43cc1ce [emres] SPARK-3276 Added a new configuration parameter spark.streaming.minRemember duration, with a default value of 1 minute.
* [SPARK-6985][streaming] Receiver maxRate over 1000 causes a StackOverflowErrorDavid McGuire2015-04-212-41/+21
| | | | | | | | | | | | | | | | | | | | | | | A simple truncation in integer division (on rates over 1000 messages / second) causes the existing implementation to sleep for 0 milliseconds, then call itself recursively; this causes what is essentially an infinite recursion, since the base case of the calculated amount of time having elapsed can't be reached before available stack space is exhausted. A fix to this truncation error is included in this patch. However, even with the defect patched, the accuracy of the existing implementation is abysmal (the error bounds of the original test were effectively [-30%, +10%], although this fact was obscured by hard-coded error margins); as such, when the error bounds were tightened down to [-5%, +5%], the existing implementation failed to meet the new, tightened, requirements. Therefore, an industry-vetted solution (from Guava) was used to get the adapted tests to pass. Author: David McGuire <david.mcguire2@nike.com> Closes #5559 from dmcguire81/master and squashes the following commits: d29d2e0 [David McGuire] Back out to +/-5% error margins, for flexibility in timing 8be6934 [David McGuire] Fix spacing per code review 90e98b9 [David McGuire] Address scalastyle errors 29011bd [David McGuire] Further ratchet down the error margins b33b796 [David McGuire] Eliminate dependency on even distribution by BlockGenerator 8f2934b [David McGuire] Remove arbitrary thread timing / cooperation code 70ee310 [David McGuire] Use Thread.yield(), since Thread.sleep(0) is system-dependent 82ee46d [David McGuire] Replace guard clause with nested conditional 2794717 [David McGuire] Replace the RateLimiter with the Guava implementation 38f3ca8 [David McGuire] Ratchet down the error rate to +/- 5%; tests fail 24b1bc0 [David McGuire] Fix truncation in integer division causing infinite recursion d6e1079 [David McGuire] Stack overflow error in RateLimiter on rates over 1000/s
* [SPARK-6979][Streaming] Replace JobScheduler.eventActor and ↵zsxwing2015-04-192-37/+41
| | | | | | | | | | | | | | | | | | | JobGenerator.eventActor with EventLoop Title says it all. cc rxin tdas Author: zsxwing <zsxwing@gmail.com> Closes #5554 from zsxwing/SPARK-6979 and squashes the following commits: 5304350 [zsxwing] Fix NotSerializableException e9d3479 [zsxwing] Add blank lines 633e279 [zsxwing] Fix NotSerializableException e496ace [zsxwing] Replace JobGenerator.eventActor with EventLoop ec6ec58 [zsxwing] Fix the import order ce0fa73 [zsxwing] Replace JobScheduler.eventActor with EventLoop
* [SPARK-6983][Streaming] Update ReceiverTrackerActor to use the new Rpc interfacezsxwing2015-04-193-69/+51
| | | | | | | | | | A subtask of [SPARK-5293](https://issues.apache.org/jira/browse/SPARK-5293) Author: zsxwing <zsxwing@gmail.com> Closes #5557 from zsxwing/SPARK-6983 and squashes the following commits: e777e9f [zsxwing] Update ReceiverTrackerActor to use the new Rpc interface
* [SPARK-6796][Streaming][WebUI] Add "Active Batches" and "Completed Batches" ↵zsxwing2015-04-143-10/+159
| | | | | | | | | | | | | | | | | | | | | | | | | | lists to StreamingPage This PR adds two lists, `Active Batches` and `Completed Batches`. Here is the screenshot: ![batch_list](https://cloud.githubusercontent.com/assets/1000778/7060458/d8898572-deb3-11e4-938b-6f8602c71a9f.png) Due to [SPARK-6766](https://issues.apache.org/jira/browse/SPARK-6766), I need to merge #5414 in my local machine to get the above screenshot. Author: zsxwing <zsxwing@gmail.com> Closes #5434 from zsxwing/SPARK-6796 and squashes the following commits: be50fc6 [zsxwing] Fix the code style 51b792e [zsxwing] Fix the unit test 6f3078e [zsxwing] Make 'startTime' readable f40e0a9 [zsxwing] Merge branch 'master' into SPARK-6796 2525336 [zsxwing] Rename 'Processed batches' and 'Waiting batches' and also add links a69c091 [zsxwing] Show the number of total completed batches too a12ad7b [zsxwing] Change 'records' to 'events' in the UI 86b5e7f [zsxwing] Make BatchTableBase abstract b248787 [zsxwing] Add tests to verify the new tables d18ab7d [zsxwing] Fix the code style 6ceffb3 [zsxwing] Add "Active Batches" and "Completed Batches" lists to StreamingPage
* [SPARK-5931][CORE] Use consistent naming for time propertiesIlya Ganelin2015-04-134-28/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
* [SPARK-6765] Enable scalastyle on test code.Reynold Xin2015-04-131-2/+2
| | | | | | | | | | | | | | | | | | | Turn scalastyle on for all test code. Most of the violations have been resolved in my previous pull requests: Core: https://github.com/apache/spark/pull/5484 SQL: https://github.com/apache/spark/pull/5412 MLlib: https://github.com/apache/spark/pull/5411 GraphX: https://github.com/apache/spark/pull/5410 Streaming: https://github.com/apache/spark/pull/5409 Author: Reynold Xin <rxin@databricks.com> Closes #5486 from rxin/test-style-enable and squashes the following commits: 01683de [Reynold Xin] Fixed new code. a4ab46e [Reynold Xin] Fixed tests. 20adbc8 [Reynold Xin] Missed one violation. 5e36521 [Reynold Xin] [SPARK-6765] Enable scalastyle on test code.
* [SPARK-6860][Streaming][WebUI] Fix the possible inconsistency of StreamingPagezsxwing2015-04-132-6/+7
| | | | | | | | | | | | | Because `StreamingPage.render` doesn't hold the `listener` lock when generating the content, the different parts of content may have some inconsistent values if `listener` updates its status at the same time. And it will confuse people. This PR added `listener.synchronized` to make sure we have a consistent view of StreamingJobProgressListener when creating the content. Author: zsxwing <zsxwing@gmail.com> Closes #5470 from zsxwing/SPARK-6860 and squashes the following commits: cec6f92 [zsxwing] Add missing 'synchronized' in StreamingJobProgressListener 7182498 [zsxwing] Add synchronized to make sure we have a consistent view of StreamingJobProgressListener when creating the content
* [SPARK-6762]Fix potential resource leaks in CheckPoint CheckpointWriter and ↵lisurprise2015-04-132-19/+31
| | | | | | | | | | | | | | | | CheckpointReader The close action should be placed within finally block to avoid the potential resource leaks Author: lisurprise <zhichao.li@intel.com> Closes #5407 from zhichao-li/master and squashes the following commits: 065999f [lisurprise] add guard for null ef862d6 [lisurprise] remove fs.close a754adc [lisurprise] refactor with tryWithSafeFinally 824adb3 [lisurprise] close before validation c877da7 [lisurprise] Fix potential resource leaks
* [SPARK-6766][Streaming] Fix issue about StreamingListenerBatchSubmitted and ↵zsxwing2015-04-104-18/+180
| | | | | | | | | | | | | | | | | | | | | StreamingListenerBatchStarted This PR includes: 1. Send `StreamingListenerBatchSubmitted` when `JobSet` is submitted 1. Fix `StreamingListenerBatchStarted.batchInfo.processingStartTime` 1. Fix a type: `completedaBatchInfos` -> `completedBatchInfos` Author: zsxwing <zsxwing@gmail.com> Closes #5414 from zsxwing/SPARK-6766 and squashes the following commits: 2f85060 [zsxwing] Update tests ca0955b [zsxwing] Combine unit tests 79b4fed [zsxwing] Add StreamingJobProgressListenerSuite to test StreamingJobProgressListener fc3a2a1 [zsxwing] Add unit tests for SPARK-6766 74aed99 [zsxwing] Refactor as per TD's suggestion 493f978 [zsxwing] Send StreamingListenerBatchSubmitted when JobSet is submitted; fix StreamingListenerBatchStarted.batchInfo.processingStartTime; fix a typo
* [SPARK-6758]block the right jetty package in logWangTaoTheTonic2015-04-091-1/+1
| | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-6758 I am not sure if it is ok to block them in test resources too (as we shade jetty in assembly?). Author: WangTaoTheTonic <wangtao111@huawei.com> Closes #5406 from WangTaoTheTonic/SPARK-6758 and squashes the following commits: e09605b [WangTaoTheTonic] block the right jetty package
* [SPARK-6765] Fix test code style for streaming.Reynold Xin2015-04-0816-58/+96
| | | | | | | | | | So we can turn style checker on for test code. Author: Reynold Xin <rxin@databricks.com> Closes #5409 from rxin/test-style-streaming and squashes the following commits: 7aea69b [Reynold Xin] [SPARK-6765] Fix test code style for streaming.
* [SPARK-6602][Core] Replace direct use of Akka with Spark RPC interface - part 1zsxwing2015-04-041-14/+11
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR replaced the following `Actor`s to `RpcEndpoint`: 1. HeartbeatReceiver 1. ExecutorActor 1. BlockManagerMasterActor 1. BlockManagerSlaveActor 1. CoarseGrainedExecutorBackend and subclasses 1. CoarseGrainedSchedulerBackend.DriverActor This is the first PR. I will split the work of SPARK-6602 to several PRs for code review. Author: zsxwing <zsxwing@gmail.com> Closes #5268 from zsxwing/rpc-rewrite and squashes the following commits: 287e9f8 [zsxwing] Fix the code style 26c56b7 [zsxwing] Merge branch 'master' into rpc-rewrite 9cc825a [zsxwing] Rmove setupThreadSafeEndpoint and add ThreadSafeRpcEndpoint 30a9036 [zsxwing] Make self return null after stopping RpcEndpointRef; fix docs and error messages 705245d [zsxwing] Fix some bugs after rebasing the changes on the master 003cf80 [zsxwing] Update CoarseGrainedExecutorBackend and CoarseGrainedSchedulerBackend to use RpcEndpoint 7d0e6dc [zsxwing] Update BlockManagerSlaveActor to use RpcEndpoint f5d6543 [zsxwing] Update BlockManagerMaster to use RpcEndpoint 30e3f9f [zsxwing] Update ExecutorActor to use RpcEndpoint 478b443 [zsxwing] Update HeartbeatReceiver to use RpcEndpoint
* [SPARK-6428] Turn on explicit type checking for public methods.Reynold Xin2015-04-035-12/+16
| | | | | | | | | | | | | | This builds on my earlier pull requests and turns on the explicit type checking in scalastyle. Author: Reynold Xin <rxin@databricks.com> Closes #5342 from rxin/SPARK-6428 and squashes the following commits: 7b531ab [Reynold Xin] import ordering 2d9a8a5 [Reynold Xin] jl e668b1c [Reynold Xin] override 9b9e119 [Reynold Xin] Parenthesis. 82e0cf5 [Reynold Xin] [SPARK-6428] Turn on explicit type checking for public methods.
* SPARK-6433 hive tests to import spark-sql test JAR for QueryTest accessSteve Loughran2015-04-011-28/+0
| | | | | | | | | | | | | | | | | | | | | | | 1. Test JARs are built & published 1. log4j.resources is explicitly excluded. Without this, downstream test run logging depends on the order the JARs are listed/loaded 1. sql/hive pulls in spark-sql &...spark-catalyst for its test runs 1. The copied in test classes were rm'd, and a test edited to remove its now duplicate assert method 1. Spark streaming is now build with the same plugin/phase as the rest, but its shade plugin declaration is kept in (so different from the rest of the test plugins). Due to (#2), this means the test JAR no longer includes its log4j file. Outstanding issues: * should the JARs be shaded? `spark-streaming-test.jar` does, but given these are test jars for developers only, especially in the same spark source tree, it's hard to justify. * `maven-jar-plugin` v 2.6 was explicitly selected; without this the apache-1.4 parent template JAR version (2.4) chosen. * Are there any other resources to exclude? Author: Steve Loughran <stevel@hortonworks.com> Closes #5119 from steveloughran/stevel/patches/SPARK-6433-test-jars and squashes the following commits: 81ceb01 [Steve Loughran] SPARK-6433 add a clearer comment explaining what the plugin is doing & why a6dca33 [Steve Loughran] SPARK-6433 : pull configuration section form archive plugin c2b5f89 [Steve Loughran] SPARK-6433 omit "jar" goal from jar plugin fdac51b [Steve Loughran] SPARK-6433 -002; indentation & delegate plugin version to parent 650f442 [Steve Loughran] SPARK-6433 patch 001: test JARs are built; sql/hive pulls in spark-sql & spark-catalyst for its test runs
* [SPARK-6428][Streaming] Added explicit types for all public methods.Reynold Xin2015-03-2438-114/+127
| | | | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #5110 from rxin/streaming-explicit-type and squashes the following commits: 2c2db32 [Reynold Xin] [SPARK-6428][Streaming] Added explicit types for all public methods.
* [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.Marcelo Vanzin2015-03-201-1/+1
| | | | | | | | | | | | | | | | Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #5056 from vanzin/SPARK-6371 and squashes the following commits: 63220df [Marcelo Vanzin] Merge branch 'master' into SPARK-6371 6506f75 [Marcelo Vanzin] Use more fine-grained exclusion. 178ba71 [Marcelo Vanzin] Oops. 75b2375 [Marcelo Vanzin] Exclude VertexRDD in MiMA. a45a62c [Marcelo Vanzin] Work around MIMA warning. 1d8a670 [Marcelo Vanzin] Re-group jetty exclusion. 0e8e909 [Marcelo Vanzin] Ignore ml, don't ignore graphx. cef4603 [Marcelo Vanzin] Indentation. 296cf82 [Marcelo Vanzin] [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
* SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid ↵Sean Owen2015-03-205-27/+14
| | | | | | | | | | | | | | | orphaned temp files Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify Author: Sean Owen <sowen@cloudera.com> Closes #5029 from srowen/SPARK-6338 and squashes the following commits: 27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir 4a212fa [Sean Owen] Standardize a bit more temp dir management 9004081 [Sean Owen] Revert some added recursive-delete calls 57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify
* [SPARK-4012] stop SparkContext when the exception is thrown from an infinite ↵CodingCat2015-03-181-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | loop https://issues.apache.org/jira/browse/SPARK-4012 This patch is a resubmission for https://github.com/apache/spark/pull/2864 What I am proposing in this patch is that ***when the exception is thrown from an infinite loop, we should stop the SparkContext, instead of let JVM throws exception forever*** So, in the infinite loops where we originally wrapped with a ` logUncaughtExceptions`, I changed to `tryOrStopSparkContext`, so that the Spark component is stopped Early stopped JVM process is helpful for HA scheme design, for example, The user has a script checking the existence of the pid of the Spark Streaming driver for monitoring the availability; with the code before this patch, the JVM process is still available but not functional when the exceptions are thrown andrewor14, srowen , mind taking further consideration about the change? Author: CodingCat <zhunansjtu@gmail.com> Closes #5004 from CodingCat/SPARK-4012-1 and squashes the following commits: 589276a [CodingCat] throw fatal error again 3c72cd8 [CodingCat] address the comments 6087864 [CodingCat] revise comments 6ad3eb0 [CodingCat] stop SparkContext instead of quit the JVM process 6322959 [CodingCat] exit JVM process when the exception is thrown from an infinite loop
* [SPARK-6222][Streaming] Dont delete checkpoint data when doing ↵Tathagata Das2015-03-193-12/+153
| | | | | | | | | | | | | | | | | | | | | pre-batch-start checkpoint This is another alternative approach to https://github.com/apache/spark/pull/4964/ I think this is a simpler fix that can be backported easily to other branches (1.2 and 1.3). All it does it introduce a flag so that the pre-batch-start checkpoint does not call clear checkpoint. There is not unit test yet. I will add it when this approach is commented upon. Not sure if this is testable easily. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #5008 from tdas/SPARK-6222 and squashes the following commits: 7315bc2 [Tathagata Das] Removed empty line. c438de4 [Tathagata Das] Revert unnecessary change. 5e98374 [Tathagata Das] Added unit test 50cb60b [Tathagata Das] Fixed style issue 295ca5c [Tathagata Das] Fixing SPARK-6222
* [SPARK-3266] Use intermediate abstract classes to fix type erasure issues in ↵Josh Rosen2015-03-173-2/+11
| | | | | | | | | | | | | | | | | Java APIs This PR addresses a Scala compiler bug ([SI-8905](https://issues.scala-lang.org/browse/SI-8905)) that was breaking some of the Spark Java APIs. In a nutshell, it seems that methods whose implementations are inherited from generic traits sometimes have their type parameters erased to Object. This was causing methods like `DoubleRDD.min()` to throw confusing NoSuchMethodErrors at runtime. The fix implemented here is to introduce an intermediate layer of abstract classes and inherit from those instead of directly extends the `Java*Like` traits. This should not break binary compatibility. I also improved the test coverage of the Java API, adding several new tests for methods that failed at runtime due to this bug. Author: Josh Rosen <joshrosen@databricks.com> Closes #5050 from JoshRosen/javardd-si-8905-fix and squashes the following commits: 2feb068 [Josh Rosen] Use intermediate abstract classes to work around SPARK-3266 d5f3e5d [Josh Rosen] Add failing regression tests for SPARK-3266
* [SPARK-6331] Load new master URL if present when recovering streaming ↵Tathagata Das2015-03-174-7/+25
| | | | | | | | | | | | | | | | | context from checkpoint In streaming driver recovery, when the SparkConf is reconstructed based on the checkpointed configuration, it recovers the old master URL. This okay if the cluster on which the streaming application is relaunched is the same cluster as it was running before. But if that cluster changes, there is no way to inject the new master URL of the new cluster. As a result, the restarted app tries to connect to the non-existent old cluster and fails. The solution is to check whether a master URL is set in the System properties (by Spark submit) before recreating the SparkConf. If a new master url is set in the properties, then use it as that is obviously the most relevant one. Otherwise load the old one (to maintain existing behavior). Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #5024 from tdas/SPARK-6331 and squashes the following commits: 392fd44 [Tathagata Das] Fixed naming issue. c7c0b99 [Tathagata Das] Addressed comments. 6a0857c [Tathagata Das] Updated testsuites. 222485d [Tathagata Das] Load new master URL if present when recovering streaming context from checkpoint
* [SPARK-6077] Remove streaming tab while stopping StreamingContextlisurprise2015-03-166-57/+107
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Currently we would create a new streaming tab for each streamingContext even if there's already one on the same sparkContext which would cause duplicate StreamingTab created and none of them is taking effect. snapshot: https://www.dropbox.com/s/t4gd6hqyqo0nivz/bad%20multiple%20streamings.png?dl=0 How to reproduce: 1) import org.apache.spark.SparkConf import org.apache.spark.streaming. {Seconds, StreamingContext} import org.apache.spark.storage.StorageLevel val ssc = new StreamingContext(sc, Seconds(1)) val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() ..... 2) ssc.stop(false) val ssc = new StreamingContext(sc, Seconds(1)) val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() Author: lisurprise <zhichao.li@intel.com> Closes #4828 from zhichao-li/master and squashes the following commits: c329806 [lisurprise] add test for attaching/detaching streaming tab 51e6c7f [lisurprise] move detach method into StreamingTab 31a44fa [lisurprise] add unit test for attaching and detaching new tab db25ed2 [lisurprise] clean code 8281bcb [lisurprise] clean code 193c542 [lisurprise] remove streaming tab while closing streaming context
* SPARK-6225 [CORE] [SQL] [STREAMING] Resolve most build warnings, 1.3.0 editionSean Owen2015-03-111-0/+4
| | | | | | | | | | | Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc. Author: Sean Owen <sowen@cloudera.com> Closes #4950 from srowen/SPARK-6225 and squashes the following commits: 3080972 [Sean Owen] Ordered imports: Java, Scala, 3rd party, Spark c67985b [Sean Owen] Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.
* [Minor] Resolve sbt warnings: postfix operator second should be enabledGuoQiang Li2015-03-061-0/+1
| | | | | | | | | | | | | | | | | | | | | Resolve sbt warnings: ``` [warn] spark/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala:155: postfix operator second should be enabled [warn] by making the implicit value scala.language.postfixOps visible. [warn] This can be achieved by adding the import clause 'import scala.language.postfixOps' [warn] or by setting the compiler option -language:postfixOps. [warn] See the Scala docs for value scala.language.postfixOps for a discussion [warn] why the feature should be explicitly enabled. [warn] Await.ready(f, 1 second) [warn] ^ ``` Author: GuoQiang Li <witgo@qq.com> Closes #4908 from witgo/sbt_warnings and squashes the following commits: 0629af4 [GuoQiang Li] Resolve sbt warnings: postfix operator second should be enabled
* SPARK-6182 [BUILD] spark-parent pom needs to be published for both 2.10 and 2.11Sean Owen2015-03-051-1/+1
| | | | | | | | | | Option 1 of 2: Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11 Author: Sean Owen <sowen@cloudera.com> Closes #4912 from srowen/SPARK-6182.1 and squashes the following commits: eff60de [Sean Owen] Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11
* [SPARK-5943][Streaming] Update the test to use new API to reduce the warningSaisai Shao2015-02-232-6/+6
| | | | | | | | Author: Saisai Shao <saisai.shao@intel.com> Closes #4722 from jerryshao/SPARK-5943 and squashes the following commits: 1b01233 [Saisai Shao] Update the test to use new API to reduce the warning
* SPARK-4682 [CORE] Consolidate various 'Clock' classesSean Owen2015-02-1916-171/+82
| | | | | | | | | | | | | | | Another one from JoshRosen 's wish list. The first commit is much smaller and removes 2 of the 4 Clock classes. The second is much larger, necessary for consolidating the streaming one. I put together implementations in the way that seemed simplest. Almost all the change is standardizing class and method names. Author: Sean Owen <sowen@cloudera.com> Closes #4514 from srowen/SPARK-4682 and squashes the following commits: 5ed3a03 [Sean Owen] Javadoc Clock classes; make ManualClock private[spark] 169dd13 [Sean Owen] Add support for legacy org.apache.spark.streaming clock class names 277785a [Sean Owen] Reduce the net change in this patch by reversing some unnecessary syntax changes along the way b5e53df [Sean Owen] FakeClock -> ManualClock; getTime() -> getTimeMillis() 160863a [Sean Owen] Consolidate Streaming Clock class into common util Clock 7c956b2 [Sean Owen] Consolidate Clocks except for Streaming Clock
* [SPARK-5826][Streaming] Fix Configuration not serializable problemjerryshao2015-02-171-2/+4
| | | | | | | | | Author: jerryshao <saisai.shao@intel.com> Closes #4612 from jerryshao/SPARK-5826 and squashes the following commits: 7ec71db [jerryshao] Remove transient for conf statement 88d84e6 [jerryshao] Fix Configuration not serializable problem
* SPARK-5795 [STREAMING] api.java.JavaPairDStream.saveAsNewAPIHadoopFiles may ↵Sean Owen2015-02-162-10/+28
| | | | | | | | | | | | | | | not friendly to java Revise JavaPairDStream API declaration on saveAs Hadoop methods, to allow it to be called directly as intended. CC tdas for review Author: Sean Owen <sowen@cloudera.com> Closes #4608 from srowen/SPARK-5795 and squashes the following commits: 36f1ead [Sean Owen] Add code that shows compile problem and fix 036bd27 [Sean Owen] Revise JavaPairDStream API declaration on saveAs Hadoop methods, to allow it to be called directly as intended.
* [SPARK-5831][Streaming]When checkpoint file size is bigger than 10, then ↵Xutingjun2015-02-161-1/+1
| | | | | | | | | | delete the old ones Author: Xutingjun <1039320815@qq.com> Closes #4621 from XuTingjun/checkpointFile and squashes the following commits: b5f2816 [Xutingjun] fix bug
* [Minor] Fix incorrect warning logLiang-Chi Hsieh2015-02-041-1/+1
| | | | | | | | | | The warning log looks incorrect. Just fix it. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #4360 from viirya/fixing_typo and squashes the following commits: 48fbe4f [Liang-Chi Hsieh] Fix incorrect warning log.
* [SPARK-5379][Streaming] Add awaitTerminationOrTimeoutzsxwing2015-02-043-0/+50
| | | | | | | | | | | | | | | | | Added `awaitTerminationOrTimeout` to return if the waiting time elapsed: * `true` if it's stopped. * `false` if the waiting time elapsed before returning from the method. * throw the reported error if it's thrown during the execution. Also deprecated `awaitTermination(timeout: Long)`. Author: zsxwing <zsxwing@gmail.com> Closes #4171 from zsxwing/SPARK-5379 and squashes the following commits: c9e660b [zsxwing] Add a unit test for awaitTerminationOrTimeout 8a89f92 [zsxwing] Add awaitTerminationOrTimeout to python cdc820b [zsxwing] Add awaitTerminationOrTimeout
* [SPARK-4969][STREAMING][PYTHON] Add binaryRecords to streamingfreeman2015-02-034-5/+174
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | In Spark 1.2 we added a `binaryRecords` input method for loading flat binary data. This format is useful for numerical array data, e.g. in scientific computing applications. This PR adds support for the same format in Streaming applications, where it is similarly useful, especially for streaming time series or sensor data. Summary of additions - adding `binaryRecordsStream` to Spark Streaming - exposing `binaryRecordsStream` in the new PySpark Streaming - new unit tests in Scala and Python This required adding an optional Hadoop configuration param to `fileStream` and `FileInputStream`, but was otherwise straightforward. tdas davies Author: freeman <the.freeman.lab@gmail.com> Closes #3803 from freeman-lab/streaming-binary-records and squashes the following commits: b676534 [freeman] Clarify note 5ff1b75 [freeman] Add note to java streaming context eba925c [freeman] Simplify notes c4237b8 [freeman] Add experimental tag 30eba67 [freeman] Add filter and newFilesOnly alongside conf c2cfa6d [freeman] Expose new version of fileStream with conf in java 34d20ef [freeman] Add experimental tag 14bca9a [freeman] Add experimental tag b85bffc [freeman] Formatting 47560f4 [freeman] Space formatting 9a3715a [freeman] Refactor to reflect changes to FileInputSuite 7373f73 [freeman] Add note and defensive assertion for byte length 3ceb684 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-binary-records 317b6d1 [freeman] Make test inline fcb915c [freeman] Formatting becb344 [freeman] Formatting d3e75b2 [freeman] Add tests in python a4324a3 [freeman] Line length 029d49c [freeman] Formatting 1c739aa [freeman] Simpler default arg handling 94d90d0 [freeman] Spelling 2843e9d [freeman] Add params to docstring 8b70fbc [freeman] Reorganization 28bff9b [freeman] Fix missing arg 9398bcb [freeman] Expose optional hadoop configuration 23dd69f [freeman] Tests for binaryRecordsStream 36cb0fd [freeman] Add binaryRecordsStream to scala fe4e803 [freeman] Add binaryRecordStream to Java API ecef0eb [freeman] Add binaryRecordsStream to python 8550c26 [freeman] Expose additional argument combination
* [STREAMING] SPARK-4986 Wait for receivers to deregister and receiver job to ↵Jesper Lundgren2015-02-033-4/+75
| | | | | | | | | | | | | | | | terminate A slow receiver might not have enough time to shutdown cleanly even when graceful shutdown is used. This PR extends graceful waiting to make sure all receivers have deregistered and that the receiver job has terminated. Author: Jesper Lundgren <jesper.lundgren@vpon.com> Closes #4338 from cleaton/stopreceivers and squashes the following commits: a9cf223 [Jesper Lundgren] remove cleaner.ttl config f969b6e [Jesper Lundgren] fix inversed logic in unit test 3d0bd35 [Jesper Lundgren] switch boleans to match running status instead of terminated 9a9ff88 [Jesper Lundgren] wait for receivers to shutdown and receiver job to terminate d179372 [Jesper Lundgren] Add graceful shutdown unit test covering slow receiver onStop
* SPARK-3996: Add jetty servlet and continuations.Patrick Wendell2015-02-021-0/+4
| | | | | | | | | | | | These are needed transitively from the other Jetty libraries we include. It was not picked up by unit tests because we disable the UI. Author: Patrick Wendell <patrick@databricks.com> Closes #4323 from pwendell/jetty and squashes the following commits: d8669da [Patrick Wendell] SPARK-3996: Add jetty servlet and continuations.
* Spark 3883: SSL support for HttpServer and AkkaJacek Lewandowski2015-02-021-2/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | SPARK-3883: SSL support for Akka connections and Jetty based file servers. This story introduced the following changes: - Introduced SSLOptions object which holds the SSL configuration and can build the appropriate configuration for Akka or Jetty. SSLOptions can be created by parsing SparkConf entries at a specified namespace. - SSLOptions is created and kept by SecurityManager - All Akka actor address creation snippets based on interpolated strings were replaced by a dedicated methods from AkkaUtils. Those methods select the proper Akka protocol - whether akka.tcp or akka.ssl.tcp - Added tests cases for AkkaUtils, FileServer, SSLOptions and SecurityManager - Added a way to use node local SSL configuration by executors and driver in standalone mode. It can be done by specifying spark.ssl.useNodeLocalConf in SparkConf. - Made CoarseGrainedExecutorBackend not overwrite the settings which are executor startup configuration - they are passed anyway from Worker Refer to https://github.com/apache/spark/pull/3571 for discussion and details Author: Jacek Lewandowski <lewandowski.jacek@gmail.com> Author: Jacek Lewandowski <jacek.lewandowski@datastax.com> Closes #3571 from jacek-lewandowski/SPARK-3883-master and squashes the following commits: 9ef4ed1 [Jacek Lewandowski] Merge pull request #2 from jacek-lewandowski/SPARK-3883-docs2 fb31b49 [Jacek Lewandowski] SPARK-3883: Added SSL setup documentation 2532668 [Jacek Lewandowski] SPARK-3883: Refactored AkkaUtils.protocol method to not use Try 90a8762 [Jacek Lewandowski] SPARK-3883: Refactored methods to resolve Akka address and made it possible to easily configure multiple communication layers for SSL 72b2541 [Jacek Lewandowski] SPARK-3883: A reference to the fallback SSLOptions can be provided when constructing SSLOptions 93050f4 [Jacek Lewandowski] SPARK-3883: SSL support for HttpServer and Akka
* Make sure only owner can read / write to directories created for the job.Marcelo Vanzin2015-02-021-2/+2
| | | | | | | Whenever a directory is created by the utility method, immediately restrict its permissions so that only the owner has access to its contents. Signed-off-by: Josh Rosen <joshrosen@databricks.com>
* [SPARK-3996]: Shade Jetty in Spark deliverablesPatrick Wendell2015-02-011-0/+16
| | | | | | | | | | | | | | | | | | | (v2 of this patch with a fix that was only relevant for the maven build). This patch piggy-back's on vanzin's work to simplify the Guava shading, and adds Jetty as a shaded library in Spark. Other than adding Jetty, it consilidates the <artifactSet>'s into the root pom. I found it was a bit easier to follow that way, since you don't need to look into child pom's to find out specific artifact sets included in shading. Author: Patrick Wendell <patrick@databricks.com> Closes #4285 from pwendell/jetty and squashes the following commits: d3e7f4e [Patrick Wendell] Fix for shaded deps causing compile errors 19f0710 [Patrick Wendell] More code review feedback 961452d [Patrick Wendell] Responding to feedback from Marcello 6df25ca [Patrick Wendell] [WIP] [SPARK-3996]: Shade Jetty in Spark deliverables
* [SPARK-4859][Core][Streaming] Refactor LiveListenerBus and StreamingListenerBuszsxwing2015-02-012-71/+27
| | | | | | | | | | | | | | | | | | | | | | | | | | This PR refactors LiveListenerBus and StreamingListenerBus and extracts the common codes to a parent class `ListenerBus`. It also includes bug fixes in #3710: 1. Fix the race condition of queueFullErrorMessageLogged in LiveListenerBus and StreamingListenerBus to avoid outputing `queue-full-error` logs multiple times. 2. Make sure the SHUTDOWN message will be delivered to listenerThread, so that we can make sure listenerThread will always be able to exit. 3. Log the error from listener rather than crashing listenerThread in StreamingListenerBus. During fixing the above bugs, we find it's better to make LiveListenerBus and StreamingListenerBus have the same bahaviors. Then there will be many duplicated codes in LiveListenerBus and StreamingListenerBus. Therefore, I extracted their common codes to `ListenerBus` as a parent class: LiveListenerBus and StreamingListenerBus only need to extend `ListenerBus` and implement `onPostEvent` (how to process an event) and `onDropEvent` (do something when droppping an event). Author: zsxwing <zsxwing@gmail.com> Closes #4006 from zsxwing/SPARK-4859-refactor and squashes the following commits: c8dade2 [zsxwing] Fix the code style after renaming 5715061 [zsxwing] Rename ListenerHelper to ListenerBus and the original ListenerBus to AsynchronousListenerBus f0ef647 [zsxwing] Fix the code style 4e85ffc [zsxwing] Merge branch 'master' into SPARK-4859-refactor d2ef990 [zsxwing] Add private[spark] 4539f91 [zsxwing] Remove final to pass MiMa tests a9dccd3 [zsxwing] Remove SparkListenerShutdown 7cc04c3 [zsxwing] Refactor LiveListenerBus and StreamingListenerBus and make them share same code base