aboutsummaryrefslogtreecommitdiff
path: root/external/kafka
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-7052][Core] Add ThreadUtils and move thread methods from Utils to ↵zsxwing2015-04-222-4/+5
| | | | | | | | | | | | | | 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
* [Streaming][minor] Remove additional quote and unneeded importsjerryshao2015-04-161-2/+1
| | | | | | | | Author: jerryshao <saisai.shao@intel.com> Closes #5540 from jerryshao/minor-fix and squashes the following commits: ebaa646 [jerryshao] Minor fix
* [SPARK-6431][Streaming][Kafka] Error message for partition metadata requ...cody koeninger2015-04-122-3/+14
| | | | | | | | | | | | ...ests The original reported problem was misdiagnosed; the topic just didn't exist yet. Agreed upon solution was to improve error handling / message Author: cody koeninger <cody@koeninger.org> Closes #5454 from koeninger/spark-6431-master and squashes the following commits: 44300f8 [cody koeninger] [SPARK-6431][Streaming][Kafka] Error message for partition metadata requests
* [SPARK-6211][Streaming] Add Python Kafka API unit testjerryshao2015-04-099-308/+441
| | | | | | | | | | | | | | | | | | | | | | | | | Refactor the Kafka unit test and add Python API support. CC tdas davies please help to review, thanks a lot. Author: jerryshao <saisai.shao@intel.com> Author: Saisai Shao <saisai.shao@intel.com> Closes #4961 from jerryshao/SPARK-6211 and squashes the following commits: ee4b919 [jerryshao] Fixed newly merged issue 82c756e [jerryshao] Address the comments 92912d1 [jerryshao] Address the commits 0708bb1 [jerryshao] Fix rebase issue 40b47a3 [Saisai Shao] Style fix f889657 [Saisai Shao] Update the code according 8a2f3e2 [jerryshao] Address the issues 0f1b7ce [jerryshao] Still fix the bug 61a04f0 [jerryshao] Fix bugs and address the issues 64d9877 [jerryshao] Fix rebase bugs 8ad442f [jerryshao] Add kafka-assembly in run-tests 6020b00 [jerryshao] Add more debug info in Shell 8102d6e [jerryshao] Fix bug in Jenkins test fde1213 [jerryshao] Code style changes 5536f95 [jerryshao] Refactor the Kafka unit test and add Python Kafka unittest support
* [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-6569 [STREAMING] Down-grade same-offset message in Kafka streaming to INFOSean Owen2015-04-061-1/+1
| | | | | | | | | | Reduce "is the same as ending offset" message to INFO level per JIRA discussion Author: Sean Owen <sowen@cloudera.com> Closes #5366 from srowen/SPARK-6569 and squashes the following commits: 8a5b992 [Sean Owen] Reduce "is the same as ending offset" message to INFO level per JIRA discussion
* [SPARK-6428] Turn on explicit type checking for public methods.Reynold Xin2015-04-032-4/+5
| | | | | | | | | | | | | | 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-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-201-7/+3
| | | | | | | | | | | | | | | 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-6225 [CORE] [SQL] [STREAMING] Resolve most build warnings, 1.3.0 editionSean Owen2015-03-113-34/+22
| | | | | | | | | | | 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.
* [SPARK-6279][Streaming]In KafkaRDD.scala, Miss expressions flag "s" at ↵zzcclp2015-03-111-1/+1
| | | | | | | | | | | | | | | logging string In KafkaRDD.scala, Miss expressions flag "s" at logging string In logging file, it print `Beginning offset $ {part.fromOffset} is the same as ending offset ` but not `Beginning offset 111 is the same as ending offset `. Author: zzcclp <xm_zzc@sina.com> Closes #4979 from zzcclp/SPARK-6279 and squashes the following commits: 768f88e [zzcclp] Miss expressions flag "s"
* 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
* [Streaming][Minor] Remove useless type signature of Java Kafka direct stream APISaisai Shao2015-02-271-1/+1
| | | | | | | | | | cc tdas . Author: Saisai Shao <saisai.shao@intel.com> Closes #4817 from jerryshao/signature-minor-fix and squashes the following commits: eebfaac [Saisai Shao] Remove useless type parameter
* [SPARK-6027][SPARK-5546] Fixed --jar and --packages not working for ↵Tathagata Das2015-02-261-1/+28
| | | | | | | | | | | | | | | | | | KafkaUtils and improved error message The problem with SPARK-6027 in short is that JARs like the kafka-assembly.jar does not work in python as the added JAR is not visible in the classloader used by Py4J. Py4J uses Class.forName(), which does not uses the systemclassloader, but the JARs are only visible in the Thread's contextclassloader. So this back uses the context class loader to create the KafkaUtils dstream object. This works for both cases where the Kafka libraries are added with --jars spark-streaming-kafka-assembly.jar or with --packages spark-streaming-kafka Also improves the error message. davies Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #4779 from tdas/kafka-python-fix and squashes the following commits: fb16b04 [Tathagata Das] Removed import c1fdf35 [Tathagata Das] Fixed long line and improved documentation 7b88be8 [Tathagata Das] Fixed --jar not working for KafkaUtils and improved error message
* [SPARK-5731][Streaming][Test] Fix incorrect test in DirectKafkaStreamSuiteTathagata Das2015-02-171-12/+16
| | | | | | | | | | | | | | | The test was incorrect. Instead of counting the number of records, it counted the number of partitions of RDD generated by DStream. Which is not its intention. I will be testing this patch multiple times to understand its flakiness. PS: This was caused by my refactoring in https://github.com/apache/spark/pull/4384/ koeninger check it out. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #4597 from tdas/kafka-flaky-test and squashes the following commits: d236235 [Tathagata Das] Unignored last test. e9a1820 [Tathagata Das] fix test
* [HOTFIX] Ignore DirectKafkaStreamSuite.Reynold Xin2015-02-131-4/+4
|
* [SPARK-4964] [Streaming] refactor createRDD to take leaders via map instead ↵cody koeninger2015-02-114-66/+287
| | | | | | | | | | | | | | of array Author: cody koeninger <cody@koeninger.org> Closes #4511 from koeninger/kafkaRdd-leader-to-broker and squashes the following commits: f7151d4 [cody koeninger] [SPARK-4964] test refactoring 6f8680b [cody koeninger] [SPARK-4964] add test of the scala api for KafkaUtils.createRDD f81e016 [cody koeninger] [SPARK-4964] leave KafkaStreamSuite host and port as private 5173f3f [cody koeninger] [SPARK-4964] test the Java variations of createRDD e9cece4 [cody koeninger] [SPARK-4964] pass leaders as a map to ensure 1 leader per TopicPartition
* [SPARK-4964][Streaming][Kafka] More updates to Exactly-once Kafka streamTathagata Das2015-02-0915-262/+864
| | | | | | | | | | | | | | | | | | | | | | | Changes - Added example - Added a critical unit test that verifies that offset ranges can be recovered through checkpoints Might add more changes. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #4384 from tdas/new-kafka-fixes and squashes the following commits: 7c931c3 [Tathagata Das] Small update 3ed9284 [Tathagata Das] updated scala doc 83d0402 [Tathagata Das] Added JavaDirectKafkaWordCount example. 26df23c [Tathagata Das] Updates based on PR comments from Cody e4abf69 [Tathagata Das] Scala doc improvements and stuff. bb65232 [Tathagata Das] Fixed test bug and refactored KafkaStreamSuite 50f2b56 [Tathagata Das] Added Java API and added more Scala and Java unit tests. Also updated docs. e73589c [Tathagata Das] Minor changes. 4986784 [Tathagata Das] Added unit test to kafka offset recovery 6a91cab [Tathagata Das] Added example
* [SPARK-4707][STREAMING] Reliable Kafka Receiver can lose data if the blo...Hari Shreedharan2015-02-041-4/+23
| | | | | | | | | | | | | | ...ck generator fails to store data. The Reliable Kafka Receiver commits offsets only when events are actually stored, which ensures that on restart we will actually start where we left off. But if the failure happens in the store() call, and the block generator reports an error the receiver does not do anything and will continue reading from the current offset and not the last commit. This means that messages between the last commit and the current offset will be lost. This PR retries the store call four times and then stops the receiver with an error message and the last exception that was received from the store. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #3655 from harishreedharan/kafka-failure-fix and squashes the following commits: 5e2e7ad [Hari Shreedharan] [SPARK-4704][STREAMING] Reliable Kafka Receiver can lose data if the block generator fails to store data.
* [SPARK-4964] [Streaming] Exactly-once semantics for Kafkacody koeninger2015-02-0412-6/+1376
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Author: cody koeninger <cody@koeninger.org> Closes #3798 from koeninger/kafkaRdd and squashes the following commits: 1dc2941 [cody koeninger] [SPARK-4964] silence ConsumerConfig warnings about broker connection props 59e29f6 [cody koeninger] [SPARK-4964] settle on "Direct" as a naming convention for the new stream 8c31855 [cody koeninger] [SPARK-4964] remove HasOffsetRanges interface from return types 0df3ebe [cody koeninger] [SPARK-4964] add comments per pwendell / dibbhatt 8991017 [cody koeninger] [SPARK-4964] formatting 825110f [cody koeninger] [SPARK-4964] rename stuff per TD 4354bce [cody koeninger] [SPARK-4964] per td, remove java interfaces, replace with final classes, corresponding changes to KafkaRDD constructor and checkpointing 9adaa0a [cody koeninger] [SPARK-4964] formatting 0090553 [cody koeninger] [SPARK-4964] javafication of interfaces 9a838c2 [cody koeninger] [SPARK-4964] code cleanup, add more tests 2b340d8 [cody koeninger] [SPARK-4964] refactor per TD feedback 80fd6ae [cody koeninger] [SPARK-4964] Rename createExactlyOnceStream so it isnt over-promising, change doc 99d2eba [cody koeninger] [SPARK-4964] Reduce level of nesting. If beginning is past end, its actually an error (may happen if Kafka topic was deleted and recreated) 19406cc [cody koeninger] Merge branch 'master' of https://github.com/apache/spark into kafkaRdd 2e67117 [cody koeninger] [SPARK-4964] one potential way of hiding most of the implementation, while still allowing access to offsets (but not subclassing) bb80bbe [cody koeninger] [SPARK-4964] scalastyle line length d4a7cf7 [cody koeninger] [SPARK-4964] allow for use cases that need to override compute for custom kafka dstreams c1bd6d9 [cody koeninger] [SPARK-4964] use newly available attemptNumber for correct retry behavior 548d529 [cody koeninger] Merge branch 'master' of https://github.com/apache/spark into kafkaRdd 0458e4e [cody koeninger] [SPARK-4964] recovery of generated rdds from checkpoint e86317b [cody koeninger] [SPARK-4964] try seed brokers in random order to spread metadata requests e93eb72 [cody koeninger] [SPARK-4964] refactor to add preferredLocations. depends on SPARK-4014 356c7cc [cody koeninger] [SPARK-4964] code cleanup per helena adf99a6 [cody koeninger] [SPARK-4964] fix serialization issues for checkpointing 1d50749 [cody koeninger] [SPARK-4964] code cleanup per tdas 8bfd6c0 [cody koeninger] [SPARK-4964] configure rate limiting via spark.streaming.receiver.maxRate e09045b [cody koeninger] [SPARK-4964] add foreachPartitionWithIndex, to avoid doing equivalent map + empty foreach boilerplate cac63ee [cody koeninger] additional testing, fix fencepost error 37d3053 [cody koeninger] make KafkaRDDPartition available to users so offsets can be committed per partition bcca8a4 [cody koeninger] Merge branch 'master' of https://github.com/apache/spark into kafkaRdd 6bf14f2 [cody koeninger] first attempt at a Kafka dstream that allows for exactly-once semantics 326ff3c [cody koeninger] add some tests 38bb727 [cody koeninger] give easy access to the parameters of a KafkaRDD 979da25 [cody koeninger] dont allow empty leader offsets to be returned 8d7de4a [cody koeninger] make sure leader offsets can be found even for leaders that arent in the seed brokers 4b078bf [cody koeninger] differentiate between leader and consumer offsets in error message 3c2a96a [cody koeninger] fix scalastyle errors 29c6b43 [cody koeninger] cleanup logging 783b477 [cody koeninger] update tests for kafka 8.1.1 7d050bc [cody koeninger] methods to set consumer offsets and get topic metadata, switch back to inclusive start / exclusive end to match typical kafka consumer behavior ce91c59 [cody koeninger] method to get consumer offsets, explicit error handling 4dafd1b [cody koeninger] method to get leader offsets, switch rdd bound to being exclusive start, inclusive end to match offsets typically returned from cluster 0b94b33 [cody koeninger] use dropWhile rather than filter to trim beginning of fetch response 1d70625 [cody koeninger] WIP on kafka cluster 76913e2 [cody koeninger] Batch oriented kafka rdd, WIP. todo: cluster metadata / finding leader
* [SPARK-5153][Streaming][Test] Increased timeout to deal with flaky ↵Tathagata Das2015-02-031-1/+1
| | | | | | | | | | | | KafkaStreamSuite Timeout increased to allow overloaded Jenkins to cope with delay in topic creation. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #4342 from tdas/SPARK-5153 and squashes the following commits: dc42762 [Tathagata Das] Increased timeout to deal with delays in overloaded Jenkins.
* SPARK-4159 [CORE] Maven build doesn't run JUnit test suitesSean Owen2015-01-062-14/+2
| | | | | | | | | | | | | | | | | | This PR: - Reenables `surefire`, and copies config from `scalatest` (which is itself an old fork of `surefire`, so similar) - Tells `surefire` to test only Java tests - Enables `surefire` and `scalatest` for all children, and in turn eliminates some duplication. For me this causes the Scala and Java tests to be run once each, it seems, as desired. It doesn't affect the SBT build but works for Maven. I still need to verify that all of the Scala tests and Java tests are being run. Author: Sean Owen <sowen@cloudera.com> Closes #3651 from srowen/SPARK-4159 and squashes the following commits: 2e8a0af [Sean Owen] Remove specialized SPARK_HOME setting for REPL, YARN tests as it appears to be obsolete 12e4558 [Sean Owen] Append to unit-test.log instead of overwriting, so that both surefire and scalatest output is preserved. Also standardize/correct comments a bit. e6f8601 [Sean Owen] Reenable Java tests by reenabling surefire with config cloned from scalatest; centralize test config in the parent
* fixed spelling errors in documentationPeter Klipfel2014-12-141-3/+3
| | | | | | | | | | changed "form" to "from" in 3 documentation entries for Kafka integration Author: Peter Klipfel <peter@klipfel.me> Closes #3691 from peterklipfel/master and squashes the following commits: 0fe7fc5 [Peter Klipfel] fixed spelling errors in documentation
* SPARK-3962 Marked scope as provided for external projects.Prashant Sharma2014-11-191-7/+1
| | | | | | | | | | | | | Somehow maven shade plugin is set in infinite loop of creating effective pom. Author: Prashant Sharma <prashant.s@imaginea.com> Author: Prashant Sharma <scrapcodes@gmail.com> Closes #2959 from ScrapCodes/SPARK-3962/scope-provided and squashes the following commits: 994d1d3 [Prashant Sharma] Fixed failing flume tests 270b4fb [Prashant Sharma] Removed most of the unused code. bb3bbfd [Prashant Sharma] SPARK-3962 Marked scope as provided for external.
* Bumping version to 1.3.0-SNAPSHOT.Marcelo Vanzin2014-11-181-1/+1
| | | | | | | | | | | | Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #3277 from vanzin/version-1.3 and squashes the following commits: 7c3c396 [Marcelo Vanzin] Added temp repo to sbt build. 5f404ff [Marcelo Vanzin] Add another exclusion. 19457e7 [Marcelo Vanzin] Update old version to 1.2, add temporary 1.2 repo. 3c8d705 [Marcelo Vanzin] Workaround for MIMA checks. e940810 [Marcelo Vanzin] Bumping version to 1.3.0-SNAPSHOT.
* [SPARK-4062][Streaming]Add ReliableKafkaReceiver in Spark Streaming Kafka ↵jerryshao2014-11-146-132/+587
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | connector Add ReliableKafkaReceiver in Kafka connector to prevent data loss if WAL in Spark Streaming is enabled. Details and design doc can be seen in [SPARK-4062](https://issues.apache.org/jira/browse/SPARK-4062). Author: jerryshao <saisai.shao@intel.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Saisai Shao <saisai.shao@intel.com> Closes #2991 from jerryshao/kafka-refactor and squashes the following commits: 5461f1c [Saisai Shao] Merge pull request #8 from tdas/kafka-refactor3 eae4ad6 [Tathagata Das] Refectored KafkaStreamSuiteBased to eliminate KafkaTestUtils and made Java more robust. fab14c7 [Tathagata Das] minor update. 149948b [Tathagata Das] Fixed mistake 14630aa [Tathagata Das] Minor updates. d9a452c [Tathagata Das] Minor updates. ec2e95e [Tathagata Das] Removed the receiver's locks and essentially reverted to Saisai's original design. 2a20a01 [jerryshao] Address some comments 9f636b3 [Saisai Shao] Merge pull request #5 from tdas/kafka-refactor b2b2f84 [Tathagata Das] Refactored Kafka receiver logic and Kafka testsuites e501b3c [jerryshao] Add Mima excludes b798535 [jerryshao] Fix the missed issue e5e21c1 [jerryshao] Change to while loop ea873e4 [jerryshao] Further address the comments 98f3d07 [jerryshao] Fix comment style 4854ee9 [jerryshao] Address all the comments 96c7a1d [jerryshao] Update the ReliableKafkaReceiver unit test 8135d31 [jerryshao] Fix flaky test a949741 [jerryshao] Address the comments 16bfe78 [jerryshao] Change the ordering of imports 0894aef [jerryshao] Add some comments 77c3e50 [jerryshao] Code refactor and add some unit tests dd9aeeb [jerryshao] Initial commit for reliable Kafka receiver
* [SPARK-2492][Streaming] kafkaReceiver minor changes to align with Kafka 0.8jerryshao2014-11-112-36/+5
| | | | | | | | | | | | | | | | | | | | | | Update the KafkaReceiver's behavior when auto.offset.reset is set. In Kafka 0.8, `auto.offset.reset` is a hint for out-range offset to seek to the beginning or end of the partition. While in the previous code `auto.offset.reset` is a enforcement to seek to the beginning or end immediately, this is different from Kafka 0.8 defined behavior. Also deleting extesting ZK metadata in Receiver when multiple consumers are launched will introduce issue as mentioned in [SPARK-2383](https://issues.apache.org/jira/browse/SPARK-2383). So Here we change to offer user to API to explicitly reset offset before create Kafka stream, while in the meantime keep the same behavior as Kafka 0.8 for parameter `auto.offset.reset`. @tdas, would you please review this PR? Thanks a lot. Author: jerryshao <saisai.shao@intel.com> Closes #1420 from jerryshao/kafka-fix and squashes the following commits: d6ae94d [jerryshao] Address the comment to remove the resetOffset() function de3a4c8 [jerryshao] Fix compile error 4a1c3f9 [jerryshao] Doc changes b2c1430 [jerryshao] Move offset reset to a helper function to let user explicitly delete ZK metadata by calling this API fac8fd6 [jerryshao] Changes to align with Kafka 0.8
* [SPARK-3748] Log thread name in unit test logsReynold Xin2014-10-011-1/+1
| | | | | | | | | | Thread names are useful for correlating failures. Author: Reynold Xin <rxin@apache.org> Closes #2600 from rxin/log4j and squashes the following commits: 83ffe88 [Reynold Xin] [SPARK-3748] Log thread name in unit test logs
* [SPARK-3615][Streaming]Fix Kafka unit test hard coded Zookeeper port issuejerryshao2014-09-242-14/+34
| | | | | | | | | | Details can be seen in [SPARK-3615](https://issues.apache.org/jira/browse/SPARK-3615). Author: jerryshao <saisai.shao@intel.com> Closes #2483 from jerryshao/SPARK_3615 and squashes the following commits: 8555563 [jerryshao] Fix Kafka unit test hard coded Zookeeper port issue
* [SPARK-3397] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOTGuoQiang Li2014-09-061-1/+1
| | | | | | | | Author: GuoQiang Li <witgo@qq.com> Closes #2268 from witgo/SPARK-3397 and squashes the following commits: eaf913f [GuoQiang Li] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT
* [SPARK-1022][Streaming][HOTFIX] Fixed zookeeper dependency of KafkaTathagata Das2014-08-051-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | https://github.com/apache/spark/pull/1751 caused maven builds to fail. ``` ~/Apache/spark(branch-1.1|✔) ➤ mvn -U -DskipTests clean install . . . [error] Apache/spark/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala:36: object NIOServerCnxnFactory is not a member of package org.apache.zookeeper.server [error] import org.apache.zookeeper.server.NIOServerCnxnFactory [error] ^ [error] Apache/spark/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala:199: not found: type NIOServerCnxnFactory [error] val factory = new NIOServerCnxnFactory() [error] ^ [error] two errors found [error] Compile failed at Aug 5, 2014 1:42:36 PM [0.503s] ``` The problem is how SBT and Maven resolves multiple versions of the same library, which in this case, is Zookeeper. Observing and comparing the dependency trees from Maven and SBT showed this. Spark depends on ZK 3.4.5 whereas Apache Kafka transitively depends on upon ZK 3.3.4. SBT decides to evict 3.3.4 and use the higher version 3.4.5. But Maven decides to stick to the closest (in the tree) dependent version of 3.3.4. And 3.3.4 does not have NIOServerCnxnFactory. The solution in this patch excludes zookeeper from the apache-kafka dependency in streaming-kafka module so that it just inherits zookeeper from Spark core. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #1797 from tdas/kafka-zk-fix and squashes the following commits: 94b3931 [Tathagata Das] Fixed zookeeper dependency of Kafka
* [SPARK-1022][Streaming] Add Kafka real unit testjerryshao2014-08-053-35/+293
| | | | | | | | | | | | | | | | | 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-2749 [BUILD]. Spark SQL Java tests aren't compiling in Jenkins' Maven ↵Sean Owen2014-07-301-0/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | builds; missing junit:junit dep The Maven-based builds in the build matrix have been failing for a few days: https://amplab.cs.berkeley.edu/jenkins/view/Spark/ On inspection, it looks like the Spark SQL Java tests don't compile: https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/hadoop.version=1.0.4,label=centos/244/consoleFull I confirmed it by repeating the command vs master: `mvn -Dhadoop.version=1.0.4 -Dlabel=centos -DskipTests clean package` The problem is that this module doesn't depend on JUnit. In fact, none of the modules do, but `com.novocode:junit-interface` (the SBT-JUnit bridge) pulls it in, in most places. However this module doesn't depend on `com.novocode:junit-interface` Adding the `junit:junit` dependency fixes the compile problem. In fact, the other modules with Java tests should probably depend on it explicitly instead of happening to get it via `com.novocode:junit-interface`, since that is a bit SBT/Scala-specific (and I am not even sure it's needed). Author: Sean Owen <srowen@gmail.com> Closes #1660 from srowen/SPARK-2749 and squashes the following commits: 858ff7c [Sean Owen] Add explicit junit dep to other modules with Java tests for robustness 9636794 [Sean Owen] Add junit dep so that Spark SQL Java tests compile
* [SPARK-2410][SQL] Merging Hive Thrift/JDBC server (with Maven profile fix)Cheng Lian2014-07-281-1/+1
| | | | | | | | | | | | | | | JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) Another try for #1399 & #1600. Those two PR breaks Jenkins builds because we made a separate profile `hive-thriftserver` in sub-project `assembly`, but the `hive-thriftserver` module is defined outside the `hive-thriftserver` profile. Thus every time a pull request that doesn't touch SQL code will also execute test suites defined in `hive-thriftserver`, but tests fail because related .class files are not included in the assembly jar. In the most recent commit, module `hive-thriftserver` is moved into its own profile to fix this problem. All previous commits are squashed for clarity. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1620 from liancheng/jdbc-with-maven-fix and squashes the following commits: 629988e [Cheng Lian] Moved hive-thriftserver module definition into its own profile ec3c7a7 [Cheng Lian] Cherry picked the Hive Thrift server
* Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"Patrick Wendell2014-07-271-1/+1
| | | | This reverts commit f6ff2a61d00d12481bfb211ae13d6992daacdcc2.
* [SPARK-2410][SQL] Merging Hive Thrift/JDBC serverCheng Lian2014-07-271-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | (This is a replacement of #1399, trying to fix potential `HiveThriftServer2` port collision between parallel builds. Please refer to [these comments](https://github.com/apache/spark/pull/1399#issuecomment-50212572) for details.) JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) Merging the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc). Thanks chenghao-intel for his initial contribution of the Spark SQL CLI. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1600 from liancheng/jdbc and squashes the following commits: ac4618b [Cheng Lian] Uses random port for HiveThriftServer2 to avoid collision with parallel builds 090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR 21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd] 199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver 1083e9d [Cheng Lian] Fixed failed test suites 7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic 9cc0f06 [Cheng Lian] Starts beeline with spark-submit cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile 061880f [Cheng Lian] Addressed all comments by @pwendell 7755062 [Cheng Lian] Adapts test suites to spark-submit settings 40bafef [Cheng Lian] Fixed more license header issues e214aab [Cheng Lian] Added missing license headers b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft 3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit 61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit 2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server
* Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"Michael Armbrust2014-07-251-1/+1
| | | | | | | | | | | | This reverts commit 06dc0d2c6b69c5d59b4d194ced2ac85bfe2e05e2. #1399 is making Jenkins fail. We should investigate and put this back after its passing tests. Author: Michael Armbrust <michael@databricks.com> Closes #1594 from marmbrus/revertJDBC and squashes the following commits: 59748da [Michael Armbrust] Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"
* [SPARK-2410][SQL] Merging Hive Thrift/JDBC serverCheng Lian2014-07-251-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | JIRA issue: - Main: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) - Related: [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678) Cherry picked the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc). (Thanks chenghao-intel for his initial contribution of the Spark SQL CLI.) TODO - [x] Use `spark-submit` to launch the server, the CLI and beeline - [x] Migration guideline draft for Shark users ---- Hit by a bug in `SparkSubmitArguments` while working on this PR: all application options that are recognized by `SparkSubmitArguments` are stolen as `SparkSubmit` options. For example: ```bash $ spark-submit --class org.apache.hive.beeline.BeeLine spark-internal --help ``` This actually shows usage information of `SparkSubmit` rather than `BeeLine`. ~~Fixed this bug here since the `spark-internal` related stuff also touches `SparkSubmitArguments` and I'd like to avoid conflict.~~ **UPDATE** The bug mentioned above is now tracked by [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678). Decided to revert changes to this bug since it involves more subtle considerations and worth a separate PR. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1399 from liancheng/thriftserver and squashes the following commits: 090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR 21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd] 199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver 1083e9d [Cheng Lian] Fixed failed test suites 7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic 9cc0f06 [Cheng Lian] Starts beeline with spark-submit cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile 061880f [Cheng Lian] Addressed all comments by @pwendell 7755062 [Cheng Lian] Adapts test suites to spark-submit settings 40bafef [Cheng Lian] Fixed more license header issues e214aab [Cheng Lian] Added missing license headers b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft 3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit 61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit 2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server
* [SPARK-1776] Have Spark's SBT build read dependencies from Maven.Prashant Sharma2014-07-101-0/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Patch introduces the new way of working also retaining the existing ways of doing things. For example build instruction for yarn in maven is `mvn -Pyarn -PHadoop2.2 clean package -DskipTests` in sbt it can become `MAVEN_PROFILES="yarn, hadoop-2.2" sbt/sbt clean assembly` Also supports `sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 clean assembly` Author: Prashant Sharma <prashant.s@imaginea.com> Author: Patrick Wendell <pwendell@gmail.com> Closes #772 from ScrapCodes/sbt-maven and squashes the following commits: a8ac951 [Prashant Sharma] Updated sbt version. 62b09bb [Prashant Sharma] Improvements. fa6221d [Prashant Sharma] Excluding sql from mima 4b8875e [Prashant Sharma] Sbt assembly no longer builds tools by default. 72651ca [Prashant Sharma] Addresses code reivew comments. acab73d [Prashant Sharma] Revert "Small fix to run-examples script." ac4312c [Prashant Sharma] Revert "minor fix" 6af91ac [Prashant Sharma] Ported oldDeps back. + fixes issues with prev commit. 65cf06c [Prashant Sharma] Servelet API jars mess up with the other servlet jars on the class path. 446768e [Prashant Sharma] minor fix 89b9777 [Prashant Sharma] Merge conflicts d0a02f2 [Prashant Sharma] Bumped up pom versions, Since the build now depends on pom it is better updated there. + general cleanups. dccc8ac [Prashant Sharma] updated mima to check against 1.0 a49c61b [Prashant Sharma] Fix for tools jar a2f5ae1 [Prashant Sharma] Fixes a bug in dependencies. cf88758 [Prashant Sharma] cleanup 9439ea3 [Prashant Sharma] Small fix to run-examples script. 96cea1f [Prashant Sharma] SPARK-1776 Have Spark's SBT build read dependencies from Maven. 36efa62 [Patrick Wendell] Set project name in pom files and added eclipse/intellij plugins. 4973dbd [Patrick Wendell] Example build using pom reader.
* 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
* [SPARK-2029] Bump pom.xml version number of master branch to 1.1.0-SNAPSHOT.Takuya UESHIN2014-06-051-1/+1
| | | | | | | | Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #974 from ueshin/issues/SPARK-2029 and squashes the following commits: e19e8f4 [Takuya UESHIN] Bump version number to 1.1.0-SNAPSHOT.
* 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
* Improved build configurationwitgo2014-04-281-14/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | 1, Fix SPARK-1441: compile spark core error with hadoop 0.23.x 2, Fix SPARK-1491: maven hadoop-provided profile fails to build 3, Fix org.scala-lang: * ,org.apache.avro:* inconsistent versions dependency 4, A modified on the sql/catalyst/pom.xml,sql/hive/pom.xml,sql/core/pom.xml (Four spaces formatted into two spaces) Author: witgo <witgo@qq.com> Closes #480 from witgo/format_pom and squashes the following commits: 03f652f [witgo] review commit b452680 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom bee920d [witgo] revert fix SPARK-1629: Spark Core missing commons-lang dependence 7382a07 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 6902c91 [witgo] fix SPARK-1629: Spark Core missing commons-lang dependence 0da4bc3 [witgo] merge master d1718ed [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom e345919 [witgo] add avro dependency to yarn-alpha 77fad08 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 62d0862 [witgo] Fix org.scala-lang: * inconsistent versions dependency 1a162d7 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 934f24d [witgo] review commit cf46edc [witgo] exclude jruby 06e7328 [witgo] Merge branch 'SparkBuild' into format_pom 99464d2 [witgo] fix maven hadoop-provided profile fails to build 0c6c1fc [witgo] Fix compile spark core error with hadoop 0.23.x 6851bec [witgo] Maintain consistent SparkBuild.scala, pom.xml
* [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.
* SPARK-1193. Fix indentation in pom.xmlsSandy Ryza2014-03-071-4/+4
| | | | | | | | Author: Sandy Ryza <sandy@cloudera.com> Closes #91 from sryza/sandy-spark-1193 and squashes the following commits: a878124 [Sandy Ryza] SPARK-1193. Fix indentation in pom.xmls
* SPARK-1121: Include avro for yarn-alpha buildsPatrick Wendell2014-03-021-0/+14
| | | | | | | | | | | | | | | | | This lets us explicitly include Avro based on a profile for 0.23.X builds. It makes me sad how convoluted it is to express this logic in Maven. @tgraves and @sryza curious if this works for you. I'm also considering just reverting to how it was before. The only real problem was that Spark advertised a dependency on Avro even though it only really depends transitively on Avro through other deps. Author: Patrick Wendell <pwendell@gmail.com> Closes #49 from pwendell/avro-build-fix and squashes the following commits: 8d6ee92 [Patrick Wendell] SPARK-1121: Add avro to yarn-alpha profile
* Remove remaining references to incubationPatrick Wendell2014-03-021-2/+2
| | | | | | | | | | This removes some loose ends not caught by the other (incubating -> tlp) patches. @markhamstra this updates the version as you mentioned earlier. Author: Patrick Wendell <pwendell@gmail.com> Closes #51 from pwendell/tlp and squashes the following commits: d553b1b [Patrick Wendell] Remove remaining references to incubation
* SPARK-1071: Tidy logging strategy and use of log4jSean Owen2014-02-231-5/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Prompted by a recent thread on the mailing list, I tried and failed to see if Spark can be made independent of log4j. There are a few cases where control of the underlying logging is pretty useful, and to do that, you have to bind to a specific logger. Instead I propose some tidying that leaves Spark's use of log4j, but gets rid of warnings and should still enable downstream users to switch. The idea is to pipe everything (except log4j) through SLF4J, and have Spark use SLF4J directly when logging, and where Spark needs to output info (REPL and tests), bind from SLF4J to log4j. This leaves the same behavior in Spark. It means that downstream users who want to use something except log4j should: - Exclude dependencies on log4j, slf4j-log4j12 from Spark - Include dependency on log4j-over-slf4j - Include dependency on another logger X, and another slf4j-X - Recreate any log config that Spark does, that is needed, in the other logger's config That sounds about right. Here are the key changes: - Include the jcl-over-slf4j shim everywhere by depending on it in core. - Exclude dependencies on commons-logging from third-party libraries. - Include the jul-to-slf4j shim everywhere by depending on it in core. - Exclude slf4j-* dependencies from third-party libraries to prevent collision or warnings - Added missing slf4j-log4j12 binding to GraphX, Bagel module tests And minor/incidental changes: - Update to SLF4J 1.7.5, which happily matches Hadoop 2’s version and is a recommended update over 1.7.2 - (Remove a duplicate HBase dependency declaration in SparkBuild.scala) - (Remove a duplicate mockito dependency declaration that was causing warnings and bugging me) Author: Sean Owen <sowen@cloudera.com> Closes #570 from srowen/SPARK-1071 and squashes the following commits: 52eac9f [Sean Owen] Add slf4j-over-log4j12 dependency to core (non-test) and remove it from things that depend on core. 77a7fa9 [Sean Owen] SPARK-1071: Tidy logging strategy and use of log4j