Re: Leader election strategy

2022-11-15 Thread Pierre Coquentin
ops_automigrate > > Luke > > On Mon, Nov 14, 2022 at 3:55 PM Pierre Coquentin < > pierre.coquen...@gmail.com> > wrote: > > > Hello, > > We have a Kafka cluster (2.4.1) with a replication factor of 3. I notice > > when we stop a broker that only one broke

Leader election strategy

2022-11-13 Thread Pierre Coquentin
Hello, We have a Kafka cluster (2.4.1) with a replication factor of 3. I notice when we stop a broker that only one broker takes all the load from the missing broker and becomes the leader to all partitions. I would have thought that Kafka would split the load evenly among the remaining brokers. S

Re: How to be notified by Kafka stream during partitions rebalancing

2019-04-04 Thread Pierre Coquentin
t; Hope this helps. > > On Thu, Apr 4, 2019 at 10:52 AM Pierre Coquentin < > pierre.coquen...@gmail.com> > wrote: > > > Hi, > > > > We have a cache in a processor based on assigned partitions and when > Kafka > > revokes those partitions we would like to f

How to be notified by Kafka stream during partitions rebalancing

2019-04-04 Thread Pierre Coquentin
Hi, We have a cache in a processor based on assigned partitions and when Kafka revokes those partitions we would like to flush the cache. The method punctuate is neet to do that except that as a client of Kafka stream, I am not notified during a revoke. I found the same question on StackOverflow h

Re: How to know the begin offset of a partition

