Different spark-submit per topic.
On Mon, Dec 21, 2015 at 11:36 AM, Neelesh wrote:
> Thanks Cody. My case is #2. Just wanted to confirm when you say different
> spark jobs, do you mean one spark-submit per topic, or just use different
> threads in the driver to submit the job?
>
> Thanks!
>
>
Thanks Cody. My case is #2. Just wanted to confirm when you say different
spark jobs, do you mean one spark-submit per topic, or just use different
threads in the driver to submit the job?
Thanks!
On Mon, Dec 21, 2015 at 8:05 AM, Cody Koeninger wrote:
> Spark streaming by default wont start t
Spark streaming by default wont start the next batch until the current
batch is completely done, even if only a few cores are still working. This
is generally a good thing, otherwise you'd have weird ordering issues.
Each topicpartition is separate.
Unbalanced partitions can happen either becaus
@Chris,
There is a 1-1 mapping b/w spark partitions & kafka partitions out of
the box . One can break it by repartitioning of course and add more
parallelism, but that has its own issues around consumer offset management-
when do I commit the offsets, for example. While its trivial to increase
separating out your code into separate streaming jobs - especially when there
are no dependencies between the jobs - is almost always the best route. it's
easier to combine atoms (fusion), then split them (fission).
I recommend splitting out jobs along batch window, stream window, and
state-tr
A related issue - When I put multiple topics in a single stream, the
processing delay is as bad as the slowest task in the number of tasks
created. Even though the topics are unrelated to each other, RDD at time
"t1" has to wait for the RDD at "t0" is fully executed, even if most
cores are idlin
You could stick them all in a single stream, and do mapPartitions, then
switch on the topic for that partition. It's probably cleaner to do
separate jobs, just depends on how you want to organize your code.
On Thu, Dec 17, 2015 at 11:11 AM, Jean-Pierre OCALAN
wrote:
> Hi Cody,
>
> First of all
Hi Cody,
First of all thanks for the note about spark.streaming.concurrentJobs. I
guess this is why it's not mentioned in the actual spark streaming doc.
Since those 3 topics contain completely different data on which I need to
apply different kind of transformations, I am not sure joining them wo
Using spark.streaming.concurrentJobs for this probably isn't a good idea,
as it allows the next batch to start processing before current one is
finished, which may have unintended consequences.
Why can't you use a single stream with all the topics you care about, or
multiple streams if you're e.g.
Nevermind, I found the answer to my questions.
The following spark configuration property will allow you to process
multiple KafkaDirectStream in parallel:
--conf spark.streaming.concurrentJobs=
--
View this message in context:
http://apache-spark-user-list.1001560.n3.nabble.com/Kafka-streami
Yeah, that general plan should work, but might be a little awkward for
adding topicPartitions after the fact (i.e. when you have stored offsets
for some, but not all, of your topicpartitions)
Personally I just query kafka for the starting offsets if they dont exist
in the DB, using the methods in
Hey Cody, I'm convinced that I'm not going to get the functionality I want
without using the Direct Stream API.
I'm now looking through
https://github.com/koeninger/kafka-exactly-once/blob/master/blogpost.md#exactly-once-using-transactional-writes
where you say "For the very first time the job is
After digging into the Kafka code some more (specifically
kafka.consumer.KafkaStream, kafka.consumer.ConsumerIterator and
kafka.message.MessageAndMetadata), it appears that the Left value of the
tuple is not the topic name but rather a key that Kafka puts on each
message. See http://kafka.apache.or
Sigh... I want to use the direct stream and have recently brought in Redis
to persist the offsets, but I really like and need to have realtime metrics
on the GUI, so I'm hoping to have Direct and Receiver stream both working.
On Wed, Dec 2, 2015 at 3:17 PM Cody Koeninger wrote:
> Use the direct
Use the direct stream. You can put multiple topics in a single stream, and
differentiate them on a per-partition basis using the offset range.
On Wed, Dec 2, 2015 at 2:13 PM, dutrow wrote:
> I found the JIRA ticket: https://issues.apache.org/jira/browse/SPARK-2388
>
> It was marked as invalid.
I found the JIRA ticket: https://issues.apache.org/jira/browse/SPARK-2388
It was marked as invalid.
--
View this message in context:
http://apache-spark-user-list.1001560.n3.nabble.com/Kafka-streaming-from-multiple-topics-tp8678p25550.html
Sent from the Apache Spark User List mailing list arch
My need is similar; I have 10+ topics and don't want to dedicate 10 cores to
processing all of them. Like yourself and others, the (String, String) pair
that comes out of the DStream has (null, StringData...) values instead of
(topic name, StringData...)
Did anyone ever find a way around this issu
Can you link to the JIRA issue? I'm having to work around this bug and it
would be nice to monitor the JIRA so I can change my code when it's fixed.
--
View this message in context:
http://apache-spark-user-list.1001560.n3.nabble.com/Kafka-streaming-from-multiple-topics-tp8678p12053.html
Sent
ursday, July 3, 2014 at 9:41 PM
To: "user@spark.apache.org<mailto:user@spark.apache.org>"
mailto:user@spark.apache.org>>
Subject: Re: Kafka - streaming from multiple topics
Sergey,
On Fri, Jul 4, 2014 at 1:06 AM, Sergey Malov
mailto:sma...@collective.com>> wrote:
On t
Sergey,
On Fri, Jul 4, 2014 at 1:06 AM, Sergey Malov wrote:
>
> On the other hand, under the hood KafkaInputDStream which is create with
> this KafkaUtils call, calls ConsumerConnector.createMessageStream which
> returns a Map[String, List[KafkaStream] keyed by topic. It is, however, not
> expo
...@preferred.jp>>
Reply-To: "user@spark.apache.org<mailto:user@spark.apache.org>"
mailto:user@spark.apache.org>>
Date: Wednesday, July 2, 2014 at 9:54 PM
To: "user@spark.apache.org<mailto:user@spark.apache.org>"
mailto:user@spark.apache.org>>
S
Sergey,
you might actually consider using two streams, like
* val stream1 = KafkaUtils.createStream(ssc,"localhost:2181","logs",
Map("retarget" -> 2))*
* val stream2 = KafkaUtils.createStream(ssc,"localhost:2181","logs",
Map("datapair" -> 2))*
to achieve what you want. This has the addit
22 matches
Mail list logo