Skip to content

Conversation

@srowen
Copy link
Member

@srowen srowen commented Oct 11, 2018

What changes were proposed in this pull request?

Remove Kafka 0.8 integration

How was this patch tested?

Existing tests, build scripts

self.ssc.stop(True, True)


class KafkaStreamTests(PySparkStreamingTestCase):
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Am I correct that all of this Pyspark Kafka integration is 0.8, not 0.10? that structured streaming is the only option now for Pyspark + Kafka?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yup. Kafka 0.10 support at PySpark was not added per SPARK-16534.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, you or @holdenk or @koeninger might want to skim this change to make sure I didn't delete Pyspark + Structured Streaming + Kafka support inadvertentently. I don't think so, but it's not my area so much.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I skimmed and seems fine. Will try to take a look few times more while it's open. (don't block by me)

@SparkQA
Copy link

SparkQA commented Oct 11, 2018

Test build #97285 has finished for PR 22703 at commit 4f0bab8.

  • This patch fails executing the dev/run-tests script.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 12, 2018

Test build #97294 has finished for PR 22703 at commit 6e34ce7.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Oct 12, 2018

Test build #97300 has finished for PR 22703 at commit 6e34ce7.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Oct 12, 2018

Test build #97307 has finished for PR 22703 at commit 6e34ce7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

The Spark Streaming integration for Kafka 0.10 provides simple parallelism, 1:1 correspondence between Kafka
partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses
the [new Kafka consumer API](https://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API,
there are notable differences in usage. This version of the integration is marked as experimental, so the API is
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we want to leave the new integration marked as experimental if it is now the only available one?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, good general point. Is the kafka 0.10 integration at all experimental anymore? Is anything that survives from 2.x to 3.x? I'd say "no" in almost all cases. What are your personal views on that?

@koeninger
Copy link
Contributor

koeninger commented Oct 13, 2018 via email

@srowen srowen changed the title [SPARK-25705][BUILD][STREAMING] Remove Kafka 0.8 integration [SPARK-25705][BUILD][STREAMING][test-maven] Remove Kafka 0.8 integration Oct 13, 2018
… code; declare existing Kafka integration non-experimental
@SparkQA
Copy link

SparkQA commented Oct 13, 2018

Test build #97343 has finished for PR 22703 at commit 3d44772.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 13, 2018

Test build #4377 has finished for PR 22703 at commit 6e34ce7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 14, 2018

Test build #4378 has finished for PR 22703 at commit 3d44772.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member Author

srowen commented Oct 15, 2018

So far looking good to those who have looked, and it passed Maven and SBT tests. I think this will help reduce complexity a bit (and test time in some cases), so will go for it tomorrow.

@asfgit asfgit closed this in 703e6da Oct 16, 2018
@srowen srowen deleted the SPARK-25705 branch October 24, 2018 16:47
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
## What changes were proposed in this pull request?

Remove Kafka 0.8 integration

## How was this patch tested?

Existing tests, build scripts

Closes apache#22703 from srowen/SPARK-25705.

Authored-by: Sean Owen <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants