aboutsummaryrefslogtreecommitdiff
path: root/external/kafka/src
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-1022][Streaming] Add Kafka real unit testjerryshao2014-08-052-35/+287
| | | | | | | | | | | | | | | | | This PR is a updated version of (https://github.com/apache/spark/pull/557) to actually test sending and receiving data through Kafka, and fix previous flaky issues. @tdas, would you mind reviewing this PR? Thanks a lot. Author: jerryshao <saisai.shao@intel.com> Closes #1751 from jerryshao/kafka-unit-test and squashes the following commits: b6a505f [jerryshao] code refactor according to comments 5222330 [jerryshao] Change JavaKafkaStreamSuite to better test it 5525f10 [jerryshao] Fix flaky issue of Kafka real unit test 4559310 [jerryshao] Minor changes for Kafka unit test 860f649 [jerryshao] Minor style changes, and tests ignored due to flakiness 796d4ca [jerryshao] Add real Kafka streaming test
* [SPARK-2103][Streaming] Change to ClassTag for KafkaInputDStream and fix ↵jerryshao2014-08-012-18/+12
| | | | | | | | | | | | | | | | | | | reflection issue This PR updates previous Manifest for KafkaInputDStream's Decoder to ClassTag, also fix the problem addressed in [SPARK-2103](https://issues.apache.org/jira/browse/SPARK-2103). Previous Java interface cannot actually get the type of Decoder, so when using this Manifest to reconstruct the decode object will meet reflection exception. Also for other two Java interfaces, ClassTag[String] is useless because calling Scala API will get the right implicit ClassTag. Current Kafka unit test cannot actually verify the interface. I've tested these interfaces in my local and distribute settings. Author: jerryshao <saisai.shao@intel.com> Closes #1508 from jerryshao/SPARK-2103 and squashes the following commits: e90c37b [jerryshao] Add Mima excludes 7529810 [jerryshao] Change Manifest to ClassTag for KafkaInputDStream's Decoder and fix Decoder construct issue when using Java API
* SPARK-2034. KafkaInputDStream doesn't close resources and may prevent JVM ↵Sean Owen2014-06-221-22/+33
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | shutdown Tobias noted today on the mailing list: ======== I am trying to use Spark Streaming with Kafka, which works like a charm – except for shutdown. When I run my program with "sbt run-main", sbt will never exit, because there are two non-daemon threads left that don't die. I created a minimal example at <https://gist.github.com/tgpfeiffer/b1e765064e983449c6b6#file-kafkadoesntshutdown-scala>. It starts a StreamingContext and does nothing more than connecting to a Kafka server and printing what it receives. Using the `future Unknown macro: { ... } ` construct, I shut down the StreamingContext after some seconds and then print the difference between the threads at start time and at end time. The output can be found at <https://gist.github.com/tgpfeiffer/b1e765064e983449c6b6#file-output1>. There are a number of threads remaining that will prevent sbt from exiting. When I replace `KafkaUtils.createStream(...)` with a call that does exactly the same, except that it calls `consumerConnector.shutdown()` in `KafkaReceiver.onStop()` (which it should, IMO), the output is as shown at <https://gist.github.com/tgpfeiffer/b1e765064e983449c6b6#file-output2>. Does anyone have any idea what is going on here and why the program doesn't shut down properly? The behavior is the same with both kafka 0.8.0 and 0.8.1.1, by the way. ======== Something similar was noted last year: http://mail-archives.apache.org/mod_mbox/spark-dev/201309.mbox/%3C1380220041.2428.YahooMailNeo@web160804.mail.bf1.yahoo.com%3E KafkaInputDStream doesn't close `ConsumerConnector` in `onStop()`, and does not close the `Executor` it creates. The latter leaves non-daemon threads and can prevent the JVM from shutting down even if streaming is closed properly. Author: Sean Owen <sowen@cloudera.com> Closes #980 from srowen/SPARK-2034 and squashes the following commits: 9f31a8d [Sean Owen] Restore ClassTag to private class because MIMA flags it; is the shadowing intended? 2d579a8 [Sean Owen] Close ConsumerConnector in onStop; shutdown() the local Executor that is created so that its threads stop when done; close the Zookeeper client even on exception; fix a few typos; log exceptions that otherwise vanish
* Package docsPrashant Sharma2014-05-142-0/+44
| | | | | | | | | | | | | | This is a few changes based on the original patch by @scrapcodes. Author: Prashant Sharma <prashant.s@imaginea.com> Author: Patrick Wendell <pwendell@gmail.com> Closes #785 from pwendell/package-docs and squashes the following commits: c32b731 [Patrick Wendell] Changes based on Prashant's patch c0463d3 [Prashant Sharma] added eof new line ce8bf73 [Prashant Sharma] Added eof new line to all files. 4c35f2e [Prashant Sharma] SPARK-1563 Add package-info.java and package.scala files for all packages that appear in docs
* SPARK-1798. Tests should clean up temp filesSean Owen2014-05-121-1/+1
| | | | | | | | | | | | | | | | | | | | Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent. Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former. The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules. Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method. _If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._ Author: Sean Owen <sowen@cloudera.com> Closes #732 from srowen/SPARK-1798 and squashes the following commits: 5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests
* [SPARK-1332] Improve Spark Streaming's Network Receiver and InputDStream API ↵Tathagata Das2014-04-214-26/+27
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | [WIP] The current Network Receiver API makes it slightly complicated to right a new receiver as one needs to create an instance of BlockGenerator as shown in SocketReceiver https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala#L51 Exposing the BlockGenerator interface has made it harder to improve the receiving process. The API of NetworkReceiver (which was not a very stable API anyways) needs to be change if we are to ensure future stability. Additionally, the functions like streamingContext.socketStream that create input streams, return DStream objects. That makes it hard to expose functionality (say, rate limits) unique to input dstreams. They should return InputDStream or NetworkInputDStream. This is still not yet implemented. This PR is blocked on the graceful shutdown PR #247 Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #300 from tdas/network-receiver-api and squashes the following commits: ea27b38 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3a4777c [Tathagata Das] Renamed NetworkInputDStream to ReceiverInputDStream, and ActorReceiver related stuff. 838dd39 [Tathagata Das] Added more events to the StreamingListener to report errors and stopped receivers. a75c7a6 [Tathagata Das] Address some PR comments and fixed other issues. 91bfa72 [Tathagata Das] Fixed bugs. 8533094 [Tathagata Das] Scala style fixes. 028bde6 [Tathagata Das] Further refactored receiver to allow restarting of a receiver. 43f5290 [Tathagata Das] Made functions that create input streams return InputDStream and NetworkInputDStream, for both Scala and Java. 2c94579 [Tathagata Das] Fixed graceful shutdown by removing interrupts on receiving thread. 9e37a0b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3223e95 [Tathagata Das] Refactored the code that runs the NetworkReceiver into further classes and traits to make them more testable. a36cc48 [Tathagata Das] Refactored the NetworkReceiver API for future stability.
* Merge pull request #557 from ScrapCodes/style. Closes #557.Patrick Wendell2014-02-092-6/+11
| | | | | | | | | | | | | | | | | | | | | SPARK-1058, Fix Style Errors and Add Scala Style to Spark Build. Author: Patrick Wendell <pwendell@gmail.com> Author: Prashant Sharma <scrapcodes@gmail.com> == Merge branch commits == commit 1a8bd1c059b842cb95cc246aaea74a79fec684f4 Author: Prashant Sharma <scrapcodes@gmail.com> Date: Sun Feb 9 17:39:07 2014 +0530 scala style fixes commit f91709887a8e0b608c5c2b282db19b8a44d53a43 Author: Patrick Wendell <pwendell@gmail.com> Date: Fri Jan 24 11:22:53 2014 -0800 Adding scalastyle snapshot
* Changed SparkConf to not be serializable. And also fixed unit-test log paths ↵Tathagata Das2014-01-142-1/+2
| | | | in log4j.properties of external modules.
* Removed StreamingContext.registerInputStream and registerOutputStream - they ↵Tathagata Das2014-01-131-3/+1
| | | | were useless as InputDStream has been made to register itself. Also made DStream.register() private[streaming] - not useful to expose the confusing function. Updated a lot of documentation.
* Fixed import formatting.Tathagata Das2014-01-121-1/+1
|
* Fixed persistence logic of WindowedDStream, and fixed default persistence ↵Tathagata Das2014-01-121-1/+2
| | | | level of input streams.
* Moved DStream, DStreamCheckpointData and PairDStream from ↵Tathagata Das2014-01-121-1/+2
| | | | org.apache.spark.streaming to org.apache.spark.streaming.dstream.
* Removed XYZFunctions and added XYZUtils as a common Scala and Java interface ↵Tathagata Das2014-01-076-215/+160
| | | | for creating XYZ streams.
* Changed JavaStreamingContextWith*** to ***Function in streaming.api.java.*** ↵Tathagata Das2014-01-062-14/+13
| | | | package. Also fixed packages of Flume and MQTT tests.
* Removed extra empty lines.Tathagata Das2013-12-311-1/+0
|
* Refactored kafka, flume, zeromq, mqtt as separate external projects, with ↵Tathagata Das2013-12-307-0/+477
their own self-contained scala API, java API, scala unit tests and java unit tests. Updated examples to use the external projects.