q(
> "org.apache.spark" % "spark-streaming_2.10" % "1.5.2",
> "org.apache.spark" % "spark-streaming-kafka_2.10" % "1.5.2"
> )
>
>
>
> Regards,
> ~Vinti
>
> On Wed, Feb 24, 2016 at 9:33 AM, Cody Koeninger
&
spark streaming is provided, kafka is not.
This build file
https://github.com/koeninger/kafka-exactly-once/blob/master/build.sbt
includes some hacks for ivy issues that may no longer be strictly
necessary, but try that build and see if it works for you.
On Wed, Feb 24, 2016 at 11:14 AM, Vinti
That's correct, when you create a direct stream, you specify the
topicpartitions you want to be a part of the stream (the other method for
creating a direct stream is just a convenience wrapper).
On Wed, Feb 24, 2016 at 2:15 AM, 陈宇航 wrote:
> Here I use the *'KafkaUtils.createDirectStream'* to in
The direct stream will let you do both of those things. Is there a reason
you want to use receivers?
http://spark.apache.org/docs/latest/streaming-kafka-integration.html
http://spark.apache.org/docs/latest/configuration.html#spark-streaming
look for maxRatePerPartition
On Mon, Feb 22, 2016 at
I saw this slide:
http://image.slidesharecdn.com/east2016v2matei-160217154412/95/2016-spark-summit-east-keynote-matei-zaharia-5-638.jpg?cb=1455724433
Didn't see the talk - was this just referring to the existing work on the
spark-streaming-kafka subproject, or is someone actually working on making
be resolved by having a smaller block interval?
>
> Regards,
> Praveen
> On 18 Feb 2016 21:30, "praveen S" wrote:
>
>> Can having a smaller block interval only resolve this?
>>
>> Regards,
>> Praveen
>> On 18 Feb 2016 21:13, "Cody Koe
Backpressure won't help you with the first batch, you'd need
spark.streaming.kafka.maxRatePerPartition
for that
On Thu, Feb 18, 2016 at 9:40 AM, praveen S wrote:
> Have a look at
>
> spark.streaming.backpressure.enabled
> Property
>
> Regards,
> Praveen
> On 18 Feb 2016 00:13, "Abhishek Anand"
You can print whatever you want wherever you want, it's just a question of
whether it's going to show up on the driver or the various executors logs
On Wed, Feb 17, 2016 at 5:50 AM, Cyril Scetbon
wrote:
> I don't think we can print an integer value in a spark streaming process
> As opposed to a
parkContext ctx = new JavaSparkContext(rdd.context());
> JavaRDD currrentBatchOffsets = ctx.parallelize(r);
> //write currrentBatchOffsets rdd to cassandra
> return null;
> });
>
>
> Is this the correct way of doing this ?
>
>
> Thanks !!
> Abhi
>
> On Tue, Feb 1
Just use a kafka rdd in a batch job or two, then start your streaming job.
On Wed, Feb 17, 2016 at 12:57 AM, Abhishek Anand
wrote:
> I have a spark streaming application running in production. I am trying to
> find a solution for a particular use case when my application has a
> downtime of say
You could use sc.parallelize... but the offsets are already available at
the driver, and they're a (hopefully) small enough amount of data that's
it's probably more straightforward to just use the normal cassandra client
to save them from the driver.
On Tue, Feb 16, 2016 at 1:15 AM, Abhishek Anand
Please don't change the behavior of DirectKafkaInputDStream.
Returning an empty rdd is (imho) the semantically correct thing to do, and
some existing jobs depend on that behavior.
If it's really an issue for you, you can either override
directkafkainputdstream, or just check isEmpty as the first t
sync.time.ms", "250");
> props.put("auto.commit.interval.ms", "1000");
>
>
> How can I do the same for the receiver inside spark-streaming for Spark V1.3.1
>
>
> Thanks
>
> Nipun
>
>
>
> On Wed, Feb 10, 2016 at 3:59 PM Cody Ko
The real way to fix this is by changing partitioning, so you don't have a
hot partition. It would be better to do this at the time you're producing
messages, but you can also do it with a shuffle / repartition during
consuming.
There is a setting to allow another batch to start in parallel, but t
It's a pair because there's a key and value for each message.
If you just want a single topic, put a single topic in the map of topic ->
number of partitions.
See
https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java
On W
[
https://issues.apache.org/jira/browse/SPARK-7827?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15139252#comment-15139252
]
Cody Koeninger commented on SPARK-7827:
---
If there is no more information avail
[
https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15139250#comment-15139250
]
Cody Koeninger commented on SPARK-3146:
---
I think this can be safely closed, g
[
https://issues.apache.org/jira/browse/SPARK-13106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15139242#comment-15139242
]
Cody Koeninger commented on SPARK-13106:
https://issues.apache.org/jira/br
gt; sparkstreaming web UI . Also I am counting the messages inside
>> foreachRDD .
>> Removed the settings for backpressure but still the same .
>>
>>
>>
>>
>>
>> Sent from Samsung Mobile.
>>
>>
>> Original message -
ounting the messages inside
> foreachRDD .
> Removed the settings for backpressure but still the same .
>
>
>
>
>
> Sent from Samsung Mobile.
>
>
> ---- Original message
> From: Cody Koeninger
> Date:06/02/2016 00:33 (GMT+05:30)
> To: Diwakar
tePerPartition=100" --driver-memory 2g
> --executor-memory 1g --class com.tcs.dime.spark.SparkReceiver --files
> /etc/hadoop/conf/core-site.xml,/etc/hadoop/conf/hdfs-site.xml,/etc/hadoop/conf/mapred-site.xml,/etc/hadoop/conf/yarn-site.xml,/etc/hive/conf/hive-site.xml
> --jars
>
2 things:
- you're only attempting to read from a single TopicAndPartition. Since
your topic has multiple partitions, this probably isn't what you want
- you're getting an offset out of range exception because the offset you're
asking for doesn't exist in kafka.
Use the other createDirectStream
If you're using the direct stream, you have 0 receivers. Do you mean you
have 1 executor?
Can you post the relevant call to createDirectStream from your code, as
well as any relevant spark configuration?
On Thu, Feb 4, 2016 at 8:13 PM, Diwakar Dhanuskodi <
diwakar.dhanusk...@gmail.com> wrote:
>
On Wed, Jan 27, 2016 at 11:52 PM, Cody Koeninger
> wrote:
>
>> Have you tried spark 1.5?
>>
>> On Wed, Jan 27, 2016 at 11:14 AM, vimal dinakaran
>> wrote:
>>
>>> Hi ,
>>> I am using spark 1.4 with direct kafka api . In my streaming ui , I
KafkaRDD will use the standard kafka configuration parameter
refresh.leader.backoff.ms if it is set in the kafkaParams map passed to
createDirectStream.
On Tue, Feb 2, 2016 at 9:10 PM, Chen Song wrote:
> For Kafka direct stream, is there a way to set the time between successive
> retries? From m
0.9 brokers should be backwards compatible with 0.8 simple consumer, which
is what the direct stream uses. If you're finding any problems, please
report them.
There's work underway to add 0.9 consumer support to spark, at
https://issues.apache.org/jira/browse/SPARK-12177
On Wed, Feb 3, 2016 at
[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15128377#comment-15128377
]
Cody Koeninger commented on SPARK-12177:
It's probably worth either wa
It's possible you could (ab)use updateStateByKey or mapWithState for this.
But honestly it's probably a lot more straightforward to just choose a
reasonable batch size that gets you a reasonable file size for most of your
keys, then use filecrush or something similar to deal with the hdfs small
fi
[
https://issues.apache.org/jira/browse/SPARK-13125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15128358#comment-15128358
]
Cody Koeninger commented on SPARK-13125:
This doesn't make sense.
That indicates a problem in network communication between the executor and
the kafka broker. Have you done any network troubleshooting?
On Mon, Feb 1, 2016 at 9:59 AM, SRK wrote:
> Hi,
>
> I see the following error in Spark Streaming with Kafka Direct. I think
> that
> this error is related t
[
https://issues.apache.org/jira/browse/SPARK-13106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15126369#comment-15126369
]
Cody Koeninger commented on SPARK-13106:
My original intention was for the k
The kafka direct stream doesn't do any explicit caching. I haven't looked
through the underlying simple consumer code in the kafka project in detail,
but I doubt it does either.
Honestly, I'd recommend not using auto created topics (it makes it too easy
to pollute your topics if someone fat-finge
Have you tried spark 1.5?
On Wed, Jan 27, 2016 at 11:14 AM, vimal dinakaran
wrote:
> Hi ,
> I am using spark 1.4 with direct kafka api . In my streaming ui , I am
> able to see the events listed in UI only if add stream.print() statements
> or else event rate and input events remains in 0 event
Where are you calling checkpointing? Metadata checkpointing for a kafa
direct stream should just be the offsets, not the data.
TD can better speak to reduceByKeyAndWindow behavior when restoring from a
checkpoint, but ultimately the only available choices would be replay the
prior window data from
Should be socket.timeout.ms on the map of kafka config parameters. The
lack of retry is probably due to the differences between running spark in
local mode vs standalone / mesos / yarn.
On Mon, Jan 25, 2016 at 1:19 PM, Supreeth wrote:
> We are running a Kafka Consumer in local mode using Spar
Yes, you should query Kafka if you want to know the latest available
offsets.
There's code to make this straightforward in KafkaCluster.scala, but the
interface isnt public. There's an outstanding pull request to expose the
api at
https://issues.apache.org/jira/browse/SPARK-10963
but frankly it
Offsets are stored in the checkpoint. If you want to manage offsets
yourself, don't restart from the checkpoint, specify the starting offsets
when you create the stream.
Have you read / watched the materials linked from
https://github.com/koeninger/kafka-exactly-once
Regarding the small files p
[
https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15111398#comment-15111398
]
Cody Koeninger commented on SPARK-11045:
There's already work being do
Looks like this response did go to the list.
As far as OffsetOutOfRange goes, right now that's an unrecoverable error,
because it breaks the underlying invariants (e.g. that the number of
messages in a partition is deterministic once the RDD is defined)
If you want to do some hacking for your own
If you can share an isolated example I'll take a look. Not something I've
run into before.
On Wed, Jan 20, 2016 at 3:53 PM, jpocalan wrote:
> Hi,
>
> I have an application which creates a Kafka Direct Stream from 1 topic
> having 5 partitions.
> As a result each batch is composed of an RDD havi
6 kafka partitions will result in 6 spark partitions, not 6 spark rdds.
The question of whether you will have a backlog isn't just a matter of
having 1 executor per partition. If a single executor can process all of
the partitions fast enough to complete a batch in under the required time,
you wo
You can't really use spark batches as the basis for any kind of reliable
time aggregation. Time of batch processing in general has nothing to do
with time of event.
You need to filter / aggregate by the time interval you care about, in your
own code, or use a data store that can do the aggregatio
Reading commands from kafka and triggering a redshift copy is sufficiently
simple it could just be a bash script. But if you've already got a spark
streaming job set up, may as well use it for consistency's sake. There's
definitely no need to mess around with akka.
On Fri, Jan 15, 2016 at 6:25 P
my use case) until the following code is executed
> stream.transform { rdd =>
> val wrapped = YourWrapper(cp, rdd)
> wrapped.join(reference)
> }
> In which case it will run through the partitioner of the wrapped RDD when
> it arrives in the cluster for the first time i.e. no sh
[
https://issues.apache.org/jira/browse/SPARK-12775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15096550#comment-15096550
]
Cody Koeninger commented on SPARK-12775:
Couldn't find leade
ere the data for each
> partition resides in the cluster.
>
> Thanks,
> Dave.
>
>
> On 13/01/16 16:21, Cody Koeninger wrote:
>
> If two rdds have an identical partitioner, joining should not involve a
> shuffle.
>
> You should be able to override the partitione
If two rdds have an identical partitioner, joining should not involve a
shuffle.
You should be able to override the partitioner without calling partitionBy.
Two ways I can think of to do this:
- subclass or modify the direct stream and kafkardd. They're private, so
you'd need to rebuild just the
eam method. Now
> here I want to get the exact offset that is being picked up by the
> createDirectStream method at the begining of the batch. I need this to
> create an initialRDD.
>
> Please let me know if anything is unclear.
>
> Thanks !!!
>
>
> On Mon, Jan 11, 2016
I'm not 100% sure what you're asking.
If you're asking if it's possible to start a stream at a particular set of
offsets, yes, one of the createDirectStream methods takes a map from
topicpartition to starting offset.
If you're asking if it's possible to query Kafka for the offset
corresponding to
[
https://issues.apache.org/jira/browse/SPARK-12693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15089306#comment-15089306
]
Cody Koeninger commented on SPARK-12693:
If an executor is given a rang
[
https://issues.apache.org/jira/browse/SPARK-12693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15088725#comment-15088725
]
Cody Koeninger commented on SPARK-12693:
What is your actual use case
te events in this queue of RDD.
>
> I'm not sure to be right but with your solution I'll get the last events
> by id for each RDD in the DStream. What I want is to process the following
> event that is the last events for an id for the DStream's RDD that are
> accumul
nt
> to process last events by each user id once ie skip intermediate events by
> user id.
> I have only one Kafka topic with all theses events.
>
> Regards,
>
> Julien Naour
>
> Le mer. 6 janv. 2016 à 16:13, Cody Koeninger a
> écrit :
>
>> Have you read
>
Have you read
http://kafka.apache.org/documentation.html#compaction
On Wed, Jan 6, 2016 at 8:52 AM, Julien Naour wrote:
> Context: Process data coming from Kafka and send back results to Kafka.
>
> Issue: Each events could take several seconds to process (Work in progress
> to improve that).
[
https://issues.apache.org/jira/browse/SPARK-10963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15076536#comment-15076536
]
Cody Koeninger commented on SPARK-10963:
There's a nonzero chance tha
er doing this, i realized that since the signatures of
> createDirectStream & createRDD (Decoder's removed or replaced) change,
> these can be added without conflict to KafkaUtils and remain separate from
> existing implementation (older kafka API).
>
> thanks
> Mario
>
&g
Have you seen
SPARK-12177
On Wed, Dec 23, 2015 at 3:27 PM, eugene miretsky
wrote:
> Hi,
>
> The Kafka connector currently uses the older Kafka Scala consumer. Kafka
> 0.9 came out with a new Java Kafka consumer.
>
> One of the main differences is that the Scala consumer uses
> a Decoder( kafka.s
[
https://issues.apache.org/jira/browse/SPARK-12524?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15071884#comment-15071884
]
Cody Koeninger commented on SPARK-12524:
mapPartitions is just giving yo
[
https://issues.apache.org/jira/browse/SPARK-12524?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15071872#comment-15071872
]
Cody Koeninger commented on SPARK-12524:
Why can't you just use mapP
Read the documentation
spark.apache.org/docs/latest/streaming-kafka-integration.html
If you still have questions, read the resources linked from
https://github.com/koeninger/kafka-exactly-once
On Wed, Dec 23, 2015 at 7:24 AM, Vyacheslav Yanuk
wrote:
> Colleagues
> Documents written about create
[
https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15069101#comment-15069101
]
Cody Koeninger commented on SPARK-11045:
Direct stream isn't doing an
gt;
> Thanks!
>
> On Mon, Dec 21, 2015 at 8:05 AM, Cody Koeninger
> wrote:
>
>> 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
Compared to what alternatives?
Honestly, if someone actually read the kafka docs, yet is still having
trouble getting a single test node up and running, the problem is probably
them. Kafka's docs are pretty good.
On Mon, Dec 21, 2015 at 11:31 AM, Andy Davidson <
a...@santacruzintegration.com> wr
Honestly it's a lot easier to deal with this using transactions.
Someone else would have to speak to the possibility of getting task
failures added to listener callbacks.
On Sat, Dec 19, 2015 at 5:44 PM, Neelesh wrote:
> Hi,
> I'm trying to build automatic Kafka watermark handling in my strea
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 idling, and just one task is still ru
op ..., I rely on kafka offset
> for Incremental data am I right ? so no duplicate data will be returned.
>
>
> Thanks
> Sri
>
>
>
>
>
> On Fri, Dec 18, 2015 at 2:41 PM, Cody Koeninger
> wrote:
>
>> If you're really doing a daily batch job, have you
If you're really doing a daily batch job, have you considered just using
KafkaUtils.createRDD rather than a streaming job?
On Fri, Dec 18, 2015 at 5:04 AM, kali.tumm...@gmail.com <
kali.tumm...@gmail.com> wrote:
> Hi All,
>
> Imagine I have a Production spark streaming kafka (direct connection)
>
ining them would
> be really efficient, unless you know something that I don't.
>
> As I really don't need any interaction between those streams, I think I
> might end up running 3 different streaming apps instead of one.
>
> Thanks again!
>
> On Thu,
[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15062313#comment-15062313
]
Cody Koeninger commented on SPARK-12177:
Honestly SSL / auth is the
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.
Kafka provides buffering, ordering, decoupling of producers from multiple
consumers. So pretty much any time you have requirements for asynchronous
process, fault tolerance, and/or a common view of the order of events
across multiple consumers kafka is worth a look.
Spark provides a much richer l
I'm a little confused as to why you have fake events rather than just doing
foreachRDD or foreachPartition on your kafka stream and updating the
accumulator there. I'd expect that to run each batch even if the batch had
0 kafka messages in it.
On Thu, Dec 10, 2015 at 2:05 PM, AliGouta wrote:
>
[
https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15048938#comment-15048938
]
Cody Koeninger commented on SPARK-9059:
---
To be clear, I think SPARK-8389 sh
he fix
> version is?
>
> On Wed, Dec 9, 2015 at 11:10 AM Cody Koeninger wrote:
>
>> Which version of spark are you on? I thought that was added to the spark
>> UI in recent versions.
>>
>> DIrect api doesn't have any inherent interaction with zookeeper. If
[
https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15048904#comment-15048904
]
Cody Koeninger edited comment on SPARK-9059 at 12/9/15 4:1
[
https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15048904#comment-15048904
]
Cody Koeninger commented on SPARK-9059:
---
HasOffsetRanges is explained here
[
https://issues.apache.org/jira/browse/SPARK-6051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15048895#comment-15048895
]
Cody Koeninger commented on SPARK-6051:
---
Responded on the mailing list, but
Which version of spark are you on? I thought that was added to the spark
UI in recent versions.
DIrect api doesn't have any inherent interaction with zookeeper. If you
need number of messages per batch and aren't on a recent enough version of
spark to see them in the ui, you can get them program
[
https://issues.apache.org/jira/browse/SPARK-12203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15048879#comment-15048879
]
Cody Koeninger commented on SPARK-12203:
Commented on the PR. I don
[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15046975#comment-15046975
]
Cody Koeninger commented on SPARK-12177:
I really think this needs to be han
Just to be clear, spark checkpoints have nothing to do with zookeeper,
they're stored in the filesystem you specify.
On Sun, Dec 6, 2015 at 1:25 AM, manasdebashiskar
wrote:
> When you enable check pointing your offsets get written in zookeeper. If
> you
> program dies or shutdowns and later rest
Personally, for jobs that I care about I store offsets in transactional
storage rather than checkpoints, which eliminates that problem (just
enforce whatever constraints you want when storing offsets).
Regarding the question of communication of errors back to the
streamingListener, there is an onR
> I agree and i was more thinking maybe there is a way to support both for a
> period of time (of course means some more code to maintain :-)).
>
>
> thanks
> Mario
>
> [image: Inactive hide details for Cody Koeninger ---04/12/2015 12:15:55
> am---Honestly my feeling on
s.scala#L395-L423
>
> Thanks,
> - Alan
>
> On Tue, Dec 1, 2015 at 8:12 AM, Cody Koeninger wrote:
>
>> I actually haven't tried that, since I tend to do the offset lookups if
>> necessary.
>>
>> It's possible that it will work, try it and let me k
king I can probably just do some if/else logic and use the basic
> createDirectStream and the more advanced
> createDirectStream(...fromOffsets...) if the offsets for my topic name
> exists in the database. Any reason that wouldn't work? If I don't include
> an offset range for a particular part
Honestly my feeling on any new API is to wait for a point release before
taking it seriously :)
Auth and encryption seem like the only compelling reason to move, but
forcing people on kafka 8.x to upgrade their brokers is questionable.
On Thu, Dec 3, 2015 at 11:30 AM, Mario Ds Briggs
wrote:
> H
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Cody Koeninger reopened SPARK-12103:
PR for doc change:
https://github.com/apache/spark/pull/10132
> KafkaUtils createStream w
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15038084#comment-15038084
]
Cody Koeninger commented on SPARK-12103:
On the off chance you're actin
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15037986#comment-15037986
]
Cody Koeninger commented on SPARK-12103:
Knowing that kafka messages have a
There's a 1:1 relationship between Kafka partitions and Spark partitions.
Have you read
https://github.com/koeninger/kafka-exactly-once/blob/master/blogpost.md
A direct stream job will use up to spark.executor.cores number of cores.
If you have fewer partitions than cores, there probably won't be
period or increase consumer rate..or do both ..
>
> What I am trying to say, streaming job should not fail in any cases ..
>
> Dibyendu
>
> On Thu, Dec 3, 2015 at 9:40 AM, Cody Koeninger wrote:
>
>> I believe that what differentiates reliable systems is individual
>&g
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15037773#comment-15037773
]
Cody Koeninger commented on SPARK-12103:
A cursory review of the Kafka pro
.
>
> Dibyendu
>
> On Thu, Dec 3, 2015 at 8:26 AM, Cody Koeninger wrote:
>
>> No, silently restarting from the earliest offset in the case of offset
>> out of range exceptions during a streaming job is not the "correct way of
>> recovery".
>>
>>
tach...@gmail.com> wrote:
> This consumer which I mentioned does not silently throw away data. If
> offset out of range it start for earliest offset and that is correct way of
> recovery from this error.
>
> Dibyendu
> On Dec 2, 2015 9:56 PM, "Cody Koeninger" wrote:
&
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.
ose we are OK
>> with at least once processing and use receiver based approach which uses
>> ZooKeeper but not query Kafka directly, would these errors(Couldn't find
>> leader offsets for
>> Set([test_stream,5])))be avoided?
>>
>> On Tue, Dec 1, 2015 at 3
Catch that exception and somehow force things to "reset" for that
> partition And how would it handle the offsets already calculated in the
> backlog (if there is one)?
>
> On Tue, Dec 1, 2015 at 6:51 AM, Cody Koeninger wrote:
>
>> If you're consistently gett
e using -1 for one
of the offsets.
On Tue, Dec 1, 2015 at 9:58 AM, Alan Braithwaite
wrote:
> Neat, thanks. If I specify something like -1 as the offset, will it
> consume from the latest offset or do I have to instrument that manually?
>
> - Alan
>
> On Tue, Dec 1, 2015
If you're consistently getting offset out of range exceptions, it's
probably because messages are getting deleted before you've processed them.
The only real way to deal with this is give kafka more retention, consume
faster, or both.
If you're just looking for a quick "fix" for an infrequent iss
Yes, there is a version of createDirectStream that lets you specify
fromOffsets: Map[TopicAndPartition, Long]
On Mon, Nov 30, 2015 at 7:43 PM, Alan Braithwaite
wrote:
> Is there any mechanism in the kafka streaming source to specify the exact
> partition id that we want a streaming job to consum
801 - 900 of 1425 matches
Mail list logo