2018-07-31 Thread Pierre Coquentin
thanks, too bad :( On Tue, Jul 31, 2018 at 4:44 PM Gabriele Paggi wrote: > Hi Pierre, > > Indeed, it doesn't support SASL_SSL. > The issue is being worked on here: > https://issues.apache.org/jira/browse/KAFKA-5235 > > On Tue, Jul 31, 2018 at 2:19 PM Pierre C

Re: How to know the begin offset of a partition

2018-07-31 Thread Pierre Coquentin
er-list $(hostname -f):9092 --topic logs --time -2 > logs:8:268728 > logs:11:268544 > logs:2:269047 > logs:5:268855 > logs:4:268919 > logs:7:268781 > logs:1:269115 > logs:10:268636 > logs:9:268676 > logs:3:268962 > logs:6:268812 > logs:0:269176 > gpaggi@kafka

How to know the begin offset of a partition

2018-07-31 Thread Pierre Coquentin
Hi How do we get the starting offset of a partition from a topic when we have a retention policy configured? I've checked in shell script kafka-topic.sh or kafka-console-consumer.sh but found nothing so far. Do we have to implement our own solution to get this information? It could be nice in kafk

Re: Infinite loop trying to connect to coordinator

2017-08-27 Thread Pierre Coquentin
FYI, since the upgrade to 0.11 two weeks ago, the error never occurred again On Mon, Jul 17, 2017 at 9:28 AM, Pierre Coquentin < pierre.coquen...@gmail.com> wrote: > Is it possible than a max.poll.records too high can cause this instability? > > On Wed, Jul 12, 2017 at 8:43 AM, P

Re: Infinite loop trying to connect to coordinator

2017-07-17 Thread Pierre Coquentin
Is it possible than a max.poll.records too high can cause this instability? On Wed, Jul 12, 2017 at 8:43 AM, Pierre Coquentin < pierre.coquen...@gmail.com> wrote: > It was on our test environment and nothing was running when the incident > occurred. > In the server log we have a b

Re: Infinite loop trying to connect to coordinator

2017-07-11 Thread Pierre Coquentin
uses or possibly heavy load on your zookeeper cluster as evidenced by > any SocketTimeoutExceptions on the Kafka and/or Zookeeper sides? > > --John > > On Tue, Jul 11, 2017 at 6:15 AM, Pierre Coquentin < > pierre.coquen...@gmail.com> wrote: > > > Hi, > >

Infinite loop trying to connect to coordinator

2017-07-11 Thread Pierre Coquentin
Hi, We are using kafka 0.10.2 with 2 brokers and 2 application nodes composed of 6 consumers each (all in one group). And recently we experienced disconnection of both nodes simultaneously and an infinite retry to connect to the coordinator. Currently, just restarting the nodes solve the problem b

Re: Potential memory leak in rocksdb

2017-02-20 Thread Pierre Coquentin
unted drive. > > This way we notice that system is stable and in steady state and no > rebalance has occurred for a while now. Also per core wait time has > considerably reduced. > > See if this helps in controlling the rocksdb state. > > Thanks > Sachin > &g

Re: Potential memory leak in rocksdb

2017-02-14 Thread Pierre Coquentin
ok.com/groups/rocksdb.dev/permalink/1171716669593512/ > > > Guozhang > > On Mon, Feb 13, 2017 at 12:28 AM, Pierre Coquentin < > pierre.coquen...@gmail.com> wrote: > > > For the K, we use a simple StringSerde, for the V, we use a custom Serde > > which translates an avro

Re: Potential memory leak in rocksdb

2017-02-13 Thread Pierre Coquentin
if the serdes kept a reference to these bytes they may not be compacted > or garbage collected immediately. > > Guozhang > > On Fri, Feb 10, 2017 at 12:18 PM, Pierre Coquentin < > pierre.coquen...@gmail.com> wrote: > > > Hi Sachin, > > > > We have 6 co

Re: Potential memory leak in rocksdb

2017-02-10 Thread Pierre Coquentin
; snapshot. > > Thanks > Sachin > > > On Fri, Feb 10, 2017 at 1:25 PM, Pierre Coquentin < > pierre.coquen...@gmail.com> wrote: > > > Here is the gist with the two gif > > > > https://gist.github.com/PierreCoquentin/d2df46e5e1c0d3506f6311b343e6f775 >

Re: Potential memory leak in rocksdb

2017-02-09 Thread Pierre Coquentin
iles are not shown up. Could you try using a > gist link? > > Guozhang > > On Wed, Feb 8, 2017 at 9:49 AM, Pierre Coquentin < > pierre.coquen...@gmail.com > > wrote: > > > Well, I am a little perplexed now... I have already recompiled the branch > > 0.10.1 wi

Re: Potential memory leak in rocksdb

2017-02-08 Thread Pierre Coquentin
M, Damian Guy wrote: > > > Looks fine > > > > On Sat, 4 Feb 2017 at 19:27, Pierre Coquentin < > pierre.coquen...@gmail.com> > > wrote: > > > > Oh ok, this is a snippet of the code we use : > > > >

Re: Potential memory leak in rocksdb

2017-02-04 Thread Pierre Coquentin
4, 2017 at 7:59 PM, Damian Guy wrote: > Keeping the rocksdb iterator wouldn't cause a memory leak in the heap. That > is why i asked. > > On Sat, 4 Feb 2017 at 16:36 Pierre Coquentin > wrote: > > > The iterator is inside a try-with-resources. And if the memory leak was

Re: Potential memory leak in rocksdb

2017-02-04 Thread Pierre Coquentin
ote: > Hi Pierre, > > When you are iterating over the entries do you close the iterator once you > are finished? If you don't then that will cause a memory leak. > > Thanks, > Damian > > On Sat, 4 Feb 2017 at 16:18 Pierre Coquentin > wrote: > > > Hi,

Potential memory leak in rocksdb

2017-02-04 Thread Pierre Coquentin
Hi, We ran a few tests with apache kafka 0.10.1.1. We use a Topology with only one processor and a KVStore configured as persistent backed by rocksdb 4.9.0. Each events received are stored using the method put(key, value) and in the punctuate method, we iterate over all entries with all(), process

Re: JVM crash when closing persistent store (rocksDB)

2016-10-12 Thread Pierre Coquentin
Ok it works against the trunk and the branch 0.10.1, both have a dependency to rockdb 4.9.0 vs 4.4.1 for kafka 0.10.0. Do you know when 0.10.1 will be released ? On Tue, Oct 11, 2016 at 9:39 PM, Pierre Coquentin < pierre.coquen...@gmail.com> wrote: > Hi, > > I already tried to

Re: JVM crash when closing persistent store (rocksDB)

2016-10-11 Thread Pierre Coquentin
ty? I'm wondering if there is some bad state left over. > > Finally looks like you are running 0.10.0, could you try running trunk to > see if the problem still exists? I'm running trunk. I know there were a > couple of RocksDb fixes after 0.10.0 > > Thanks > Eno > &

JVM crash when closing persistent store (rocksDB)

2016-10-11 Thread Pierre Coquentin
Hi, I have a simple test where I create a topology builder with one topic, one processor using a persistent store, then I create a kafka streams, start it, wait a bit, then close. Each time, the jvm crashes (seg fault) when flushing the data. Anyone has already met this kind of problem ? OS: Ubun

Re: Synchronized block in StreamTask

2016-07-28 Thread Pierre Coquentin
016 at 7:09 AM, Pierre Coquentin < > pierre.coquen...@gmail.com> wrote: > > > Hi, > > > > I've a simple technical question about kafka streams. > > In class org.apache.kafka.streams.processor.internals.StreamTask, the > > method "process" uses

Synchronized block in StreamTask

2016-07-28 Thread Pierre Coquentin
Hi, I've a simple technical question about kafka streams. In class org.apache.kafka.streams.processor.internals.StreamTask, the method "process" uses a synchronized block but I don't see why, the method doesn't seem to be called in a multi-threaded environnement as it's created and only accessed b