Re: Spark Streaming : is spark.streaming.receiver.maxRate valid for DirectKafkaApproach

2016-05-10 Thread Cody Koeninger
Pretty much the same problems you'd expect any time you have skew in a
distributed system - some leaders are going to be working harder than
others & have more disk space used, some consumers are going to be
working harder than others.

It sounds like you're talking about differences in topics, not
partitions, although 3 partitions per topic may not be enough to
balance depending on the size of your cluster.  If your job has
significantly higher number of topicpartitions than it does executors,
that reduces the chance that some executors will be idle though,
because once an executor finishes processing an rdd partition for a
small topic it will be assigned another one.

If you're worried about some particular topic monopolizing resources,
maxRatePerPartition will let you limit that.  If you have some more
complicated need, you may need to modify the code to suit your
purposes.




On Tue, May 10, 2016 at 10:50 AM, chandan prakash
 wrote:
> Hey Cody,
> What kind of problems exactly?
> ...data rate in kafka topics do vary significantly in my
> caseout of total 50 topics(with 3 partitions each),half of the
> topics generate data at very high speed say 1lakh/sec while other half
> generate at very low rate say 1k/sec...
> i have to process them together and insert into the same database
> table...will it be better to have 2 different spark streaming
> applications instead?
> I dont have control over kafka topics and partitions, they are a central
> system used by many other systems as well.
>
> Regards,
> Chandan
>
> On Tue, May 10, 2016 at 8:01 PM, Cody Koeninger  wrote:
>>
>> maxRate is not used by the direct stream.
>>
>> Significant skew in rate across different partitions for the same
>> topic is going to cause you all kinds of problems, not just with spark
>> streaming.
>>
>> You can turn on backpressure, but you're better off addressing the
>> underlying issue if you can.
>>
>> On Tue, May 10, 2016 at 8:08 AM, Soumitra Siddharth Johri
>>  wrote:
>> > Also look at back pressure enabled. Both of these can be used to limit
>> > the
>> > rate
>> >
>> > Sent from my iPhone
>> >
>> > On May 10, 2016, at 8:02 AM, chandan prakash 
>> > wrote:
>> >
>> > Hi,
>> > I am using Spark Streaming with Direct kafka approach.
>> > Want to limit number of event records coming in my batches.
>> > Have question regarding  following 2 parameters :
>> > 1. spark.streaming.receiver.maxRate
>> > 2. spark.streaming.kafka.maxRatePerPartition
>> >
>> >
>> > The documentation
>> >
>> > (http://spark.apache.org/docs/latest/streaming-programming-guide.html#deploying-applications
>> > ) says .
>> > " spark.streaming.receiver.maxRate for receivers and
>> > spark.streaming.kafka.maxRatePerPartition for Direct Kafka approach "
>> >
>> > Does it mean that  spark.streaming.receiver.maxRate  is valid only for
>> > Receiver based approach only ?  (not the DirectKafkaApproach as well)
>> >
>> > If yes, then how do we control total number of records/sec in
>> > DirectKafka
>> > ?.because spark.streaming.kafka.maxRatePerPartition  only controls
>> > max
>> > rate per partition and not whole records. There might be many
>> > partitions
>> > some with very fast rate and some with very slow rate.
>> >
>> > Regards,
>> > Chandan
>> >
>> >
>> >
>> > --
>> > Chandan Prakash
>> >
>
>
>
>
> --
> Chandan Prakash
>

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



Re: Spark Streaming : is spark.streaming.receiver.maxRate valid for DirectKafkaApproach

2016-05-10 Thread chandan prakash
Hey Cody,
What kind of problems exactly?
...data rate in kafka topics do vary significantly in my
caseout of total 50 topics(with 3 partitions each),half of the
topics generate data at very high speed say 1lakh/sec while other half
generate at very low rate say 1k/sec...
i have to process them together and insert into the same database
table...will it be better to have 2 different spark streaming
applications instead?
I dont have control over kafka topics and partitions, they are a central
system used by many other systems as well.

Regards,
Chandan

On Tue, May 10, 2016 at 8:01 PM, Cody Koeninger  wrote:

> maxRate is not used by the direct stream.
>
> Significant skew in rate across different partitions for the same
> topic is going to cause you all kinds of problems, not just with spark
> streaming.
>
> You can turn on backpressure, but you're better off addressing the
> underlying issue if you can.
>
> On Tue, May 10, 2016 at 8:08 AM, Soumitra Siddharth Johri
>  wrote:
> > Also look at back pressure enabled. Both of these can be used to limit
> the
> > rate
> >
> > Sent from my iPhone
> >
> > On May 10, 2016, at 8:02 AM, chandan prakash 
> > wrote:
> >
> > Hi,
> > I am using Spark Streaming with Direct kafka approach.
> > Want to limit number of event records coming in my batches.
> > Have question regarding  following 2 parameters :
> > 1. spark.streaming.receiver.maxRate
> > 2. spark.streaming.kafka.maxRatePerPartition
> >
> >
> > The documentation
> > (
> http://spark.apache.org/docs/latest/streaming-programming-guide.html#deploying-applications
> > ) says .
> > " spark.streaming.receiver.maxRate for receivers and
> > spark.streaming.kafka.maxRatePerPartition for Direct Kafka approach "
> >
> > Does it mean that  spark.streaming.receiver.maxRate  is valid only for
> > Receiver based approach only ?  (not the DirectKafkaApproach as well)
> >
> > If yes, then how do we control total number of records/sec in DirectKafka
> > ?.because spark.streaming.kafka.maxRatePerPartition  only controls
> max
> > rate per partition and not whole records. There might be many
> partitions
> > some with very fast rate and some with very slow rate.
> >
> > Regards,
> > Chandan
> >
> >
> >
> > --
> > Chandan Prakash
> >
>



