[ 
https://issues.apache.org/jira/browse/SPARK-33635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17251911#comment-17251911
 ] 

David Wyles commented on SPARK-33635:
-------------------------------------

[~gsomogyi] Just so you know, I'm still doing this.

I've simplied the test case to kafka to kafka, also tried with v2 sources (by 
removing kafka from the v1 sources list).

On 3.0.1 My data rate is still on the order of 650 - 700k rows (10 partitions, 
700 million rows, 1 core per executors - of which there are 10), there is no 
noticable change when using v1 or v2 sources (v2 may have been every so 
slightly faster, but I'd need a lot more test runs to make that a concrete 
fact, faster by maybe 4%).

I'll get back to you next week when I've run this on 2.4.5

Remember, based on all my testing, and raw kafka reads on my system - the 3.0.1 
spark is performing in line with expectations. The odd one here is 2.4.5, and 
maybe when I run it with these test setup we will get numbers more in line with 
kafka consumer behaviour.

> Performance regression in Kafka read
> ------------------------------------
>
>                 Key: SPARK-33635
>                 URL: https://issues.apache.org/jira/browse/SPARK-33635
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 3.0.0, 3.0.1
>         Environment: A simple 5 node system. A simple data row of csv data in 
> kafka, evenly distributed between the partitions.
> Open JDK 1.8.0.252
> Spark in stand alone - 5 nodes, 10 workers (2 worker per node, each locked to 
> a distinct NUMA group)
> kafka (v 2.3.1) cluster - 5 nodes (1 broker per node).
> Centos 7.7.1908
> 1 topic, 10 partiions, 1 hour queue life
> (this is just one of clusters we have, I have tested on all of them and 
> theyall exhibit the same performance degredation)
>            Reporter: David Wyles
>            Priority: Major
>
> I have observed a slowdown in the reading of data from kafka on all of our 
> systems when migrating from spark 2.4.5 to Spark 3.0.0 (and Spark 3.0.1)
> I have created a sample project to isolate the problem as much as possible, 
> with just a read all data from a kafka topic (see 
> [https://github.com/codegorillauk/spark-kafka-read] ).
> With 2.4.5, across multiple runs, 
>  I get a stable read rate of 1,120,000 (1.12 mill) rows per second
> With 3.0.0 or 3.0.1, across multiple runs,
>  I get a stable read rate of 632,000 (0.632 mil) rows per second
> The represents a *44% loss in performance*. Which is, a lot.
> I have been working though the spark-sql-kafka-0-10 code base, but change for 
> spark 3 have been ongoing for over a year and its difficult to pin point an 
> exact change or reason for the degradation.
> I am happy to help fix this problem, but will need some assitance as I am 
> unfamiliar with the spark-sql-kafka-0-10 project.
>  
> A sample of the data my test reads (note: its not parsing csv - this is just 
> test data)
>  
> 1606921800000,001e0610e532,lightsense,tsl250rd,intensity,21853,53.262,acceleration_z,651,ep,290,commit,913,pressure,138,pm1,799,uv_intensity,823,idletime,-372,count,-72,ir_intensity,185,concentration,-61,flags,-532,tx,694.36,ep_heatsink,-556.92,acceleration_x,-221.40,fw,910.53,sample_flow_rate,-959.60,uptime,-515.15,pm10,-768.03,powersupply,214.72,magnetic_field_y,-616.04,alphasense,606.73,AoT_Chicago,053,Racine
>  Ave & 18th St Chicago IL,41.857959,-87.65642700000002,AoT Chicago (S) 
> [C],2017/12/15 00:00:00,



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to