>  500ms is I believe the minimum batch interval for Spark micro batching.

It's better to test than to believe, I've run 250ms jobs.  Same
applies to the comments around JDBC, why assume when you could
(dis)prove?  It's not like it's a lot of effort to set up a minimal
job that does foreach(println) from kafka.

On Tue, May 31, 2016 at 9:59 AM, Mich Talebzadeh
<mich.talebza...@gmail.com> wrote:
> 500ms is I believe the minimum batch interval for Spark micro batching.
>
> However, a JDBC call is a use of Unix file descriptor and context switch and
> it does have performance implication. That is irrespective of Kafka as it is
> happening one is actually going through Hive JDBC.
>
> It is a classic data access issue. Opening and closing JDBC connection once
> every 0.5 second is very problematic.
>
> HTH
>
> Dr Mich Talebzadeh
>
>
>
> LinkedIn
> https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
>
>
>
> http://talebzadehmich.wordpress.com
>
>
>
>
> On 31 May 2016 at 15:34, Cody Koeninger <c...@koeninger.org> wrote:
>>
>> There isn't a magic spark configuration setting that would account for
>> multiple-second-long fixed overheads, you should be looking at maybe
>> 200ms minimum for a streaming batch.  1024 kafka topicpartitions is
>> not reasonable for the volume you're talking about.  Unless you have
>> really extreme workloads, 32 or 64 is a better starting guess.
>>
>> Rather than jumping to conclusions about sql operations being the
>> problem, start from the very beginning.  Read a stream of messages
>> from kafka and just do .foreach(println), at a reasonable batch size
>> (say 500ms or a second), and see how that keeps up in your
>> environment.  Don't use take(), don't use count(), don't use print(),
>> since they may have non-obvious performance implications.
>>
>> If that works, add on further operations one step at a time and see
>> when issues arise.
>>
>> On Mon, May 30, 2016 at 8:45 PM, Malcolm Lockyer
>> <malcolm.lock...@hapara.com> wrote:
>> > Hopefully this is not off topic for this list, but I am hoping to
>> > reach some people who have used Kafka + Spark before.
>> >
>> > We are new to Spark and are setting up our first production
>> > environment and hitting a speed issue that maybe configuration related
>> > - and we have little experience in configuring Spark environments.
>> >
>> > So we've got a Spark streaming job that seems to take an inordinate
>> > amount of time to process. I realize that without specifics, it is
>> > difficult to trace - however the most basic primitives in Spark are
>> > performing horribly. The lazy nature of Spark is making it difficult
>> > for me to understand what is happening - any suggestions are very much
>> > appreciated.
>> >
>> > Environment is MBP 2.2 i7. Spark master is "local[*]". We are using
>> > Kafka and PostgreSQL, both local. The job is designed to:
>> >
>> > a) grab some data from Kafka
>> > b) correlate with existing data in PostgreSQL
>> > c) output data to Kafka
>> >
>> > I am isolating timings by calling System.nanoTime() before and after
>> > something that forces calculation, for example .count() on a
>> > DataFrame. It seems like every operation has a MASSIVE fixed overhead
>> > and that is stacking up making each iteration on the RDD extremely
>> > slow. Slow operations include pulling a single item from the Kafka
>> > queue, running a simple query against PostgresSQL, and running a Spark
>> > aggregation on a RDD with a handful of rows.
>> >
>> > The machine is not maxing out on memory, disk or CPU. The machine
>> > seems to be doing nothing for a high percentage of the execution time.
>> > We have reproduced this behavior on two other machines. So we're
>> > suspecting a configuration issue
>> >
>> > As a concrete example, we have a DataFrame produced by running a JDBC
>> > query by mapping over an RDD from Kafka. Calling count() (I guess
>> > forcing execution) on this DataFrame when there is *1* item/row (Note:
>> > SQL database is EMPTY at this point so this is not a factor) takes 4.5
>> > seconds, calling count when there are 10,000 items takes 7 seconds.
>> >
>> > Can anybody offer experience of something like this happening for
>> > them? Any suggestions on how to understand what is going wrong?
>> >
>> > I have tried tuning the number of Kafka partitions - increasing this
>> > seems to increase the concurrency and ultimately number of things
>> > processed per minute, but to get something half decent, I'm going to
>> > need running with 1024 or more partitions. Is 1024 partitions a
>> > reasonable number? What do you use in you environments?
>> >
>> > I've tried different options for batchDuration. The calculation seems
>> > to be batchDuration * Kafka partitions for number of items per
>> > iteration, but this is always still extremely slow (many per iteration
>> > vs. very few doesn't seem to really improve things). Can you suggest a
>> > list of the Spark configuration parameters related to speed that you
>> > think are key - preferably with the values you use for those
>> > parameters?
>> >
>> > I'd really really appreciate any help or suggestions as I've been
>> > working on this speed issue for 3 days without success and my head is
>> > starting to hurt. Thanks in advance.
>> >
>> >
>> >
>> > Thanks,
>> >
>> > --
>> >
>> > Malcolm Lockyer
>> >
>> > ---------------------------------------------------------------------
>> > To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> > For additional commands, e-mail: user-h...@spark.apache.org
>> >
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> For additional commands, e-mail: user-h...@spark.apache.org
>>
>

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

Reply via email to