[Kafka Consumer] deserializer (documentation) mismatch?

2018-10-15 Thread Bart Vercammen
Hi, I found a mismatch between the documentation in the org.apache.kafka.common.serialization.Deserializer and the implementation in KafkaConsumer. Deserializer documentation sais: *"serialized bytes; may be null; implementations are recommended to handle null by returning a value or null rather

Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-15 Thread Bart Vercammen
a known issue discovered in version 1.1 - > > > https://issues.apache.org/jira/browse/KAFKA-7144 > > > > > > This issue has been fixed in Kafka Streams 2.0, any chance you can > > upgrade > > > to 2.0? > > > > > > Thanks, > > > Bill >

Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-09 Thread Bart Vercammen
s, > Bill > > On Mon, Oct 8, 2018 at 2:46 PM Bart Vercammen wrote: > > > Thanks John, > > > > I'll see what I can do regarding the logs ... > > As a side not, our Kafka cluster is running version v1.1.1 in v0.10.2.1 > log > > format configuration (due

Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bart Vercammen
luck! > > If the repro doesn't turn out, maybe you could just extract the assignment > lines from your logs? > > Thanks, > -John > > On Mon, Oct 8, 2018 at 1:24 PM Bart Vercammen wrote: > > > Hi John, > > > > Zipping up some logs from our running K

Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bart Vercammen
sounds a bit surprising. Is there any chance you can zip up some logs > so we can see the assignment protocol on the nodes? > > Thanks, > -John > > On Mon, Oct 8, 2018 at 4:32 AM Bart Vercammen wrote: > > > Hi, > > > > I recently moved some KafkaStreams app

[KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bart Vercammen
Hi, I recently moved some KafkaStreams applications from v0.10.2.1 to v1.1.1 and now I notice a weird behaviour in the partition assignment. When starting 4 instances of my Kafka Streams application (on v1.1.1) I see that 17 of the 20 partitions (of a source topic) are assigned to 1 instance of th

Re: [kafka streams] discuss: dynamically update subscription pattern

2017-08-16 Thread Bart Vercammen
is a nice work-around, but still leaves me with my initial remark that it would be useful to somehow be able to alter the subscriptions in a running streams application. Bart On Tue, Aug 15, 2017 at 1:45 PM, Bart Vercammen wrote: > HI Guozhang, > > Thanks for your swift feedback. >

Re: [kafka streams] discuss: dynamically update subscription pattern

2017-08-15 Thread Bart Vercammen
wn an existing program piping from a topic. This will > admittedly introduce a duplicate topic containing the aggregated data, but > operational-wise may still be simpler. > > > Guozhang > > > On Mon, Aug 14, 2017 at 2:47 AM, Bart Vercammen wrote: > > > Hi Guozhang, >

Re: [kafka streams] discuss: dynamically update subscription pattern

2017-08-14 Thread Bart Vercammen
way > to do that is to make PartitionAssignor customizable by users as you > suggested, so that only the selected partitions are used to re-form the > tasks; but one still need some way to trigger a rebalance so that the > PartitionAssignor can be called. > > > Guozhang > > O

[kafka streams] discuss: dynamically update subscription pattern

2017-08-11 Thread Bart Vercammen
Hi, I have a question basically on how it would be the best way to implement something within Kafka Streams. The thing I would like to do: "dynamically update the subscription pattern of the source topics. The reasoning behind this (in my project): meta data about the source topics is evented on

Re: [kafka streams] 'null' values in state stores

2017-08-09 Thread Bart Vercammen
ithub.com/apache/kafka/blob/0.11.0/streams/src/ > main/java/org/apache/kafka/streams/state/internals/ > InMemoryKeyValueStore.java#L79 > > But, even then it is just putting `null` when it should be deleting it. > Feel free to raise a JIRA > Thanks, > Damian > > On Tue, 8 Aug

Re: [kafka streams] 'null' values in state stores

2017-08-08 Thread Bart Vercammen
wrote: > Hi, > The null values are treated as deletes when they are written to the store. > You can see here: > https://github.com/apache/kafka/blob/0.11.0/streams/src/ > main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L261 > > On Tue, 8 Aug 2017

[kafka streams] 'null' values in state stores

2017-08-08 Thread Bart Vercammen
Hi, I noticed the following: When a kafka streams application starts, it will restore its state in its state-stores (from the log-compacted kafka topic). All good so far, but I noticed that the 'deleted' entries are actually read in into the store as 'key' with value:`null` Is this expected beha

Re: Kafka 0.10.1.0 consumer group rebalance broken?

2016-11-29 Thread Bart Vercammen
d for thought. > > – > Best regards, > Radek Gruchalski > ra...@gruchalski.com > > > On November 28, 2016 at 9:04:16 PM, Bart Vercammen (b...@cloutrix.com) > wrote: > > Hi, > > It seems that consumer group rebalance is broken in Kafka 0.10.1.0 ? > When running a smal

Kafka 0.10.1.0 consumer group rebalance broken?

2016-11-28 Thread Bart Vercammen
Hi, It seems that consumer group rebalance is broken in Kafka 0.10.1.0 ? When running a small test-project : - consumers running in own JVM (with different 'client.id') - producer running in own JVM - kafka broker : the embedded kafka : KafkaServerStartable It looks like the consumers loose their