aboutsummaryrefslogtreecommitdiff
path: root/external/kafka-0-8
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16950] [PYSPARK] fromOffsets parameter support in ↵Mariusz Strzelecki2016-08-091-4/+4
| | | | | | | | | | | | | | | | | | KafkaUtils.createDirectStream for python3 ## What changes were proposed in this pull request? Ability to use KafkaUtils.createDirectStream with starting offsets in python 3 by using java.lang.Number instead of Long during param mapping in scala helper. This allows py4j to pass Integer or Long to the map and resolves ClassCastException problems. ## How was this patch tested? unit tests jerryshao - could you please look at this PR? Author: Mariusz Strzelecki <mariusz.strzelecki@allegrogroup.com> Closes #14540 from szczeles/kafka_pyspark.
* [SPARK-16779][TRIVIAL] Avoid using postfix operators where they do not add ↵Holden Karau2016-08-081-2/+0
| | | | | | | | | | | | | | | | much and remove whitelisting ## What changes were proposed in this pull request? Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability. ## How was this patch tested? Existing tests. Author: Holden Karau <holden@us.ibm.com> Closes #14407 from holdenk/SPARK-16779.
* [SPARK-16535][BUILD] In pom.xml, remove groupId which is redundant ↵Xin Ren2016-07-191-1/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | definition and inherited from the parent https://issues.apache.org/jira/browse/SPARK-16535 ## What changes were proposed in this pull request? When I scan through the pom.xml of sub projects, I found this warning as below and attached screenshot ``` Definition of groupId is redundant, because it's inherited from the parent ``` ![screen shot 2016-07-13 at 3 13 11 pm](https://cloud.githubusercontent.com/assets/3925641/16823121/744f893e-4916-11e6-8a52-042f83b9db4e.png) I've tried to remove some of the lines with groupId definition, and the build on my local machine is still ok. ``` <groupId>org.apache.spark</groupId> ``` As I just find now `<maven.version>3.3.9</maven.version>` is being used in Spark 2.x, and Maven-3 supports versionless parent elements: Maven 3 will remove the need to specify the parent version in sub modules. THIS is great (in Maven 3.1). ref: http://stackoverflow.com/questions/3157240/maven-3-worth-it/3166762#3166762 ## How was this patch tested? I've tested by re-building the project, and build succeeded. Author: Xin Ren <iamshrek@126.com> Closes #14189 from keypointt/SPARK-16535.
* [SPARK-16477] Bump master version to 2.1.0-SNAPSHOTReynold Xin2016-07-111-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #14130 from rxin/SPARK-16477.
* [SPARK-16212][STREAMING][KAFKA] apply test tweaks from 0-10 to 0-8 as wellcody koeninger2016-07-062-25/+24
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Bring the kafka-0-8 subproject up to date with some test modifications from development on 0-10. Main changes are - eliminating waits on concurrent queue in favor of an assert on received results, - atomics instead of volatile (although this probably doesn't matter) - increasing uniqueness of topic names ## How was this patch tested? Unit tests Author: cody koeninger <cody@koeninger.org> Closes #14073 from koeninger/kafka-0-8-test-direct-cleanup.
* [SPARK-16212][STREAMING][KAFKA] use random port for embedded kafkacody koeninger2016-07-051-2/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Testing for 0.10 uncovered an issue with a fixed port number being used in KafkaTestUtils. This is making a roughly equivalent fix for the 0.8 connector ## How was this patch tested? Unit tests, manual tests Author: cody koeninger <cody@koeninger.org> Closes #14018 from koeninger/kafka-0-8-test-port.
* [SPARK-16212][STREAMING][KAFKA] code cleanup from review feedbackcody koeninger2016-06-303-14/+12
| | | | | | | | | | | | ## What changes were proposed in this pull request? code cleanup in kafka-0-8 to match suggested changes for kafka-0-10 branch ## How was this patch tested? unit tests Author: cody koeninger <cody@koeninger.org> Closes #13908 from koeninger/kafka-0-8-cleanup.
* [SPARK-15508][STREAMING][TESTS] Fix flaky test: ↵Shixiong Zhu2016-05-241-6/+15
| | | | | | | | | | | | | | | | | | | | JavaKafkaStreamSuite.testKafkaStream ## What changes were proposed in this pull request? `JavaKafkaStreamSuite.testKafkaStream` assumes when `sent.size == result.size`, the contents of `sent` and `result` should be same. However, that's not true. The content of `result` may not be the final content. This PR modified the test to always retry the assertions even if the contents of `sent` and `result` are not same. Here is the failure in Jenkins: http://spark-tests.appspot.com/tests/org.apache.spark.streaming.kafka.JavaKafkaStreamSuite/testKafkaStream ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13281 from zsxwing/flaky-kafka-test.
* [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into ↵Sean Owen2016-05-171-1/+1
| | | | | | | | | | | | | | | | | | spark-tags ## What changes were proposed in this pull request? (See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.) Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags` ## How was this patch tested? Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #13074 from srowen/SPARK-15290.
* [SPARK-15085][STREAMING][KAFKA] Rename streaming-kafka artifactcody koeninger2016-05-1122-0/+4317
## What changes were proposed in this pull request? Renaming the streaming-kafka artifact to include kafka version, in anticipation of needing a different artifact for later kafka versions ## How was this patch tested? Unit tests Author: cody koeninger <cody@koeninger.org> Closes #12946 from koeninger/SPARK-15085.