aboutsummaryrefslogtreecommitdiff
path: root/external
Commit message (Collapse)AuthorAgeFilesLines
* [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-5666][streaming][MQTT streaming] some trivial fixesprabs2015-02-253-20/+21
| | | | | | | | | | | | | | | | | | | | modified to adhere to accepted coding standards as pointed by tdas in PR #3844 Author: prabs <prabsmails@gmail.com> Author: Prabeesh K <prabsmails@gmail.com> Closes #4178 from prabeesh/master and squashes the following commits: bd2cb49 [Prabeesh K] adress the comment ccc0765 [prabs] adress the comment 46f9619 [prabs] adress the comment c035bdc [prabs] adress the comment 22dd7f7 [prabs] address the comments 0cc67bd [prabs] adress the comment 838c38e [prabs] adress the comment cd57029 [prabs] address the comments 66919a3 [Prabeesh K] changed MqttDefaultFilePersistence to MemoryPersistence 5857989 [prabs] modified to adhere to accepted coding standards
* [SPARK-5993][Streaming][Build] Fix assembly jar location of kafka-assemblyTathagata Das2015-02-241-4/+0
| | | | | | | | | | | | | | | Published Kafka-assembly JAR was empty in 1.3.0-RC1 This is because the maven build generated two Jars- 1. an empty JAR file (since kafka-assembly has no code of its own) 2. a assembly JAR file containing everything in a different location as 1 The maven publishing plugin uploaded 1 and not 2. Instead if 2 is not configure to generate in a different location, there is only 1 jar containing everything, which gets published. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #4753 from tdas/SPARK-5993 and squashes the following commits: c390db8 [Tathagata Das] Fix assembly jar location of kafka-assembly
* SPARK-4682 [CORE] Consolidate various 'Clock' classesSean Owen2015-02-195-8/+7
| | | | | | | | | | | | | | | 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-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
* SPARK-5850: Remove experimental label for Scala 2.11 and FlumePollingStreamPatrick Wendell2015-02-161-8/+0
| | | | | | | | Author: Patrick Wendell <patrick@databricks.com> Closes #4638 from pwendell/SPARK-5850 and squashes the following commits: 386126f [Patrick Wendell] SPARK-5850: Remove experimental label for Scala 2.11 and FlumePollingStream.
* [HOTFIX] Ignore DirectKafkaStreamSuite.Reynold Xin2015-02-131-4/+4
|
* SPARK-5728 [STREAMING] MQTTStreamSuite leaves behind ActiveMQ database filesSean Owen2015-02-111-0/+1
| | | | | | | | | | Use temp dir for ActiveMQ database Author: Sean Owen <sowen@cloudera.com> Closes #4517 from srowen/SPARK-5728 and squashes the following commits: 1d3aeb8 [Sean Owen] Use temp dir for ActiveMQ database
* [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-4905][STREAMING] FlumeStreamSuite fix.Hari Shreedharan2015-02-091-4/+3
| | | | | | | | | | | | Using String constructor instead of CharsetDecoder to see if it fixes the issue of empty strings in Flume test output. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #4371 from harishreedharan/Flume-stream-attempted-fix and squashes the following commits: 550d363 [Hari Shreedharan] Fix imports. 8695950 [Hari Shreedharan] Use Charsets.UTF_8 instead of "UTF-8" in String constructors. af3ba14 [Hari Shreedharan] [SPARK-4905][STREAMING] FlumeStreamSuite fix.
* [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-5154] [PySpark] [Streaming] Kafka streaming support in PythonDavies Liu2015-02-021-0/+106
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR brings the Python API for Spark Streaming Kafka data source. ``` class KafkaUtils(__builtin__.object) | Static methods defined here: | | createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False, 2), keyDecoder=<function utf8_decoder>, valueDecoder=<function utf8_decoder>) | Create an input stream that pulls messages from a Kafka Broker. | | :param ssc: StreamingContext object | :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..). | :param groupId: The group id for this consumer. | :param topics: Dict of (topic_name -> numPartitions) to consume. | Each partition is consumed in its own thread. | :param storageLevel: RDD storage level. | :param keyDecoder: A function used to decode key | :param valueDecoder: A function used to decode value | :return: A DStream object ``` run the example: ``` bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test ``` Author: Davies Liu <davies@databricks.com> Author: Tathagata Das <tdas@databricks.com> Closes #3715 from davies/kafka and squashes the following commits: d93bfe0 [Davies Liu] Update make-distribution.sh 4280d04 [Davies Liu] address comments e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka f257071 [Davies Liu] add tests for null in RDD 23b039a [Davies Liu] address comments 9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka a74da87 [Davies Liu] address comments dc1eed0 [Davies Liu] Update kafka_wordcount.py 31e2317 [Davies Liu] Update kafka_wordcount.py 370ba61 [Davies Liu] Update kafka.py 97386b3 [Davies Liu] address comment 2c567a5 [Davies Liu] update logging and comment 33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api aea8953 [Tathagata Das] Kafka-assembly for Python API eea16a7 [Davies Liu] refactor f6ce899 [Davies Liu] add example and fix bugs 98c8d17 [Davies Liu] fix python style 5697a01 [Davies Liu] bypass decoder in scala 048dbe6 [Davies Liu] fix python style 75d485e [Davies Liu] add mqtt 07923c4 [Davies Liu] support kafka in Python
* [SPARK-4631][streaming][FIX] Wait for a receiver to start before publishing ↵Iulian Dragos2015-02-022-15/+46
| | | | | | | | | | | | | | | | | | | | test data. This fixes two sources of non-deterministic failures in this test: - wait for a receiver to be up before pushing data through MQTT - gracefully handle the case where the MQTT client is overloaded. There’s a hard-coded limit of 10 in-flight messages, and this test may hit it. Instead of crashing, we retry sending the message. Both of these are needed to make the test pass reliably on my machine. Author: Iulian Dragos <jaguarul@gmail.com> Closes #4270 from dragos/issue/fix-flaky-test-SPARK-4631 and squashes the following commits: f66c482 [Iulian Dragos] [SPARK-4631][streaming] Wait for a receiver to start before publishing test data. d408a8e [Iulian Dragos] Install callback before connecting to MQTT broker.
* [SPARK-5006][Deploy]spark.port.maxRetries doesn't workWangTaoTheTonic2015-01-132-2/+3
| | | | | | | | | | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-5006 I think the issue is produced in https://github.com/apache/spark/pull/1777. Not digging mesos's backend yet. Maybe should add same logic either. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits: 8cdf96d [WangTao] indent thing 2d86d65 [WangTaoTheTonic] fix line length 7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor 61a370d [WangTaoTheTonic] some minor fixes bc6e1ec [WangTaoTheTonic] rebase 67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg 29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable 396c226 [WangTaoTheTonic] make the grammar more like scala 191face [WangTaoTheTonic] invalid value name 62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work
* [Minor]Resolve sbt warnings during build (MQTTStreamSuite.scala).GuoQiang Li2015-01-101-0/+1
| | | | | | | | | | cc andrewor14 Author: GuoQiang Li <witgo@qq.com> Closes #3989 from witgo/MQTTStreamSuite and squashes the following commits: a6e967e [GuoQiang Li] Resolve sbt warnings during build (MQTTStreamSuite.scala).
* [Minor] Fix import order and other coding stylebilna2015-01-091-6/+11
| | | | | | | | | | | | | | | | | | | | | | | | | | | fixed import order and other coding style Author: bilna <bilnap@am.amrita.edu> Author: Bilna P <bilna.p@gmail.com> Closes #3966 from Bilna/master and squashes the following commits: 5e76f04 [bilna] fix import order and other coding style 5718d66 [bilna] Merge remote-tracking branch 'upstream/master' ae56514 [bilna] Merge remote-tracking branch 'upstream/master' acea3a3 [bilna] Adding dependency with scope test 28681fa [bilna] Merge remote-tracking branch 'upstream/master' fac3904 [bilna] Correction in Indentation and coding style ed9db4c [bilna] Merge remote-tracking branch 'upstream/master' 4b34ee7 [Bilna P] Update MQTTStreamSuite.scala 04503cf [bilna] Added embedded broker service for mqtt test 89d804e [bilna] Merge remote-tracking branch 'upstream/master' fc8eb28 [bilna] Merge remote-tracking branch 'upstream/master' 4b58094 [Bilna P] Update MQTTStreamSuite.scala b1ac4ad [bilna] Added BeforeAndAfter 5f6bfd2 [bilna] Added BeforeAndAfter e8b6623 [Bilna P] Update MQTTStreamSuite.scala 5ca6691 [Bilna P] Update MQTTStreamSuite.scala 8616495 [bilna] [SPARK-4631] unit test for MQTT
* [SPARK-4048] Enhance and extend hadoop-provided profile.Marcelo Vanzin2015-01-083-34/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This change does a few things to make the hadoop-provided profile more useful: - Create new profiles for other libraries / services that might be provided by the infrastructure - Simplify and fix the poms so that the profiles are only activated while building assemblies. - Fix tests so that they're able to run when the profiles are activated - Add a new env variable to be used by distributions that use these profiles to provide the runtime classpath for Spark jobs and daemons. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #2982 from vanzin/SPARK-4048 and squashes the following commits: 82eb688 [Marcelo Vanzin] Add a comment. eb228c0 [Marcelo Vanzin] Fix borked merge. 4e38f4e [Marcelo Vanzin] Merge branch 'master' into SPARK-4048 9ef79a3 [Marcelo Vanzin] Alternative way to propagate test classpath to child processes. 371ebee [Marcelo Vanzin] Review feedback. 52f366d [Marcelo Vanzin] Merge branch 'master' into SPARK-4048 83099fc [Marcelo Vanzin] Merge branch 'master' into SPARK-4048 7377e7b [Marcelo Vanzin] Merge branch 'master' into SPARK-4048 322f882 [Marcelo Vanzin] Fix merge fail. f24e9e7 [Marcelo Vanzin] Merge branch 'master' into SPARK-4048 8b00b6a [Marcelo Vanzin] Merge branch 'master' into SPARK-4048 9640503 [Marcelo Vanzin] Cleanup child process log message. 115fde5 [Marcelo Vanzin] Simplify a comment (and make it consistent with another pom). e3ab2da [Marcelo Vanzin] Fix hive-thriftserver profile. 7820d58 [Marcelo Vanzin] Fix CliSuite with provided profiles. 1be73d4 [Marcelo Vanzin] Restore flume-provided profile. d1399ed [Marcelo Vanzin] Restore jetty dependency. 82a54b9 [Marcelo Vanzin] Remove unused profile. 5c54a25 [Marcelo Vanzin] Fix HiveThriftServer2Suite with *-provided profiles. 1fc4d0b [Marcelo Vanzin] Update dependencies for hive-thriftserver. f7b3bbe [Marcelo Vanzin] Add snappy to hadoop-provided list. 9e4e001 [Marcelo Vanzin] Remove duplicate hive profile. d928d62 [Marcelo Vanzin] Redirect child stderr to parent's log. 4d67469 [Marcelo Vanzin] Propagate SPARK_DIST_CLASSPATH on Yarn. 417d90e [Marcelo Vanzin] Introduce "SPARK_DIST_CLASSPATH". 2f95f0d [Marcelo Vanzin] Propagate classpath to child processes during testing. 1adf91c [Marcelo Vanzin] Re-enable maven-install-plugin for a few projects. 284dda6 [Marcelo Vanzin] Rework the "hadoop-provided" profile, add new ones.
* SPARK-4159 [CORE] Maven build doesn't run JUnit test suitesSean Owen2015-01-0612-81/+11
| | | | | | | | | | | | | | | | | | 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
* [SPARK-4631] unit test for MQTTbilna2015-01-042-15/+101
| | | | | | | | | | | | | | | | | | | | | | | | Please review the unit test for MQTT Author: bilna <bilnap@am.amrita.edu> Author: Bilna P <bilna.p@gmail.com> Closes #3844 from Bilna/master and squashes the following commits: acea3a3 [bilna] Adding dependency with scope test 28681fa [bilna] Merge remote-tracking branch 'upstream/master' fac3904 [bilna] Correction in Indentation and coding style ed9db4c [bilna] Merge remote-tracking branch 'upstream/master' 4b34ee7 [Bilna P] Update MQTTStreamSuite.scala 04503cf [bilna] Added embedded broker service for mqtt test 89d804e [bilna] Merge remote-tracking branch 'upstream/master' fc8eb28 [bilna] Merge remote-tracking branch 'upstream/master' 4b58094 [Bilna P] Update MQTTStreamSuite.scala b1ac4ad [bilna] Added BeforeAndAfter 5f6bfd2 [bilna] Added BeforeAndAfter e8b6623 [Bilna P] Update MQTTStreamSuite.scala 5ca6691 [Bilna P] Update MQTTStreamSuite.scala 8616495 [bilna] [SPARK-4631] unit test for MQTT
* [SPARK-1010] Clean up uses of System.setProperty in unit testsJosh Rosen2014-12-304-8/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Several of our tests call System.setProperty (or test code which implicitly sets system properties) and don't always reset/clear the modified properties, which can create ordering dependencies between tests and cause hard-to-diagnose failures. This patch removes most uses of System.setProperty from our tests, since in most cases we can use SparkConf to set these configurations (there are a few exceptions, including the tests of SparkConf itself). For the cases where we continue to use System.setProperty, this patch introduces a `ResetSystemProperties` ScalaTest mixin class which snapshots the system properties before individual tests and to automatically restores them on test completion / failure. See the block comment at the top of the ResetSystemProperties class for more details. Author: Josh Rosen <joshrosen@databricks.com> Closes #3739 from JoshRosen/cleanup-system-properties-in-tests and squashes the following commits: 0236d66 [Josh Rosen] Replace setProperty uses in two example programs / tools 3888fe3 [Josh Rosen] Remove setProperty use in LocalJavaStreamingContext 4f4031d [Josh Rosen] Add note on why SparkSubmitSuite needs ResetSystemProperties 4742a5b [Josh Rosen] Clarify ResetSystemProperties trait inheritance ordering. 0eaf0b6 [Josh Rosen] Remove setProperty call in TaskResultGetterSuite. 7a3d224 [Josh Rosen] Fix trait ordering 3fdb554 [Josh Rosen] Remove setProperty call in TaskSchedulerImplSuite bee20df [Josh Rosen] Remove setProperty calls in SparkContextSchedulerCreationSuite 655587c [Josh Rosen] Remove setProperty calls in JobCancellationSuite 3f2f955 [Josh Rosen] Remove System.setProperty calls in DistributedSuite cfe9cce [Josh Rosen] Remove use of system properties in SparkContextSuite 8783ab0 [Josh Rosen] Remove TestUtils.setSystemProperty, since it is subsumed by the ResetSystemProperties trait. 633a84a [Josh Rosen] Remove use of system properties in FileServerSuite 25bfce2 [Josh Rosen] Use ResetSystemProperties in UtilsSuite 1d1aa5a [Josh Rosen] Use ResetSystemProperties in SizeEstimatorSuite dd9492b [Josh Rosen] Use ResetSystemProperties in AkkaUtilsSuite b0daff2 [Josh Rosen] Use ResetSystemProperties in BlockManagerSuite e9ded62 [Josh Rosen] Use ResetSystemProperties in TaskSchedulerImplSuite 5b3cb54 [Josh Rosen] Use ResetSystemProperties in SparkListenerSuite 0995c4b [Josh Rosen] Use ResetSystemProperties in SparkContextSchedulerCreationSuite c83ded8 [Josh Rosen] Use ResetSystemProperties in SparkConfSuite 51aa870 [Josh Rosen] Use withSystemProperty in ShuffleSuite 60a63a1 [Josh Rosen] Use ResetSystemProperties in JobCancellationSuite 14a92e4 [Josh Rosen] Use withSystemProperty in FileServerSuite 628f46c [Josh Rosen] Use ResetSystemProperties in DistributedSuite 9e3e0dd [Josh Rosen] Add ResetSystemProperties test fixture mixin; use it in SparkSubmitSuite. 4dcea38 [Josh Rosen] Move withSystemProperty to TestUtils class.
* 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-3154][STREAMING] Replace ConcurrentHashMap with mutable.HashMap and ↵zsxwing2014-12-081-11/+12
| | | | | | | | | | | | remove @volatile from 'stopped' Since `sequenceNumberToProcessor` and `stopped` are both protected by the lock `sequenceNumberToProcessor`, `ConcurrentHashMap` and `volatile` is unnecessary. So this PR updated them accordingly. Author: zsxwing <zsxwing@gmail.com> Closes #3634 from zsxwing/SPARK-3154 and squashes the following commits: 0d087ac [zsxwing] Replace ConcurrentHashMap with mutable.HashMap and remove @volatile from 'stopped'
* [SPARK-4632] version updatePrabeesh K2014-11-301-2/+2
| | | | | | | | | | | | | | Author: Prabeesh K <prabsmails@gmail.com> Closes #3495 from prabeesh/master and squashes the following commits: ab03d50 [Prabeesh K] Update pom.xml 8c6437e [Prabeesh K] Revert e10b40a [Prabeesh K] version update dbac9eb [Prabeesh K] Revert ec0b1c3 [Prabeesh K] [SPARK-4632] version update a835505 [Prabeesh K] [SPARK-4632] version update 831391b [Prabeesh K] [SPARK-4632] version update
* SPARK-3962 Marked scope as provided for external projects.Prashant Sharma2014-11-1914-48/+264
| | | | | | | | | | | | | 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-186-6/+6
| | | | | | | | | | | | 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
* Support cross building for Scala 2.11Prashant Sharma2014-11-111-5/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Let's give this another go using a version of Hive that shades its JLine dependency. Author: Prashant Sharma <prashant.s@imaginea.com> Author: Patrick Wendell <pwendell@gmail.com> Closes #3159 from pwendell/scala-2.11-prashant and squashes the following commits: e93aa3e [Patrick Wendell] Restoring -Phive-thriftserver profile and cleaning up build script. f65d17d [Patrick Wendell] Fixing build issue due to merge conflict a8c41eb [Patrick Wendell] Reverting dev/run-tests back to master state. 7a6eb18 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into scala-2.11-prashant 583aa07 [Prashant Sharma] REVERT ME: removed hive thirftserver 3680e58 [Prashant Sharma] Revert "REVERT ME: Temporarily removing some Cli tests." 935fb47 [Prashant Sharma] Revert "Fixed by disabling a few tests temporarily." 925e90f [Prashant Sharma] Fixed by disabling a few tests temporarily. 2fffed3 [Prashant Sharma] Exclude groovy from sbt build, and also provide a way for such instances in future. 8bd4e40 [Prashant Sharma] Switched to gmaven plus, it fixes random failures observer with its predecessor gmaven. 5272ce5 [Prashant Sharma] SPARK_SCALA_VERSION related bugs. 2121071 [Patrick Wendell] Migrating version detection to PySpark b1ed44d [Patrick Wendell] REVERT ME: Temporarily removing some Cli tests. 1743a73 [Patrick Wendell] Removing decimal test that doesn't work with Scala 2.11 f5cad4e [Patrick Wendell] Add Scala 2.11 docs 210d7e1 [Patrick Wendell] Revert "Testing new Hive version with shaded jline" 48518ce [Patrick Wendell] Remove association of Hive and Thriftserver profiles. e9d0a06 [Patrick Wendell] Revert "Enable thritfserver for Scala 2.10 only" 67ec364 [Patrick Wendell] Guard building of thriftserver around Scala 2.10 check 8502c23 [Patrick Wendell] Enable thritfserver for Scala 2.10 only e22b104 [Patrick Wendell] Small fix in pom file ec402ab [Patrick Wendell] Various fixes 0be5a9d [Patrick Wendell] Testing new Hive version with shaded jline 4eaec65 [Prashant Sharma] Changed scripts to ignore target. 5167bea [Prashant Sharma] small correction a4fcac6 [Prashant Sharma] Run against scala 2.11 on jenkins. 80285f4 [Prashant Sharma] MAven equivalent of setting spark.executor.extraClasspath during tests. 034b369 [Prashant Sharma] Setting test jars on executor classpath during tests from sbt. d4874cb [Prashant Sharma] Fixed Python Runner suite. null check should be first case in scala 2.11. 6f50f13 [Prashant Sharma] Fixed build after rebasing with master. We should use ${scala.binary.version} instead of just 2.10 e56ca9d [Prashant Sharma] Print an error if build for 2.10 and 2.11 is spotted. 937c0b8 [Prashant Sharma] SCALA_VERSION -> SPARK_SCALA_VERSION cb059b0 [Prashant Sharma] Code review 0476e5e [Prashant Sharma] Scala 2.11 support with repl and all build changes.
* [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-4295][External]Fix exception in SparkSinkSuitemaji20142014-11-112-0/+30
| | | | | | | | | | | | Handle exception in SparkSinkSuite, please refer to [SPARK-4295] Author: maji2014 <maji3@asiainfo.com> Closes #3177 from maji2014/spark-4295 and squashes the following commits: 312620a [maji2014] change a new statement for spark-4295 24c3d21 [maji2014] add log4j.properties for SparkSinkSuite and spark-4295 c807bf6 [maji2014] Fix exception in SparkSinkSuite
* [SPARK-4183] Close transport-related resources between SparkContextsAaron Davidson2014-11-021-5/+10
| | | | | | | | | | | | A leak of event loops may be causing test failures. Author: Aaron Davidson <aaron@databricks.com> Closes #3053 from aarondav/leak and squashes the following commits: e676d18 [Aaron Davidson] Typo! 8f96475 [Aaron Davidson] Keep original ssc semantics 7e49f10 [Aaron Davidson] A leak of event loops may be causing test failures.
* [SPARK-4080] Only throw IOException from [write|read][Object|External]Josh Rosen2014-10-241-2/+2
| | | | | | | | | | | | | | | | | If classes implementing Serializable or Externalizable interfaces throw exceptions other than IOException or ClassNotFoundException from their (de)serialization methods, then this results in an unhelpful "IOException: unexpected exception type" rather than the actual exception that produced the (de)serialization error. This patch fixes this by adding a utility method that re-wraps any uncaught exceptions in IOException (unless they are already instances of IOException). Author: Josh Rosen <joshrosen@databricks.com> Closes #2932 from JoshRosen/SPARK-4080 and squashes the following commits: cd3a9be [Josh Rosen] [SPARK-4080] Only throw IOException from [write|read][Object|External].
* [SPARK-3912][Streaming] Fixed flakyFlumeStreamSuiteTathagata Das2014-10-131-64/+102
| | | | | | | | | | | | | | | | | @harishreedharan @pwendell See JIRA for diagnosis of the problem https://issues.apache.org/jira/browse/SPARK-3912 The solution was to reimplement it. 1. Find a free port (by binding and releasing a server-scoket), and then use that port 2. Remove thread.sleep()s, instead repeatedly try to create a sender and send data and check whether data was sent. Use eventually() to minimize waiting time. 3. Check whether all the data was received, without caring about batches. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #2773 from tdas/flume-test-fix and squashes the following commits: 93cd7f6 [Tathagata Das] Reimplimented FlumeStreamSuite to be more robust.
* [SPARK-3748] Log thread name in unit test logsReynold Xin2014-10-015-5/+5
| | | | | | | | | | 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-3744 [STREAMING] FlumeStreamSuite will fail during port contentionSean Owen2014-09-301-10/+15
| | | | | | | | | | Since it looked quite easy, I took the liberty of making a quick PR that just uses `Utils.startServiceOnPort` to fix this. It works locally for me. Author: Sean Owen <sowen@cloudera.com> Closes #2601 from srowen/SPARK-3744 and squashes the following commits: ddc9319 [Sean Owen] Avoid port contention in tests by retrying several ports for Flume stream
* [SPARK-3686][STREAMING] Wait for sink to commit the channel before check...Hari Shreedharan2014-09-254-10/+48
| | | | | | | | | | | ...ing for the channel size. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #2531 from harishreedharan/sparksinksuite-fix and squashes the following commits: 30393c1 [Hari Shreedharan] Use more deterministic method to figure out when batches come in. 6ce9d8b [Hari Shreedharan] [SPARK-3686][STREAMING] Wait for sink to commit the channel before checking for the channel size.
* [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-066-6/+6
| | | | | | | | 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
* [Minor]Remove extra semicolon in FlumeStreamSuite.scalaGuoQiang Li2014-09-041-5/+5
| | | | | | | | Author: GuoQiang Li <witgo@qq.com> Closes #2265 from witgo/FlumeStreamSuite and squashes the following commits: 6c99e6e [GuoQiang Li] Remove extra semicolon in FlumeStreamSuite.scala
* [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner.Hari Shreedharan2014-08-275-86/+236
| | | | | | | | | | | | | | | | | | | | | | | | Currently lot of errors get thrown from Avro IPC layer when the dstream or sink is shutdown. This PR cleans it up. Some refactoring is done in the receiver code to put all of the RPC code into a single Try and just recover from that. The sink code has also been cleaned up. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #2065 from harishreedharan/clean-flume-shutdown and squashes the following commits: f93a07c [Hari Shreedharan] Formatting fixes. d7427cc [Hari Shreedharan] More fixes! a0a8852 [Hari Shreedharan] Fix race condition, hopefully! Minor other changes. 4c9ed02 [Hari Shreedharan] Remove unneeded list in Callback handler. Other misc changes. 8fee36f [Hari Shreedharan] Scala-library is required, else maven build fails. Also catch InterruptedException in TxnProcessor. 445e700 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown 87232e0 [Hari Shreedharan] Refactor Flume Input Stream. Clean up code, better error handling. 9001d26 [Hari Shreedharan] Change log level to debug in TransactionProcessor#shutdown method e7b8d82 [Hari Shreedharan] Incorporate review feedback 598efa7 [Hari Shreedharan] Clean up some exception handling code e1027c6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown ed608c8 [Hari Shreedharan] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner.
* SPARK-2798 [BUILD] Correct several small errors in Flume module pom.xml filesSean Owen2014-08-252-15/+12
| | | | | | | | | | | | | | | | (EDIT) Since the scalatest issue was since resolved, this is now about a few small problems in the Flume Sink `pom.xml` - `scalatest` is not declared as a test-scope dependency - Its Avro version doesn't match the rest of the build - Its Flume version is not synced with the other Flume module - The other Flume module declares its dependency on Flume Sink slightly incorrectly, hard-coding the Scala 2.10 version - It depends on Scala Lang directly, which it shouldn't Author: Sean Owen <sowen@cloudera.com> Closes #1726 from srowen/SPARK-2798 and squashes the following commits: a46e2c6 [Sean Owen] scalatest to test scope, harmonize Avro and Flume versions, remove direct Scala dependency, fix '2.10' in Flume dependency
* [SPARK-3169] Removed dependency on spark streaming test from spark flume sinkTathagata Das2014-08-222-11/+17
| | | | | | | | | | | | | Due to maven bug https://jira.codehaus.org/browse/MNG-1378, maven could not resolve spark streaming classes required by the spark-streaming test-jar dependency of external/flume-sink. There is no particular reason that the external/flume-sink has to depend on Spark Streaming at all, so I am eliminating this dependency. Also I have removed the exclusions present in the Flume dependencies, as there is no reason to exclude them (they were excluded in the external/flume module to prevent dependency collisions with Spark). Since Jenkins will test the sbt build and the unit test, I only tested maven compilation locally. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #2101 from tdas/spark-sink-pom-fix and squashes the following commits: 8f42621 [Tathagata Das] Added Flume sink exclusions back, and added netty to test dependencies 93b559f [Tathagata Das] Removed dependency on spark streaming test from spark flume sink
* [SPARK-3054][STREAMING] Add unit tests for Spark Sink.Hari Shreedharan2014-08-204-2/+212
| | | | | | | | | | | | | | | | | | | | | | | | This patch adds unit tests for Spark Sink. It also removes the private[flume] for Spark Sink, since the sink is instantiated from Flume configuration (looks like this is ignored by reflection which is used by Flume, but we should still remove it anyway). Author: Hari Shreedharan <hshreedharan@apache.org> Author: Hari Shreedharan <hshreedharan@cloudera.com> Closes #1958 from harishreedharan/spark-sink-test and squashes the following commits: e3110b9 [Hari Shreedharan] Add a sleep to allow sink to commit the transactions 120b81e [Hari Shreedharan] Fix complexity in threading model in test 4df5be6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into spark-sink-test c9190d1 [Hari Shreedharan] Indentation and spaces changes 7fedc5a [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into spark-sink-test abc20cb [Hari Shreedharan] Minor test changes 7b9b649 [Hari Shreedharan] Merge branch 'master' into spark-sink-test f2c56c9 [Hari Shreedharan] Update SparkSinkSuite.scala a24aac8 [Hari Shreedharan] Remove unused var c86d615 [Hari Shreedharan] [SPARK-3054][STREAMING] Add unit tests for Spark Sink.
* [HOTFIX][STREAMING] Allow the JVM/Netty to decide which port to bind to in ↵Hari Shreedharan2014-08-172-29/+34
| | | | | | | | | | | | | | | | | | Flume Polling Tests. Author: Hari Shreedharan <harishreedharan@gmail.com> Closes #1820 from harishreedharan/use-free-ports and squashes the following commits: b939067 [Hari Shreedharan] Remove unused import. 67856a8 [Hari Shreedharan] Remove findFreePort. 0ea51d1 [Hari Shreedharan] Make some changes to getPort to use map on the serverOpt. 1fb0283 [Hari Shreedharan] Merge branch 'master' of https://github.com/apache/spark into use-free-ports b351651 [Hari Shreedharan] Allow Netty to choose port, and query it to decide the port to bind to. Leaving findFreePort as is, if other tests want to use it at some point. e6c9620 [Hari Shreedharan] Making sure the second sink uses the correct port. 11c340d [Hari Shreedharan] Add info about race condition to scaladoc. e89d135 [Hari Shreedharan] Adding Scaladoc. 6013bb0 [Hari Shreedharan] [STREAMING] Find free ports to use before attempting to create Flume Sink in Flume Polling Suite
* [HOTFIX][Streaming] Handle port collisions in flume polling testAndrew Or2014-08-061-1/+31
| | | | | | | | | | | | | This is failing my tests in #1777. @tdas Author: Andrew Or <andrewor14@gmail.com> Closes #1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits: ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions 54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test 664095c [Andrew Or] Tone down bind exception message af3ddc9 [Andrew Or] Handle port collisions in flume polling test
* [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