[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15128377#comment-15128377
]
Cody Koeninger commented on SPARK-12177:
It's probably worth either waiting for a point release
[
https://issues.apache.org/jira/browse/SPARK-13125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15128358#comment-15128358
]
Cody Koeninger commented on SPARK-13125:
This doesn't make sense.
You can either shuffle
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
>
[
https://issues.apache.org/jira/browse/SPARK-13106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15126369#comment-15126369
]
Cody Koeninger commented on SPARK-13106:
My original intention was for the kafka offset api
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
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
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
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
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
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
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
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
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
[
https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15111398#comment-15111398
]
Cody Koeninger commented on SPARK-11045:
There's already work being done on 0.9
https
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
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
; 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 shuf
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
llow Spark to know where 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
[
https://issues.apache.org/jira/browse/SPARK-12775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15096550#comment-15096550
]
Cody Koeninger commented on SPARK-12775:
Couldn't find leader offsets ... because you couldn't
fsets using createDirectStream 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 !!!
>
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
[
https://issues.apache.org/jira/browse/SPARK-12693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15089306#comment-15089306
]
Cody Koeninger commented on SPARK-12693:
If an executor is given a range of kakfa offsets
[
https://issues.apache.org/jira/browse/SPARK-12693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15088725#comment-15088725
]
Cody Koeninger commented on SPARK-12693:
What is your actual use case for changing the log
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
kind of user id. I want
> 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 <c...@koeninger.org>
rds,
>
> Julien
>
> Le mer. 6 janv. 2016 à 17:35, Cody Koeninger <c...@koeninger.org> a
> écrit :
>
>> if you don't have hot users, you can use the user id as the hash key for
>> publishing into kafka.
>> That will put all events for a given user in the sa
[
https://issues.apache.org/jira/browse/SPARK-10963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15076536#comment-15076536
]
Cody Koeninger commented on SPARK-10963:
There's a nonzero chance that the 0.9 integration
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
[
https://issues.apache.org/jira/browse/SPARK-12524?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15071872#comment-15071872
]
Cody Koeninger commented on SPARK-12524:
Why can't you just use mapPartitions?
> Group by
[
https://issues.apache.org/jira/browse/SPARK-12524?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15071884#comment-15071884
]
Cody Koeninger commented on SPARK-12524:
mapPartitions is just giving you an iterator, what you
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
>
[
https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15069101#comment-15069101
]
Cody Koeninger commented on SPARK-11045:
Direct stream isn't doing any caching unless you
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
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
river to submit the job?
>
> Thanks!
>
> On Mon, Dec 21, 2015 at 8:05 AM, Cody Koeninger <c...@koeninger.org>
> 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.
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>
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)
., 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 <c...@koeninger.org>
> wrote:
>
>> If you're really doing a daily bat
[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15062313#comment-15062313
]
Cody Koeninger commented on SPARK-12177:
Honestly SSL / auth is the only compelling thing about
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
am not sure joining 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
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
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
ue. Any idea what the fix
> version is?
>
> On Wed, Dec 9, 2015 at 11:10 AM Cody Koeninger <c...@koeninger.org> 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 an
[
https://issues.apache.org/jira/browse/SPARK-12203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048879#comment-15048879
]
Cody Koeninger commented on SPARK-12203:
Commented on the PR. I don't think this makes sense
[
https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048904#comment-15048904
]
Cody Koeninger edited comment on SPARK-9059 at 12/9/15 4:19 PM
[
https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048938#comment-15048938
]
Cody Koeninger commented on SPARK-9059:
---
To be clear, I think SPARK-8389 should make it possible
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
[
https://issues.apache.org/jira/browse/SPARK-6051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048895#comment-15048895
]
Cody Koeninger commented on SPARK-6051:
---
Responded on the mailing list, but for posterity's sake
[
https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048904#comment-15048904
]
Cody Koeninger commented on SPARK-9059:
---
HasOffsetRanges is explained here
http://spark.apache.org
[
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15046975#comment-15046975
]
Cody Koeninger commented on SPARK-12177:
I really think this needs to be handled as a separate
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
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
rs is questionable.
> <<
>
> 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:5
spark/streaming/kafka/KafkaUtils.scala#L395-L423
>
> Thanks,
> - Alan
>
> On Tue, Dec 1, 2015 at 8:12 AM, Cody Koeninger <c...@koeninger.org> wrote:
>
>> I actually haven't tried that, since I tend to do the offset lookups if
>> necessary.
>>
>> It's
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15037773#comment-15037773
]
Cody Koeninger commented on SPARK-12103:
A cursory review of the Kafka project documentation
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15038084#comment-15038084
]
Cody Koeninger commented on SPARK-12103:
On the off chance you're acting in good faith, actually
[
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 createStr
[
https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15037986#comment-15037986
]
Cody Koeninger commented on SPARK-12103:
Knowing that kafka messages have a key and value isn't
ease 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 <c...@koeninger.org> wrote:
>
>> I believe that what differentiates reliable system
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
ably 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 partit
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
e 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:40 PM, Cod
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
;
> Dibyendu
>
> On Thu, Dec 3, 2015 at 8:26 AM, Cody Koeninger <c...@koeninger.org> 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&quo
ach...@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" <c...@koeninger.or
requirements.
>
>
> 2.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 <c...@koeninger.org
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
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
of the offsets.
On Tue, Dec 1, 2015 at 9:58 AM, Alan Braithwaite <a...@cloudflare.com>
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 at
If you had exactly 1 message in the 0th topicpartition, to read it you
would use
OffsetRange("topicname", 0, 0, 1)
Kafka's simple shell consumer in that case would print
next offset = 1
So instead trying to consume
OffsetRange("topicname", 0, 1, 2)
shouldn't be expected to work
On Sat,
Can you post the relevant code?
On Fri, Nov 27, 2015 at 4:25 AM, u...@moosheimer.com
wrote:
> Hi,
>
> we have some strange behavior with KafkaUtils DirectStream and the size of
> the MapPartitionsRDDs.
>
> We use a permanent direct steam where we consume about 8.500 json
>
Starting from the checkpoint using getOrCreate should be sufficient if all
you need is at-least-once semantics
http://spark.apache.org/docs/latest/streaming-programming-guide.html#checkpointing
On Mon, Nov 30, 2015 at 9:38 AM, Guillermo Ortiz
wrote:
> Hello,
>
> I have
>
> On Thu, Nov 12, 2015 at 9:07 AM, Cody Koeninger <c...@koeninger.org>
> wrote:
>
>> To be blunt, if you care about being able to recover from weird
>> situations, you should be tracking offsets yourself and specifying offsets
>> on job start, not relying on
[
https://issues.apache.org/jira/browse/SPARK-12002?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15029058#comment-15029058
]
Cody Koeninger commented on SPARK-12002:
Just to be clear, this is a only an issue with python
Spark's direct stream kafka integration should take advantage of data
locality if you're running Spark executors on the same nodes as Kafka
brokers.
On Wed, Nov 25, 2015 at 9:50 AM, Dave Ariens wrote:
> I just finished reading up on Kafka Connect<
>
The direct stream shouldn't silently lose data in the case of a leader
loss. Loss of a leader is handled like any other failure, retrying
up to spark.task.maxFailures
times.
But really if you're losing leaders and taking that long to rebalance you
should figure out what's wrong with your
am partition 88 start
> 221563725. This should not happen, and indicates that messages may have
> been
> lost
>
> On Tue, Nov 24, 2015 at 6:31 AM, Cody Koeninger <c...@koeninger.org>
> wrote:
>
>> No, the direct stream only communicates with Kafka brokers, not Zookeeper
>>
What exactly do you mean by kafka consumer reporting?
I'd log the offsets in your spark job and try running
kafka-simple-consumer-shell.sh --partition $yourbadpartition --print-offsets
at the same time your spark job is running
On Mon, Nov 23, 2015 at 7:37 PM, swetha
If you really want to just not process the bad topicpartitions, you can use
the version of createDirectStream that takes
fromOffsets: Map[TopicAndPartition, Long]
and exclude the broken topicpartitions from the map.
On Mon, Nov 23, 2015 at 4:54 PM, Hudong Wang wrote:
>
No, that means that at the time the batch was scheduled, the kafka leader
reported the ending offset was 221572238, but during processing, kafka
stopped returning messages before reaching that ending offset.
That probably means something got screwed up with Kafka - e.g. you lost a
leader and lost
rtitioned after mapToPair
> function. It would be great if you could brieftly explain ( or send me some
> document, i couldnt find it) about how shuffle work on mapToPair function.
>
> Thank you very much.
> On Nov 23, 2015 12:26 AM, "Cody Koeninger" <c...@koening
Spark specific questions are better directed to the Spark user list.
Spark will retry failed tasks automatically up to a configurable number of
times. The direct stream will retry failures on the driver up to a
configurable number of times.
See
Also, if you actually want to use kafka, you're much better off with a
replication factor greater than 1, so you get leader re-election.
On Fri, Nov 20, 2015 at 9:20 AM, Cody Koeninger <c...@koeninger.org> wrote:
> Spark specific questions are better directed to the Spark user list.
You're confused about which parts of your code are running on the driver vs
the executor, which is why you're getting serialization errors.
Read
http://spark.apache.org/docs/latest/streaming-programming-guide.html#design-patterns-for-using-foreachrdd
On Fri, Nov 20, 2015 at 1:07 PM, Saiph
Not sure what you mean by "no documentation regarding ways to achieve
effective communication between the 2", but the docs on integrating with
kafka are at
http://spark.apache.org/docs/latest/streaming-kafka-integration.html
As far as custom partitioners go, Learning Spark from O'Reilly has a
Sure, just call count on each rdd and track it in your driver however you
want.
If count is called directly on a kafkardd (e.g. createDirectStream, then
foreachRDD before doing any other transformations), it should just be using
the beginning and ending offsets rather than doing any real work.
Are you using the direct stream? Each batch should contain all of the
unprocessed messages for each topic, unless you're doing some kind of rate
limiting.
On Tue, Nov 17, 2015 at 3:07 AM, Antony Mayi
wrote:
> Hi,
>
> I have two streams coming from two different
There are already private methods in the code for interacting with Kafka's
offset management api.
There's a jira for making those methods public, but TD has been reluctant
to merge it
https://issues.apache.org/jira/browse/SPARK-10963
I think adding any ZK specific behavior to spark is a bad
Ordering would be on a per-partition basis, not global ordering.
You typically want to acquire resources inside the foreachpartition
closure, just before handling the iterator.
http://spark.apache.org/docs/latest/streaming-programming-guide.html#design-patterns-for-using-foreachrdd
On Mon, Nov
int does not have rdd stored for beyond 2 hrs which is my window
> duration. Because of this my job takes more time than usual.
>
> Is there a way or some configuration parameter which would help avoid
> repartitioning twice ?
>
> I am attaching the snapshot for the same.
>
>
[
https://issues.apache.org/jira/browse/SPARK-11693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15004072#comment-15004072
]
Cody Koeninger commented on SPARK-11693:
You've under-provisioned Kafka storage and / or Spark
Unless you change maxRatePerPartition, a batch is going to contain all of
the offsets from the last known processed to the highest available.
Offsets are not time-based, and Kafka's time-based api currently has very
poor granularity (it's based on filesystem timestamp of the log segment).
There's
[
https://issues.apache.org/jira/browse/SPARK-11698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15004050#comment-15004050
]
Cody Koeninger commented on SPARK-11698:
That looks like a reasonable way to solve your
[
https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15004052#comment-15004052
]
Cody Koeninger commented on SPARK-10320:
This is for practical purposes blocked on SPARK-10963
int folder to help the job
> recover? E.g. Go 2 steps back, hoping that kafka has those offsets.
>
> -adrian
>
> From: swetha kasireddy
> Date: Monday, November 9, 2015 at 10:40 PM
> To: Cody Koeninger
> Cc: "user@spark.apache.org"
> Subject: Re: Kafka Direct does
The direct stream will fail the task if there is a problem with the kafka
broker. Spark will retry failed tasks automatically, which should handle
broker rebalances that happen in a timely fashion. spark.tax.maxFailures
controls the maximum number of retries before failing the job. Direct
stream
t; for monitoring like every 5 minutes and then send an email alert and
> automatically restart the Streaming job by deleting the Checkpoint
> directory. Would that help?
>
>
>
> Thanks!
>
> On Mon, Nov 9, 2015 at 11:09 AM, Cody Koeninger <c...@koeninger.org>
> wrote:
801 - 900 of 1347 matches
Mail list logo