There is no free lunch. (TD, when do we get shirts made that say that?)
If you want exactly-once delivery semantics for arbitrary workflows into
arbitrary datastores, you're going to have to do some of your own work.
If someone is telling you otherwise, they're probably lying to you.
I think wr
Thanks Cody for your answers. This discussion helped me a lot.
Though, I still feel that offsets management could be better handled by
Spark, if it really wants to be easy streaming framework. If it won't help
users to do so, I'm affraid it will be superseded for many by other
frameworks that might
Hi,
Just to clarify one point which may not be clear to many. If someone
decides to use Receiver based approach , the best options at this point is
to use https://github.com/dibbhatt/kafka-spark-consumer. This will also
work with WAL like any other receiver based consumer. The major issue with
K
You have to store offsets somewhere.
If you're going to store them in checkpoints, then you have to deal with
the fact that checkpoints aren't recoverable on code change. Starting up
the new version helps because you don't start it from the same checkpoint
directory as the running one... it has y
Thanks guys for your answers. I put my answers in text, below.
Cheers,
Krzysztof Zarzycki
2015-09-10 15:39 GMT+02:00 Cody Koeninger :
> The kafka direct stream meets those requirements. You don't need
> checkpointing for exactly-once. Indeed, unless your output operations are
> idempotent, you
The kafka direct stream meets those requirements. You don't need
checkpointing for exactly-once. Indeed, unless your output operations are
idempotent, you can't get exactly-once if you're relying on checkpointing.
Instead, you need to store the offsets atomically in the same transaction
as your r
>> checkpoints can't be used between controlled restarts
Is that true? If so, why? From my testing, checkpoints appear to be working
fine, we get the data we've missed between the time the consumer went down
and the time we brought it back up.
>> If I cannot make checkpoints between code upgrades
Hi,
This is being running in Production in many organization who has adopted
this consumer as an alternative option. The Consumer will run with spark
1.3.1 .
This is being running in Pearson for sometime in production.
This is part of spark packages and you can see how to include it in your
mvn
Thanks Akhil, seems like an interesting option to consider.
Do you know if the package is production-ready? Do you use it in production?
And do you know if it works for Spark 1.3.1 as well? README mentions that
package in spark-packages.org is built with Spark 1.4.1.
Anyway, it seems that core
This consumer pretty much covers all those scenarios you listed
github.com/dibbhatt/kafka-spark-consumer Give it a try.
Thanks
Best Regards
On Thu, Sep 10, 2015 at 3:32 PM, Krzysztof Zarzycki
wrote:
> Hi there,
> I have a problem with fulfilling all my needs when using Spark Streaming
> on Kafk
10 matches
Mail list logo