spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Saisai Shao (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-11045) Contributing Receiver based Low Level Kafka Consumer from Spark-Packages to Apache Spark Project
Date Mon, 12 Oct 2015 03:57:05 GMT

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

Saisai Shao commented on SPARK-11045:
-------------------------------------

Hi Dib,

To get better performance and fix the issues as you mentioned, you have to do  a lot of works
that should be done in Kafka side and unavoidable in your codebase, that's why Storm Kafka
connector is so complicated and makes difficult for others to maintain. If finally Kafka's
solution is better than your current solution, how to embrace that solution is another thing
should be considered.

Whether to choose the low-level API or high-level API is not the key point here, the reason
to choose low-level API in Storm Kafka connector and here in direct Kafka stream is that low-level
API is more flexible and performant right now, but if high-level API also brings such features,
I don't think we have to stick on low-level API, especially for receiver-based Kafka connector.

Just my concern :).

> Contributing Receiver based Low Level Kafka Consumer from Spark-Packages to Apache Spark
Project
> ------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-11045
>                 URL: https://issues.apache.org/jira/browse/SPARK-11045
>             Project: Spark
>          Issue Type: New Feature
>          Components: Streaming
>            Reporter: Dibyendu Bhattacharya
>
> This JIRA is to track the progress of making the Receiver based Low Level Kafka Consumer
from spark-packages (http://spark-packages.org/package/dibbhatt/kafka-spark-consumer) to be
contributed back to Apache Spark Project.
> This Kafka consumer has been around for more than year and has matured over the time
. I see there are many adoptions of this package . I receive positive feedbacks that this
consumer gives better performance and fault tolerant capabilities. 
> This is the primary intent of this JIRA to give community a better alternative if they
want to use Receiver Base model. 
> If this consumer make it to Spark Core, it will definitely see more adoption and support
from community and help many who still prefer the Receiver Based model of Kafka Consumer.

> I understand the Direct Stream is the consumer which can give Exact Once semantics and
uses Kafka Low Level API  , which is good . But Direct Stream has concerns around recovering
checkpoint on driver code change . Application developer need to manage their own offset which
complex . Even if some one does manages their own offset , it limits the parallelism Spark
Streaming can achieve. If someone wants more parallelism and want spark.streaming.concurrentJobs
more than 1 , you can no longer rely on storing offset externally as you have no control which
batch will run in which sequence. 
> Furthermore , the Direct Stream has higher latency , as it fetch messages form Kafka
during RDD action . Also number of RDD partitions are limited to topic partition . So unless
your Kafka topic does not have enough partitions, you have limited parallelism while RDD processing.

> Due to above mentioned concerns , many people who does not want Exactly Once semantics
, still prefer Receiver based model. Unfortunately, when customer fall back to KafkaUtil.CreateStream
approach, which use Kafka High Level Consumer, there are other issues around the reliability
of Kafka High Level API.  Kafka High Level API is buggy and has serious issue around Consumer
Re-balance. Hence I do not think this is correct to advice people to use KafkaUtil.CreateStream
in production . 
> The better option presently is there is to use the Consumer from spark-packages . It
is is using Kafka Low Level Consumer API , store offset in Zookeeper, and can recover from
any failure . Below are few highlights of this consumer  ..
> 1. It has a inbuilt PID Controller for dynamic rate limiting.
> 2. In this consumer ,  The Rate Limiting is done by modifying the size blocks by controlling
the size of messages pulled from Kafka. Whereas , in Spark the Rate Limiting is done by controlling
number of  messages. The issue with throttling by number of message is, if message size various,
block size will also vary . Let say your Kafka has messages for different sizes from 10KB
to 500 KB. Thus throttling by number of message can never give any deterministic size of your
block hence there is no guarantee that Memory Back-Pressure can really take affect. 
> 3. This consumer is using Kafka low level API which gives better performance than KafkaUtils.createStream
based High Level API.
> 4. This consumer can give end to end no data loss channel if enabled with WAL.
> By accepting this low level kafka consumer from spark packages to apache spark project
, we will give community a better options for Kafka connectivity both for Receiver less and
Receiver based model. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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


Mime
View raw message