-- 
Chandan Prakash


Re: Spark Streaming : is spark.streaming.receiver.maxRate valid for DirectKafkaApproach

2016-05-10 Thread Soumitra Johri
I think a better partitioning scheme can help u too.
On Tue, May 10, 2016 at 10:31 AM Cody Koeninger  wrote:

> maxRate is not used by the direct stream.
>
> Significant skew in rate across different partitions for the same
> topic is going to cause you all kinds of problems, not just with spark
> streaming.
>
> You can turn on backpressure, but you're better off addressing the
> underlying issue if you can.
>
> On Tue, May 10, 2016 at 8:08 AM, Soumitra Siddharth Johri
>  wrote:
> > Also look at back pressure enabled. Both of these can be used to limit
> the
> > rate
> >
> > Sent from my iPhone
> >
> > On May 10, 2016, at 8:02 AM, chandan prakash 
> > wrote:
> >
> > Hi,
> > I am using Spark Streaming with Direct kafka approach.
> > Want to limit number of event records coming in my batches.
> > Have question regarding  following 2 parameters :
> > 1. spark.streaming.receiver.maxRate
> > 2. spark.streaming.kafka.maxRatePerPartition
> >
> >
> > The documentation
> > (
> http://spark.apache.org/docs/latest/streaming-programming-guide.html#deploying-applications
> > ) says .
> > " spark.streaming.receiver.maxRate for receivers and
> > spark.streaming.kafka.maxRatePerPartition for Direct Kafka approach "
> >
> > Does it mean that  spark.streaming.receiver.maxRate  is valid only for
> > Receiver based approach only ?  (not the DirectKafkaApproach as well)
> >
> > If yes, then how do we control total number of records/sec in DirectKafka
> > ?.because spark.streaming.kafka.maxRatePerPartition  only controls
> max
> > rate per partition and not whole records. There might be many
> partitions
> > some with very fast rate and some with very slow rate.
> >
> > Regards,
> > Chandan
> >
> >
> >
> > --
> > Chandan Prakash
> >
>


Re: Spark Streaming : is spark.streaming.receiver.maxRate valid for DirectKafkaApproach

2016-05-10 Thread Cody Koeninger
maxRate is not used by the direct stream.

Significant skew in rate across different partitions for the same
topic is going to cause you all kinds of problems, not just with spark
streaming.

You can turn on backpressure, but you're better off addressing the
underlying issue if you can.

On Tue, May 10, 2016 at 8:08 AM, Soumitra Siddharth Johri
 wrote:
> Also look at back pressure enabled. Both of these can be used to limit the
> rate
>
> Sent from my iPhone
>
> On May 10, 2016, at 8:02 AM, chandan prakash 
> wrote:
>
> Hi,
> I am using Spark Streaming with Direct kafka approach.
> Want to limit number of event records coming in my batches.
> Have question regarding  following 2 parameters :
> 1. spark.streaming.receiver.maxRate
> 2. spark.streaming.kafka.maxRatePerPartition
>
>
> The documentation
> (http://spark.apache.org/docs/latest/streaming-programming-guide.html#deploying-applications
> ) says .
> " spark.streaming.receiver.maxRate for receivers and
> spark.streaming.kafka.maxRatePerPartition for Direct Kafka approach "
>
> Does it mean that  spark.streaming.receiver.maxRate  is valid only for
> Receiver based approach only ?  (not the DirectKafkaApproach as well)
>
> If yes, then how do we control total number of records/sec in DirectKafka
> ?.because spark.streaming.kafka.maxRatePerPartition  only controls max
> rate per partition and not whole records. There might be many partitions
> some with very fast rate and some with very slow rate.
>
> Regards,
> Chandan
>
>
>
> --
> Chandan Prakash
>

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



Re: Spark Streaming : is spark.streaming.receiver.maxRate valid for DirectKafkaApproach

2016-05-10 Thread Soumitra Siddharth Johri
Also look at back pressure enabled. Both of these can be used to limit the rate

Sent from my iPhone

> On May 10, 2016, at 8:02 AM, chandan prakash  
> wrote:
> 
> Hi,
> I am using Spark Streaming with Direct kafka approach.
> Want to limit number of event records coming in my batches.
> Have question regarding  following 2 parameters : 
> 1. spark.streaming.receiver.maxRate
> 2. spark.streaming.kafka.maxRatePerPartition
> 
> 
> The documentation 
> (http://spark.apache.org/docs/latest/streaming-programming-guide.html#deploying-applications
>  ) says .
> " spark.streaming.receiver.maxRate for receivers and 
> spark.streaming.kafka.maxRatePerPartition for Direct Kafka approach "
> 
> Does it mean that  spark.streaming.receiver.maxRate  is valid only for 
> Receiver based approach only ?  (not the DirectKafkaApproach as well)
> 
> If yes, then how do we control total number of records/sec in DirectKafka 
> ?.because spark.streaming.kafka.maxRatePerPartition  only controls max 
> rate per partition and not whole records. There might be many partitions 
> some with very fast rate and some with very slow rate.
> 
> Regards,
> Chandan
> 
> 
> 
> -- 
> Chandan Prakash
>