Re: Is Kafka documentation regarding null key misleading?

2014-12-05 Thread Michal Michalski
Yes, it is *very* misleading in my opinion - I've seen so many people surprised with that behaviour... Technically it's 100% correct of course: "If the key is null, then the Producer will assign the message to a random Partition." - that's what actually happens, because assignment is random. Howeve

Re: Resetting Offsets

2015-02-18 Thread Michal Michalski
See https://cwiki.apache.org/confluence/display/KAFKA/System+Tools and check the following: GetOffsetShell (not very accurate - will set your offsets to much smaller values than you really need; we log offsets frequently in application logs and get it from there) ImportZkOffsets Kind regards, Mich

The idea of "composite key" to make log compaction more flexible - question / proposal

2017-10-02 Thread Michal Michalski
Hi, TL;DR: I'd love to be able to make log compaction more "granular" than just per-partition-key, so I was thinking about the concept of a "composite key", where partitioning logic is using one part of the key, while compaction uses the whole key - is this something desirable / doable / worth a K

Re: Kafka - leadership

2017-10-04 Thread Michal Michalski
Hi Tomer, 1. Yes, one broker can (will) be a leader for multiple partitions - you'll usually have a relatively small number of brokers in your cluster, but they will store data for many topics, with many partitions, so (assuming auto.leader.rebalance.enable is set to true; I think it's thedefault

Re: Topics and Partitions

2017-10-05 Thread Michal Michalski
Hi Josh, 1. I don't know for sure (haven't seen the code that does it), but it's probably the most "even" split possible for given number of brokers and partitions. So for 8 partitions and 3 brokers it would be [3, 3, 2]. 2. See "num.partitions" in broker config. BTW. only producer can create topi

Re: Topics and Partitions

2017-10-06 Thread Michal Michalski
sumer starts on a non-existent topic, it throws an error? > > Kind regards > Meeraj > > On Thu, Oct 5, 2017 at 9:20 PM, Josh Maidana > wrote: > > > Thank you, Michal. > > > > That answers all my questions, many thanks. > > > > Josh > > > &

Re: The idea of "composite key" to make log compaction more flexible - question / proposal

2017-10-09 Thread Michal Michalski
custom partitioner, no? > > https://kafka.apache.org/0110/javadoc/org/apache/kafka/ > clients/producer/Partitioner.html > > -Jay > > On Mon, Oct 2, 2017 at 6:29 AM Michal Michalski < > michal.michal...@zalando.ie> > wrote: > > > Hi, > > > >

Re: Consumer service that supports retry with exponential backoff

2017-10-10 Thread Michal Michalski
Hi John, It doesn't seem like you care for the ordering (since you're using multiple "fallback" topics that are processed in parallel if I understood you correctly), but the alternative would be to implement the backoff using the same topic and consumer. We're using the "pausing" feature of the co

Re: kafka broker loosing offsets?

2017-10-11 Thread Michal Michalski
Hi Dmitriy, I didn't follow the whole thread, but if it's not an issue with Kafka 0.11.0.0 (there was another thread about it recently), make sure your Replication Factor for the offsets topic is 3 (you mentioned "RF=3 for all topics", but I wasn't sure it includes the offsets one). There was a b

Re: Number of Topics and Partitions in recent Kafka?

2017-10-14 Thread Michal Michalski
Hi Sven, Normally I'd say that it mainly depends on your latency and throughput requirements, so starting from https://www.confluent.io/blog/how-to-choose-the-number-of-topicspartitions-in-a-kafka-cluster/ is probably a good idea. As a rule of the thumb I'd say that for a "typical" cloud deploymen

Re: Kafka Monitoring

2017-12-06 Thread Michal Michalski
Hi Irtiza, We're using Jolokia and we had no problems with it. It would be useful to know what exactly you did (how you "plugged in" Jolokia, how you configured it, what endpoint are you querying etc.) to help you. On 6 December 2017 at 10:36, Irtiza Ali wrote: > Hello everyone, > > I am workin

Re: Kafka Monitoring

2017-12-08 Thread Michal Michalski
Can you send me link to tutorial or provide some resources for the Jolokia > configuration with kafka> > > > Thank you > Irtiza > > On Wed, Dec 6, 2017 at 8:00 PM, Michal Michalski < > michal.michal...@zalando.ie> wrote: > > > Hi Irtiza, > > > > W

Re: Kafka Monitoring

2017-12-08 Thread Michal Michalski
I have look what have you done. Because I have > done same thing that you explained above but when i do this < > http://localhost:/jolokia/list> i get only metrics for the zookeeper > but not the above metrics. > > How are you using Jolokia for monitoring kafka cluster?? > &g

Re: Kafka Monitoring

2017-12-12 Thread Michal Michalski
ontroller:* > :8074/jolokia/read/kafka.log:* > :8074/jolokia/read/kafka.network:* > > > Can you check what I have done wrong? > > Thanks > > With Regards > Irtiza > > On Fri, Dec 8, 2017 at 8:19 PM, Michal Michalski < > michal.michal...@zalando.ie>

Re: Fixing Wrecked Kafka Topic Partitions after a cluster reconfigure

2018-02-01 Thread Michal Michalski
Hey, Changing broker IDs made existing data in ZK irrelevant - since ZK uses broker ID to identify brokers and "associate" them with the partition placement (where the data is for each partition), it expects your data to be on the "old" brokers, while the ones that are connecting now are completel

Re: Hardware Guidance

2018-03-01 Thread Michal Michalski
I'm quite sure it's per broker (it's a standard way to provide recommendation on node sizes in systems like Kafka), but you should definitely read it in the context of the data size and traffic the cluster has to handle. I didn't read the presentation, so not sure if it contains such information (i

Re: Prioritized Topics for Kafka

2019-01-17 Thread Michal Michalski
Hi, This sounds like a great idea, and thanks for reaching out for feedback. Here are two use cases I've worked on that I'd seriously consider using such feature for: 1. Priority Republish of Data - in an event driven system, there's a "republish" functionality used for e.g. fixing data affected

Re: Consumer offset is getting reset back to some old value automatically

2014-06-25 Thread Michal Michalski
My understanding is that "bringing down 1 node our of a 3 node zookeeper cluster is risky, since any subsequent leader election *might* not reach a quorum"and "It is less likely but still risky to some extent" - *"it might not reach a quorum"*, because you need both of the remaining nodes to be up

Re: Rolling update Kafka 0.8 -> Kafka 0.8.1.1 in detail

2014-06-30 Thread Michal Michalski
Hi Yury, If I understand correctly, the case you're describing is equivalent to the leader re-election (in terms of data consistency). In that case messages can be lost depending on your "acks" setting: https://kafka.apache.org/documentation.html see: request.required.acks: E.g. "only messages th

Re: status of 0.8.2

2014-07-08 Thread Michal Michalski
One more question regarding 0.8.2 - is it planned to be a in-place, no-downtime release (I'm using 0.8.0 now)? By looking at the version number changes only I'd guess it is, but... ;-) Michal Kind regards, MichaƂ Michalski, michal.michal...@boxever.com On 8 July 2014 18:22, Joel Koshy wrote:

Consumer rebalancing retry settings and reconnecting after failure

2014-07-10 Thread Michal Michalski
Hi, Just wondering - is there any reason why rebalance.max.retries is 4 by default? Is there any good reason why I shouldn't expect my consumers to keep trying to rebalance for minutes (e.g. 30 retries every 6 seconds), rather than seconds (4 retries every 2 seconds by default)? Also, if my consu

Re: Consumer rebalancing retry settings and reconnecting after failure

2014-07-11 Thread Michal Michalski
uot;hide" some issues in the systems that actually > caused the rebalance failure. The general design is that if some > exception/errors are not expected like the rebalance failures we will let > it to possibly hault/kill the instance rather than automatically restart > and let it

Re: Consumer rebalancing retry settings and reconnecting after failure

2014-07-14 Thread Michal Michalski
latency jitters. > > Guozhang > > > On Fri, Jul 11, 2014 at 5:25 AM, Michal Michalski < > michal.michal...@boxever.com> wrote: > > > Hey Guozhang, > > > > Thanks for reply. I get your point on "hiding" some issues, but I'd > prefer > > t

Re: request.required.acks=-1 under high data volume

2014-07-22 Thread Michal Michalski
> Interesting, I had missed that. Is it worth updating the documentation to > make that more explicit, or do other people find it clear enough? I agree, it's easy to miss, we missed that information too - we noticed it only few days ago, while we've been using Kafka for weeks and we spent long hou