Re: Kafka streams state store return hostname as unavailable when calling queryMetadataForKey method

2024-05-13 Thread Sophie Blee-Goldman
Ah. Well this isn't anything new then since it's been the case since 2.6,
but the default task assignor in Kafka Streams will sometimes assign
partitions unevenly for a time if it's trying to move around stateful tasks
and there's no copy of that task's state on the local disk attached to the
KafkaStreams instance it's trying to move that task to. This imbalance
should only be temporary however, and it should converge on an even
distribution of partitions over time as it finishes "warming up" the task
state in the background and can finish moving those stateful tasks to their
final destination.

An upgrade can sometimes trigger a large-scale redistribution of tasks,
which in turn can lead to a lot of these "warmup tasks" and a longer
duration of uneven task assignment. But it should always level out
eventually if the group is stable. So when you say "we've observed that the
state store of Kafka Streams instances is not evenly distributed as it was
before the upgrade" was this just referring to immediately after the
upgrade? If so, give it some time and it should trend towards an even
distribution. If it seems to be stuck in an uneven state,  then that can
either be because (a) there's a bug in the assignor, or more likely (b) the
group itself is unstable and the assignment can't converge.

Given this issue is accompanied by the "hostname unavailable", it sounds
like the group is stuck rebalancing. Do you monitor rebalances in any way?
If you're seeing them about every 10 minutes exactly, then it's most likely
just the "probing rebalances" that happen while tasks are being warmed up
via the process described above. But if the application is rebalancing
repeatedly, nonstop, or over a very long period of time (many hours/days),
then that would be a problem.

So I guess my first question for you would be, has it settled down any
since the upgrade? If you have very large state stores then the "warming
up" can take a long time, even on the order of an hour or two. But
definitely not days. There are some configs you can tweak if this is the
case.

Second question would be whether it's been rebalancing the whole time, or
only every 10 minutes. If you don't monitor this already, there are a few
ways to tell. One would be setting up a state listener via the
KafkaStreams#setStateListener API, which has a REBALANCING state.
Unfortunately this isn't always enough to go on since the REBALANCING state
actually includes both literal rebalancing and also task restoration. It's
still useful to know, especially when paired with a metric that helps
differentiate between actual rebalancing vs task restoration. One such
metric I personally always look at is the consumer's
last-rebalance-seconds-ago, which basically represents how long it's been
since a rebalance occurred. This metric can always instantly identify
probing rebalances/warmup tasks by the sawtooth pattern with an amplitude
of 10 min, corresponding to the regular 10 minute probing rebalances.

Hope this helps,
Sophie

On Thu, May 9, 2024 at 9:20 PM Penumarthi Durga Prasad Chowdary <
prasad.penumar...@gmail.com> wrote:

> Kafka upgraded from 3.5.1 to 3.7.0 version
>
> On Fri, May 10, 2024 at 2:13 AM Sophie Blee-Goldman  >
> wrote:
>
> > What version did you upgrade from?
> >
> > On Wed, May 8, 2024 at 10:32 PM Penumarthi Durga Prasad Chowdary <
> > prasad.penumar...@gmail.com> wrote:
> >
> > > Hi Team,
> > >   I'm utilizing Kafka Streams to handle data from Kafka topics, running
> > > multiple instances with the same application ID. This enables
> distributed
> > > processing of Kafka data across these instances. Furthermore, I've
> > > implemented state stores with time windows and session windows. To
> > retrieve
> > > windows efficiently, I've established a remote query mechanism between
> > > Kafka Streams instances. By leveraging the queryMetadataForKey method
> on
> > > streams, I can retrieve the hostname where a specific key was processed
> > and
> > > where the corresponding window data resides in the state store.
> > >  *streams.queryMetadataForKey(storeName, recordKey, new
> > > DataKeySerilizer()).activeHost();*
> > > This functionality has been running smoothly for quite some time, up
> > until
> > > we upgraded our Kafka and Kafka Streams versions to 3.7.0. Since the
> > > upgrade, we've noticed some unexpected behavior that we didn't
> encounter
> > > with the previous versions.
> > >
> > >- The queryMetadataForKey method is returning "unavailable" as the
> > >hostname, despite having two Kafka Streams instances in a running
> > state.
> > >The issue seems to persist intermittently, and rest

Re: Kafka streams state store return hostname as unavailable when calling queryMetadataForKey method

2024-05-09 Thread Sophie Blee-Goldman
What version did you upgrade from?

On Wed, May 8, 2024 at 10:32 PM Penumarthi Durga Prasad Chowdary <
prasad.penumar...@gmail.com> wrote:

> Hi Team,
>   I'm utilizing Kafka Streams to handle data from Kafka topics, running
> multiple instances with the same application ID. This enables distributed
> processing of Kafka data across these instances. Furthermore, I've
> implemented state stores with time windows and session windows. To retrieve
> windows efficiently, I've established a remote query mechanism between
> Kafka Streams instances. By leveraging the queryMetadataForKey method on
> streams, I can retrieve the hostname where a specific key was processed and
> where the corresponding window data resides in the state store.
>  *streams.queryMetadataForKey(storeName, recordKey, new
> DataKeySerilizer()).activeHost();*
> This functionality has been running smoothly for quite some time, up until
> we upgraded our Kafka and Kafka Streams versions to 3.7.0. Since the
> upgrade, we've noticed some unexpected behavior that we didn't encounter
> with the previous versions.
>
>- The queryMetadataForKey method is returning "unavailable" as the
>hostname, despite having two Kafka Streams instances in a running state.
>The issue seems to persist intermittently, and restarting the Kafka
> Streams
>instances temporarily resolves it. However, the problem resurfaces after
>some time.
>- Additionally, we've observed that the state store of Kafka Streams
>instances is not evenly distributed as it was before the upgrade.
>Previously, if a Kafka topic had 10 partitions and two Kafka Streams
>instances were running, these 10 partitions would be evenly shared
> between
>the two instances. However, this behavior seems to have changed after
> the
>upgrade.
>
> When can this issue happen?
> How can I fix the issue ?
> I would like to express my gratitude in advance for any assistance
> provided.
> --
>
>
>
> Thank's's,
> Prasad,
> 91-9030546248.
>


Re: Kafka Stream App Rolling Restarts - Too Many Rebalances Per Partition

2024-05-06 Thread Sophie Blee-Goldman
Hey,

Just skimming the config list, there are two things that immediately jumped
out at me:

1. The default session timeout was bumped up to 45 seconds a little while
ago. Not sure if you're overriding this or just using an older version, but
I definitely recommend bumping this up to 45s. Especially in combination
with...
2. The internal.leave.group.on.close should always be set to "false" by
Kafka Streams. Are you overriding this? If so, that definitely explains a
lot of the rebalances. This config is basically like an internal backdoor
used by Kafka Streams to do exactly what it sounds like you want to do --
avoid triggering a rebalance when closing the consumer/KafkaStreams. It
also works in combination with the session timeout, and basically means
"don't kick off an extra rebalance if a bounced consumer rejoins within the
session timeout".

I'd start with that and see how it goes before fiddling with other things,
like the probing.rebalance.interval and max.warmup.replicas, since that'll
have implications/tradeoffs you may not want.

Lastly: I know this is somewhat contrary to common sense, but with consumer
groups/Kafka Streams it can often be much better to bounce as many nodes as
you can at once, rather than doing a true rolling bounce. If for any reason
you can't bounce multiple nodes at once, at the very least you should make
sure they are bounced as quickly as possible, ie minimize the time between
when one node comes back up and the next one is bounced. Often people will
wait for each node to come online, rejoin the consumer group, and fully
stabilize before bouncing the next node. But that means every single bounce
will not just necesitate a rebalance, but also guarantees that partitions
will be shuffled around the entire time. So my main piece of advice
(besides fixing the two configs above) is: do the rolling restart as fast
as you can!

On Mon, May 6, 2024 at 7:02 AM Nagendra Mahesh (namahesh)
 wrote:

> Hi,
>
>
> We have multiple replicas of an application running on a kubernetes
> cluster. Each application instance runs a stateful kafka stream application
> with an in-memory state-store (backed by a changelog topic). All instances
> of the stream apps are members of the same consumer group.
>
>
> Deployments happen using the “rolling restart” method i.e. new replica(s)
> come up successfully, and existing (old) replica(s) are killed. Due to
> members joining the consumer group (new app instances) and members leaving
> the consumer group (old app instances), there is rebalancing of topic
> partitions within the group.
>
>
> Ultimately, when all instances of the app have completed rolling restart,
> we see partitions have undergone rebalancing an excessive number of times.
> For example, the app has 48 instances and it is observed that each
> partition (say, partition #50) has undergone rebalancing a lot of times (50
> - 57 times) by moving across several app instances. Total count of
> partition movements during the entire rolling restart is greater than 3000.
>
>
> This excessive rebalancing incurs an overall lag on message processing
> SLAs, and is creating reliability issues.
>
>
> So, we are wondering:
>
>
> (1) is this expected, especially since cooperative rebalancing should
> ensure that not a lot of partitions get rebalanced
>
>
> (2) why would any partition undergo so many rebalances across several app
> instances?
>
>
> (3) is there some configuration (broker config or client config) that we
> can apply to reduce the total rebalances and partition movements during
> rolling restarts? We cannot consider static membership due to other
> technical constraints.
>
>
> The runtime and network is extremely stable — no heartbeat misses, session
> timeouts etc.
>
>
> DETAILS
>
> ---
>
>   *   Kafka Broker Version = 2.6
>
>   *   Kafka Streams Client Version = 2.7.0
>
>   *   No. of app instances = 48
>
>   *   No. of stream threads per stream app = 3
>
>   *   Total partition count = 60
>
>   *   Warmup Replicas (max.warmup.replicas) = 5
>
>   *   Standby Replicas (num.standby.replicas) = 2
>
>   *   probing.rebalance.interval.ms) = 30 (5 minutes)
>
>   *   session.timeout.ms = 1 (10 seconds)
>
>   *   heartbeat.interval.ms = 3000 (3 seconds)
>
>   *   internal.leave.group.on.close = true
>
>   *   linger.ms = 5
>
>   *   processing.guarantee = at_least_once
>
>
> Any help or information would be greatly appreciated.
>
> Thanks,
> Nagendra U M
>


Re: Is the KafkaStreams#store() method thread-safe?

2024-01-02 Thread Sophie Blee-Goldman
I'll give you two answers here: theoretical and practical.

First up, theoretically there is unfortunately no way to work around this
issue, even if you restrict your access to the KafkaStreams#store API to
the thread that created the KafkaStreams. This is because we currently have
write locking on this map, but no read locking, and the thread that created
the KafkaStreams object (and any user threads under your application's
control) would be a reader with unprotected access to the map, whereas the
"writers" here are internal StreamThreads that can modify this map at any
time. So in theory, you can hit a ConcurrentModificationException any time
a reader thread attempts to iterate that map while a REPLACE_THREAD event
causes a StreamThread to modify it in the background.

It's actually a bit worse than it seems, because the actual *access* to the
map isn't even confined to the KafkaStreams#store call -- all that does is
pass a view of the storeProvider map's values() along to the
WrappingStoreProvider class, which then iterates over the storeProviders on
each query to the store you got back from KafkaStreams#store (in other
words, the store you get from this API is technically a composite store
that wraps a collection of the literal StateStore instances used by the
application topology). All that is to say, you actually don't have any
thread safety issues with the KafkaStreams#store call itself, but might hit
this ConcurrentModificationException any time you try to use it.

Now for my second answer...despite all that, I think you should just go for
it, and use the API as if this were thread safe. It should be exceedingly
rare to run into this race condition, and you can protect yourself from it
completely by catching any ConcurrentModificationExceptions and retrying
the query. Even if you do get one, it should be transient and the store
query should succeed on the retry. Frankly, if it doesn't succeed on the
first retry then you have bigger problems since it probably indicates your
stream threads are failing continuously.

Hope this helps!

On Wed, Dec 27, 2023 at 2:52 PM Kohei Nozaki  wrote:

> Hi Sophie, thank you so much for sharing that. It all makes sense to me.
>
> Unfortunately my application uses REPLACE_THREAD, so it seems like I need
> a workaround for this until this thread unsafeness is removed. As I raised
> in my first email, would sharing only the ReadOnlyWindowStore instance with
> other threads be a workaround for this? Would the store object here be able
> to capture the changes that would be made by rebalancing?
>
> I've filed a ticket here (I'm interested in submitting a patch, but I
> cannot make any commitment):
> https://issues.apache.org/jira/browse/KAFKA-16055
>
> Regards,
> Kohei
>
>
> > On Dec 27, 2023, at 5:43, Sophie Blee-Goldman 
> wrote:
> >
> > Hey Kohei,
> >
> > Good question -- I don't think there's exactly a short answer to this
> > seemingly simple question so bear with me for a second.
> >
> > My understanding is that KafkaStreams#store is very much intended to be
> > thread-safe, and would have been back when it was first added a long time
> > ago, and the javadocs should probably be updated to reflect that.
> >
> > That said, you are totally right that whatever the intention, it is
> > technically not completely thread safe anymore since the storeProviders
> map
> > can be mutated when threads are added or removed. Of course, as long as
> you
> > are not adding or removing StreamThreads in your application, it should
> be
> > effectively thread-safe (but note: this includes using the REPLACE_THREAD
> > option with the StreamsUncaughtExceptionHandler)
> >
> > We should go ahead and fix this of course. I'm pretty sure we can just
> > change the HashMap to a ConcurrentHashMap and be done with it -- there's
> > already locking around the actual map modifications with the
> > "changeThreadCount" lock in KafkaStreams, so we just need to make sure we
> > don't accidentally hit a ConcurrentModificationException by accessing
> this
> > map while it's being modified.
> >
> > Would you mind submitting a JIRA ticket
> > <https://issues.apache.org/jira/projects/KAFKA/issues> for this bug you
> > found? And would you be interested in submitting a patch yourself?
> >
> > Thanks!
> > Sophie
> >
> > On Fri, Dec 22, 2023 at 6:55 PM Kohei Nozaki  wrote:
> >
> >> Hello, I have Kafka Streams questions related to thread safety.
> >>
> >> In my Kafka Streams application, I have 2 threads below:
> >>
> >> * Thread A: this creates a Topology object including state stores and
> >> everything and then eventually calls the constructor of the KafkaStreams

Re: Is the KafkaStreams#store() method thread-safe?

2023-12-26 Thread Sophie Blee-Goldman
Hey Kohei,

Good question -- I don't think there's exactly a short answer to this
seemingly simple question so bear with me for a second.

My understanding is that KafkaStreams#store is very much intended to be
thread-safe, and would have been back when it was first added a long time
ago, and the javadocs should probably be updated to reflect that.

That said, you are totally right that whatever the intention, it is
technically not completely thread safe anymore since the storeProviders map
can be mutated when threads are added or removed. Of course, as long as you
are not adding or removing StreamThreads in your application, it should be
effectively thread-safe (but note: this includes using the REPLACE_THREAD
option with the StreamsUncaughtExceptionHandler)

We should go ahead and fix this of course. I'm pretty sure we can just
change the HashMap to a ConcurrentHashMap and be done with it -- there's
already locking around the actual map modifications with the
"changeThreadCount" lock in KafkaStreams, so we just need to make sure we
don't accidentally hit a ConcurrentModificationException by accessing this
map while it's being modified.

Would you mind submitting a JIRA ticket
 for this bug you
found? And would you be interested in submitting a patch yourself?

Thanks!
Sophie

On Fri, Dec 22, 2023 at 6:55 PM Kohei Nozaki  wrote:

> Hello, I have Kafka Streams questions related to thread safety.
>
> In my Kafka Streams application, I have 2 threads below:
>
> * Thread A: this creates a Topology object including state stores and
> everything and then eventually calls the constructor of the KafkaStreams
> class and the start() method.
>
> * Thread B: this has a reference to the KafkaStreams object the thread A
> created. This periodically calls KafkaStreams#store on the object, gets a
> ReadOnlyWindowStore instance and reads the data in the store for monitoring
> purposes.
>
> I'm wondering if what my app does is ok in terms of thread safeness. I'm
> not so worried about ReadOnlyWindowStore because the javadoc says:
> “Implementations should be thread-safe as concurrent reads and writes are
> expected.”
>
> But as for KafkaStreams#store, I'm not so sure if it is ok to call from
> separate threads. One thing which concerns me is that it touches a HashMap,
> which is not thread safe here
> https://github.com/apache/kafka/blob/3.6.1/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java#L39
> . A KafkaStreams#removeStreamThread() call can mutate this HashMap object.
> Given that, I'm not so sure if this is designed to be thread-safe.
>
> My questions here: is it ok to call KafkaStreams#store from a thread which
> is different from the one which instantiated the KafkaStreams object? Or
> would that be better to call the store() method in the same thread and
> share only the ReadOnlyWindowStore instance with other threads? If that was
> better, would the store object be able to capture the changes that would be
> made by rebalancing? Is the KafkaStream class designed to be thread-safe at
> all?
>
> Regards,
> Kohei
>


Re: GlobalKTable with RocksDB - queries before state RUNNING?

2023-11-26 Thread Sophie Blee-Goldman
Ah, yeah, IQ v2 was a pretty big feature so it hasn't yet been implemented
across all parts of Kafka Streams. You'll notice that we're still actively
putting out new KIPs trying to complete this feature. I don't think there's
any particular reason that Global KTables can't be made to work with IQ v2,
but no one's gotten around to doing it yet. I did a quick search and
couldn't even find a JIRA ticket for this, so it doesn't seem to be on
anyone's radar.

If this is something you'd like to see implemented, go ahead and file a
ticket for it on JIRA
<https://issues.apache.org/jira/projects/KAFKA/issues/>. Honestly it may
have been overlooked completely. Sometimes the global table stuff can get
overshadowed since it's relatively less common. So definitely go ahead and
file a ticket to hopefully kick off the conversation.

I'm not saying it'll get picked up right away, but if there's no ticket and
no one asking for it, then it might never happen at all. Of course, we
accept KIPs -- if you really need this feature, or if you've ever been
interested in contributing to Apache Kafka/Kafka Streams, consider
implementing it yourself!

On Wed, Nov 22, 2023 at 12:26 AM Christian Zuegner
 wrote:

> Hi Sophie,
>
> thanks a lot for you tip! I've implemented a StateListener - to block
> queries when the state does not equal RUNNING. This will work perfectly now
> for our use-case!
>
>
> In the meantime I noticed the InteractiveQuery API v2 and give it a try.
> Unfortunately it seems not to cope with GlobalKTable. When try to run this:
>
> return
> streams.query(StateQueryRequest.inStore(STORE_NAME).withQuery(KeyQuery.withKey(key)));
>
> I got: "Global stores do not yet support the KafkaStreams#query API. Use
> KafkaStreams#store instead."
>
> From my point of view it would be great if this will work and behave like
> with IN_MEMORY StoreType as it is straight forward to use.
>
> Do you see a chance to get InteractiveQueryV2 work with GlobalKTable?
>
> Kind regards,
> Christian
>
> -Original Message-
> From: Sophie Blee-Goldman 
> Sent: Wednesday, November 22, 2023 1:51 AM
> To: christian.zueg...@ams-osram.com.invalid
> Cc: users@kafka.apache.org
> Subject: Re: GlobalKTable with RocksDB - queries before state RUNNING?
>
> [Sie erhalten nicht häufig E-Mails von sop...@responsive.dev. Weitere
> Informationen, warum dies wichtig ist, finden Sie unter
> https://aka.ms/LearnAboutSenderIdentification ]
>
> Just to make sure I understand the logs, you're saying the "new file
> processed" lines represent store queries, and presumably the
> com.osr.serKafkaStreamsService is your service that's issuing these queries?
>
> You need to wait for the app to finish restoring state before querying it.
> Based on this message -- "KafkaStreams has not been started, you can retry
> after calling start()" -- I assume you're kicking off the querying service
> right away and blocking queries until after KafkaStreams#start is called.
> But you need to wait for it to actually finish starting up, not just for
> start() to be called. The best way to do this is by setting a state
> listener via KafkaStreams#setStateListener, and then using this to listen
> in on the KafkaStreams.State and blocking the queries until the state has
> changed to RUNNING.
>
> In case you're curious about why this seems to work with in-memory stores
> but not with rocksdb, it seems like in the in-memory case, the queries that
> are attempted during restoration are blocked due to the store being closed
> (according to "(Quarkus Main Thread) the state store, store-name, is not
> open.")
>
> So why is the store closed for most of the restoration in the in-memory
> case only? This gets a bit into the weeds, but it has to do with the
> sequence of events in starting up a state store. When the global thread
> starts up, it'll first loop over all its state stores and call #init on
> them. Two things have to happen inside #init: the store is opened, and the
> store registers itself with the ProcessorContext. The #register involves
> various things, including a call to fetch the end offsets of the topic for
> global state stores. This is a blocking call, so the store might stay
> inside the #register call for a relatively long while.
>
> For RocksDB stores, we open the store first and then call #register, so by
> the time the GlobalStreamThread is sitting around waiting on the end
> offsets response, the store is open and your queries are getting through to
> it. However the in-memory store actually registers itself *first*, before
> marking itself as open, and so it remains closed for most of the time it
> spends in restoration and blocks any query attempts during this time.
>
> I suppose it would make sense t

Re: GlobalKTable with RocksDB - queries before state RUNNING?

2023-11-21 Thread Sophie Blee-Goldman
Just to make sure I understand the logs, you're saying the "new file
processed" lines represent store queries, and presumably the
com.osr.serKafkaStreamsService is your service that's issuing these queries?

You need to wait for the app to finish restoring state before querying it.
Based on this message -- "KafkaStreams has not been started, you can retry
after calling start()" -- I assume you're kicking off the querying service
right away and blocking queries until after KafkaStreams#start is called.
But you need to wait for it to actually finish starting up, not just for
start() to be called. The best way to do this is by setting a state
listener via KafkaStreams#setStateListener, and then using this to listen
in on the KafkaStreams.State and blocking the queries until the state has
changed to RUNNING.

In case you're curious about why this seems to work with in-memory stores
but not with rocksdb, it seems like in the in-memory case, the queries that
are attempted during restoration are blocked due to the store being closed
(according to "(Quarkus Main Thread) the state store, store-name, is not
open.")

So why is the store closed for most of the restoration in the in-memory
case only? This gets a bit into the weeds, but it has to do with the
sequence of events in starting up a state store. When the global thread
starts up, it'll first loop over all its state stores and call #init on
them. Two things have to happen inside #init: the store is opened, and the
store registers itself with the ProcessorContext. The #register involves
various things, including a call to fetch the end offsets of the topic for
global state stores. This is a blocking call, so the store might stay
inside the #register call for a relatively long while.

For RocksDB stores, we open the store first and then call #register, so by
the time the GlobalStreamThread is sitting around waiting on the end
offsets response, the store is open and your queries are getting through to
it. However the in-memory store actually registers itself *first*, before
marking itself as open, and so it remains closed for most of the time it
spends in restoration and blocks any query attempts during this time.

I suppose it would make sense to align the two store implementations to
have the same behavior, and the in-memory store is probably technically
more correct. But in the end you really should just wait for the
KafkaStreams.State to get to RUNNING before querying the state store, as
that's the only true guarantee.

Hope this helps!

-Sophie

On Tue, Nov 21, 2023 at 6:44 AM Christian Zuegner
 wrote:

> Hi,
>
> we have the following problem - a Kafka Topic ~20Megabytes is made
> available as GlobalKTable for queries. With using RocksDB the GKTable is
> ready for queries instantly even without having reading the data complete -
> all get() requests return null. After a few seconds the data is querieable
> correctly - but this is to late for our application. Once we switch to
> IN_MEMORY we get the expected behavior. The store is only ready after all
> data has been read from topic.
>
> How can we achieve the same behavior with the RocksDB setup?
>
> Snipet to build KafkaStreams Topology
>
> builder.globalTable(
>   "topic-name",
>   Consumed.with(Serdes.String(), Serdes.String()),
>
> Materialized.as(STORE_NAME).withStoreType(Materialized.StoreType.ROCKS_DB)
> );
>
> Query the Table
>
> while (true) {
> try {
> return streams.store(
>
> StoreQueryParameters.fromNameAndType(FileCrawlerKafkaTopologyProducer.STORE_NAME,
> QueryableStoreTypes.keyValueStore()));
> } catch (InvalidStateStoreException e) {
> logger.warn(e.getMessage());
> try {
> Thread.sleep(3000);
> } catch (InterruptedException ignored) {
> }
> }
> }
>
> The store is queried with getStore().get(key); <- here we get the null
> values.
>
> This is the Log Output when RocksDB - first query before state RUNNING
>
> ...
> 2023-11-21 15:15:40,629 INFO  [com.osr.serKafkaStreamsService] (Quarkus
> Main Thread) wait for kafka streams store to get ready: KafkaStreams has
> not been started, you can retry after calling start()
> 2023-11-21 15:15:41,781 INFO  [org.apa.kaf.str.KafkaStreams]
> (pool-10-thread-1) stream-client
> [topic-name-7c35d436-f18c-4cb9-9d87-80855df5d1a2] State transition from
> CREATED to REBALANCING
> 2023-11-21 15:15:41,819 INFO  
> [org.apa.kaf.str.sta.int.RocksDBTimestampedStore]
> (topic-name-7c35d436-f18c-4cb9-9d87-80855df5d1a2-GlobalStreamThread)
> Opening store store-name in regular mode
> 2023-11-21 15:15:41,825 INFO  [org.apa.kaf.str.pro.int.GlobalStateManagerImpl]
> (topic-name-7c35d436-f18c-4cb9-9d87-80855df5d1a2-GlobalStreamThread)
> global-stream-thread
> [topic-name-7c35d436-f18c-4cb9-9d87-80855df5d1a2-GlobalStreamThread]
> Restoring state for global store store-name
> 2023-11-21 15:15:43,753 INFO  [io.quarkus] (Quarkus Main 

[VOTE] 3.4.0 RC1

2023-01-20 Thread Sophie Blee-Goldman
Hello Kafka users, developers and client-developers,

This is the first candidate for release of Apache Kafka 3.4.0. Some of the
major features include:

* KIP-881: Rack-aware Partition Assignment for Kafka Consumers
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-881%3A+Rack-aware+Partition+Assignment+for+Kafka+Consumers>
(protocol changes only)

* KIP-876: Time based cluster metadata snapshots
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-876%3A+Time+based+cluster+metadata+snapshots
>

* KIP-787: MM2 manage Kafka resources with custom Admin implementation.
<https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=191335620>

* KIP-866 ZooKeeper to KRaft Migration
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-866+ZooKeeper+to+KRaft+Migration
>
(Early
Access)

For a full list of the features in this release, please refer to the
release notes:
https://home.apache.org/~ableegoldman/kafka-3.4.0-rc1/RELEASE_NOTES.html

*** Please download, test and vote by Friday, Jan 27th, 9am PT

Kafka's KEYS file containing PGP keys we use to sign the release:
https://kafka.apache.org/KEYS

* Release artifacts to be voted upon (source and binary):
https://home.apache.org/~ableegoldman/kafka-3.4.0-rc1/

* Maven artifacts to be voted upon:
https://repository.apache.org/content/groups/staging/org/apache/kafka/

* Javadoc:
https://home.apache.org/~ableegoldman/kafka-3.4.0-rc1/javadoc/

* Tag to be voted upon (off 3.4 branch) is the 3.4.0 tag:
https://github.com/apache/kafka/releases/tag/3.4.0-rc1

* Documentation:
https://kafka.apache.org/34/documentation.html

* Protocol:
https://kafka.apache.org/34/protocol.html

* Still working on getting a fully green build for both unit/integration
and system tests. I will update this thread with the successful builds as
we get them.

/******

Thanks,
Sophie Blee-Goldman


Re: Kafka Streams possible partitioner bug

2022-11-18 Thread Sophie Blee-Goldman
Hey Upesh, are you trying to plug in the custom partitioner via the
`partitioner.class` ProducerConfig?
That won't work in Streams for the exact reason you highlighted, which is
why Streams has its own
version of the interface called StreamPartitioner -- this is what you need
to implement instead.

Unfortunately there is currently no config for Streams that will be applied
across the application, so
you will have to plug in the custom partitioner by passing it in directly
to the operators. If you look
at the various APIs of the DSL you'll notice many have an overload which
takes in this parameter
(eg see "Produced")

As it turns out however I am currently working on a KIP for a
default.stream.partitioner config that
you will be able to set once rather than carefully passing it in across the
topology. I'll take this as
good evidence of the usefulness of this feature -- unfortunately you'll
have to wait for a bit as it
will not be available until version 3.5 most likely.

Anyways trying to use the Producer config is an honest mistake, and we
don't seem to include it
in the documented list of client configs that can't be set in Streams. I've
filed a ticket to fix up
the docs and also to explicitly log a warning if any of these are set
instead of silently ignoring them
or flat out breaking as in this case

https://issues.apache.org/jira/browse/KAFKA-14404
https://issues.apache.org/jira/browse/KAFKA-14405

On Fri, Nov 18, 2022 at 4:07 PM Upesh Desai  wrote:

> Hello all,
>
>
>
> We have been working on implementing a custom partitioner for our producer
> within a simple stream application, that will partition the records by a
> member field when sending them to the output topic. By looking at the
> contract of the partition() method in the Partitioner interface, it would
> seem that the value Object would be in its deserialized form when this
> method is called:
>
>
>
>
>
>
>
>
>
>
>
>
>
> */** * Compute the partition for the given record. * * @param topic The
> topic name * @param key The key to partition on (or null if no key) *
> @param keyBytes The serialized key to partition on( or null if no key) *
> @param value The value to partition on or null * @param valueBytes The
> serialized value to partition on or null * @param cluster The current
> cluster metadata */ *int partition(String topic, Object key, byte[]
> keyBytes, Object value, byte[] valueBytes, Cluster cluster);
>
>
>
> For a regular producer that’s instantiated, this seems to work correctly.
> However, within the RecordCollectorImpl class, we found that in a streams
> app, the record key and value are serialized prior to being sent as seen
> below:
>
>
>
>
>
> final ProducerRecord serializedRecord = new
> ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes, headers);
>
> streamsProducer.send(serializedRecord, (metadata, exception) -> {
>
>
>
> We didn’t want to have to deserialize the value object again within the
> custom partitioner, so is there another way around this? Or is this a bug
> within the streams producer code?
>
>
>
> Thanks in advance!
>
> Upesh Desai
> 
> Upesh Desai​
> Senior Software Developer
> *ude...@itrsgroup.com* 
> *www.itrsgroup.com* 
> Internet communications are not secure and therefore the ITRS Group does
> not accept legal responsibility for the contents of this message. Any view
> or opinions presented are solely those of the author and do not necessarily
> represent those of the ITRS Group unless otherwise specifically stated.
> [itrs.email.signature]
>


Re: Out of order messages when kafka streams application catches up

2022-11-15 Thread Sophie Blee-Goldman
Tomasz you'll need to upgrade the kafka Streams dependency to 3.0 (or above)
to get the fix that John mentioned before -- this behavior is
known/expected on
earlier versions such as 2.8.1 as you are using

On Tue, Nov 15, 2022 at 2:21 AM Tomasz Gac 
wrote:

> Hi John,
>
> I've reviewed the test you sent and it seems to be correct, but it may not
> reproduce our setup.
>
> We are using Java 8, kafka-client 2.8.1 with kafka streams version 2.8.1
> against the kafka broker version 2.1.1. We are running it as an OSGI bundle
> with dependencies packaged within the bundle.
>
> Thank you,
> Tomasz
>
>
> pt., 30 wrz 2022 o 17:57 John Roesler  napisał(a):
>
> > Hi again, Tomasz,
> >
> > Your issue is really bugging me, since I'm pretty sure it shouldn't be
> > happening.
> >
> > I went ahead and added an integration test with your exact scenario, as I
> > understand it: https://github.com/apache/kafka/pull/12706
> >
> > The test passes for me.
> >
> > Do you think you can check it out and try adjusting the test setup until
> > you're able to reproduce the behavior you're seeing? If you can do that,
> I
> > think we will get to the bottom of it.
> >
> > Thanks,
> > -John
> >
> > On Fri, Sep 30, 2022, at 09:51, John Roesler wrote:
> > > Hi Tomasz,
> > >
> > > Thanks for trying that out. It’s not the way I’d expect it to work. I
> > > don’t remember if there were any follow-up bugs that have been solved
> > > in subsequent releases. Just as a long shot, perhaps you can try it on
> > > the latest release (3.3.0)?
> > >
> > > Otherwise, I think the best path forward would be to file a bug report
> > > on the Apache Kafka Jira with enough information to reproduce the issue
> > > (or if you’re able to provide a repro, that would be awesome).
> > >
> > > Thanks, and sorry for the trouble.
> > > -John
> > >
> > > On Tue, Sep 27, 2022, at 03:15, Tomasz Gac wrote:
> > >> I upgraded to kafka streams 3.0.0 with positive task.max.idle.ms and
> > it did
> > >> not help.
> > >> When lag is large, the application still consumes data batches without
> > >> interleaving.
> > >>
> > >>
> > >>
> > >> wt., 27 wrz 2022 o 05:51 John Roesler 
> napisał(a):
> > >>
> > >>> Hi Tomasz,
> > >>>
> > >>> Thanks for asking. This sounds like the situation that we fixed in
> > Apache
> > >>> Kafka 3.0, with KIP-695 (
> > >>>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-695%3A+Further+Improve+Kafka+Streams+Timestamp+Synchronization
> > >>> ).
> > >>>
> > >>> Can you try upgrading and let us know if this fixes the problem?
> > >>>
> > >>> Thanks,
> > >>> -John
> > >>>
> > >>> On Mon, Sep 26, 2022, at 01:35, Tomasz Gac wrote:
> > >>> > Hi group,
> > >>> >
> > >>> > I wrote a simple kafka streams application with topology such as
> > below:
> > >>> >
> > >>> > builder.addStateStore(
> > >>> >> Stores.keyValueStoreBuilder(
> > >>> >> Stores.persistentKeyValueStore("STORE"),
> > >>> >> Serdes.String(), Serdes.String())
> > >>> >> .withLoggingEnabled(storeConfig))|
> > >>> >
> > >>> >
> > >>> >
> > >>> > builder.stream("TOPIC_1", Consumed.with(...))
> > >>> >> .merge(builder.stream("TOPIC_2", Consumed.with(...))
> > >>> >> .merge(builder.stream("TOPIC_3", Consumed.with(...))
> > >>> >> .map(...) // stateless
> > >>> >> .transform(..., "STORE")  // stateful
> > >>> >
> > >>> > .to("TOPIC_4");
> > >>> >
> > >>> >
> > >>> > All input topics have 6 partitions, and for the purpose of testing,
> > we
> > >>> are
> > >>> > producing data to partition number 5.
> > >>> > We are using kafka streams version 2.8.1, broker version 2.12-2.1.1
> > >>> >
> > >>> > The application works as expected when it has caught up to the lag,
> > eg.
> > >>> > when reset tool is used with --to-latest parameter.
> > >>> > However, when the application is processing the messages starting
> > from
> > >>> the
> > >>> > earliest offset, the inputs are provided in batches such as:
> > >>> >
> > >>> >- ~1000 messages from TOPIC_1
> > >>> >- ~1000 messages from TOPIC_2
> > >>> >- ~1000 messages from TOPIC_3
> > >>> >
> > >>> > All of the messages have timestamps provided in headers, so I would
> > >>> expect
> > >>> > the application to interleave the messages from these three topics
> so
> > >>> that
> > >>> > their timestamps are in the ascending order.
> > >>> > However, this is not the case that I am observing. The messages are
> > >>> > processed in batches.
> > >>> >
> > >>> > How do I configure my application so that it processes messages in
> > order
> > >>> > when it is catching up to the lag?
> > >>>
> >
>


Re: Cannot send in state COMMITTING_TRANSACTION

2022-11-15 Thread Sophie Blee-Goldman
Interesting, this does look like it could be a bug in Streams and I'm not
aware of
any known or already-fixed issues resembling this. Could you file a bug
report
over here 
and
include as much context/information as possible? Providing logs
from around the time leading up to this exception in particular would
greatly
help in debugging this

On Tue, Nov 15, 2022 at 2:15 AM Tomasz Gac 
wrote:

> Hi group,
>
> We've encountered a problem during regular operation of the kafka-streams
> application. While processing a record we received the following error.
> There's very little documentation on this kind of problem but I've gathered
> that it's a synchronization issue between kafka consumer and producer. Have
> you ever encountered it before?
>
> My questions are: is this a matter of misconfiguration, or rather a bug?
> Has it been fixed?
>
> We're using Java 8, kafka-client 2.8.1 with kafka streams version 2.8.1
> against the kafka broker version 2.1.1. We are running it as an OSGI bundle
> with dependencies packaged within the bundle.
>
> Thank you.
>
> 2022-10-18T01:20:19,665 | ERROR |
> TRADE-ENRICHER-35daba19-3907-4a55-8fce-606abf0e9b3a-StreamThread-1 |
> TaskManager | 218 - org.apache.servicemix.bundles.kafka-clients - 2.8.1.1 |
> stream-thread
> [TRADE-ENRICHER-35daba19-3907-4a55-8fce-606abf0e9b3a-StreamThread-1] Failed
> to process stream task 0_2 due to the following error:
> org.apache.kafka.streams.errors.StreamsException: Exception caught in
> process. taskId=0_2, processor=KSTREAM-SOURCE-02,
> topic=ORDER_BOOK_BEST, partition=2, offset=13452067,
> stacktrace=java.lang.IllegalStateException: Cannot call send in state
> COMMITTING_TRANSACTION at
>
> org.apache.kafka.clients.producer.internals.TransactionManager.failIfNotReadyForSend(TransactionManager.java:451)
> at
>
> org.apache.kafka.clients.producer.KafkaProducer.doSend(KafkaProducer.java:948)
> at
>
> org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:885)
> at
>
> org.apache.kafka.streams.processor.internals.StreamsProducer.send(StreamsProducer.java:211)
> at
>
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.send(RecordCollectorImpl.java:182)
> at
>
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.send(RecordCollectorImpl.java:139)
> at
>
> org.apache.kafka.streams.processor.internals.SinkNode.process(SinkNode.java:87)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forwardInternal(ProcessorContextImpl.java:281)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:260)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:219)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:172)
> at
>
> org.apache.kafka.streams.kstream.internals.KStreamMapValues$KStreamMapProcessor.process(KStreamMapValues.java:41)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorAdapter.process(ProcessorAdapter.java:71)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorNode.lambda$process$2(ProcessorNode.java:181)
> at
>
> org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency(StreamsMetricsImpl.java:884)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:181)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forwardInternal(ProcessorContextImpl.java:281)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:260)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:219)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:172)
> at
>
> org.apache.kafka.streams.kstream.internals.KStreamFlatMap$KStreamFlatMapProcessor.process(KStreamFlatMap.java:42)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorAdapter.process(ProcessorAdapter.java:71)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorNode.lambda$process$2(ProcessorNode.java:181)
> at
>
> org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency(StreamsMetricsImpl.java:884)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:181)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forwardInternal(ProcessorContextImpl.java:281)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:260)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:219)
> at
>
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:172)
> at
>
> 

Re: Kafka Streams - Producer attempted to produce with an old epoch.

2022-10-27 Thread Sophie Blee-Goldman
I'm not one of the real experts on the Producer and even further from one
with broker performance, so someone else may need to chime in for that, but
I did have a few questions:

What specifically are you unsatisfied with w.r.t the performance? Are you
hoping for a higher throughput
of your Streams app's output, or is there something about the brokers? I'm
curious why you started with
increasing the broker threads, especially if the perf issue/bottleneck is
with the Streams app's processing
(but maybe it is not). I would imagine that throwing more and more threads
at the machine could even
make things worse, it definitely will if the thread count gets high enough
though it's hard to say where/when
it might start to decline. Point is, if the brokers are eating up all the
cpu time with their own threads then
the clients embedded in Streams may be getting starved out at times,
causing that StreamThread/consumer
to drop out of the group and resulting in the producer getting fenced. Or
it could be blocking i/o for rocksdb
 and leading to write stalls, which could similarly get that StreamThread
kicked from the consumer group
(if the application has state, especially if quite a lot).

How many StreamThreads did you give the app btw?

On Thu, Oct 27, 2022 at 8:01 PM Andrew Muraco  wrote:

> Hi,
> I have a kafka streams application deployed on 5 nodes and with full
> traffic I am getting the error message:
>
> org.apache.kafka.common.errors.InvalidProducerEpochException: Producer
> attempted to produce with an old epoch.
> Written offsets would not be recorded and no more records would be sent
> since the producer is fenced, indicating the task may be migrated out
>
> I have 5 x 24 CPU/48 core machines with 128gb of ram. These machines are
> the kafka brokers with 2x1TB disks for kafka logs and also running the
> kafka Streams application.
> 2x replication factor on topic, topic is producing about 250k per second.
> I have 2 aggregations in the topology to 2 output topics, the final output
> topics are in the 10s of k range per second.
>
> I'm assuming I have a bottleneck somewhere, I increased the broker thread
> counts and observed that this frequency of this error reduced, but it's
> still happening.
> Here's the broker configuration I'm using now, but I might be overshooting
> some of these values.
>
> num.network.threads=48
> num.io.threads=48
> socket.send.buffer.bytes=512000
> socket.receive.buffer.bytes=512000
> replica.socket.receive.buffer.bytes=1024000
> socket.request.max.bytes=10485760
> num.replica.fetchers=48
> log.cleaner.threads=48
> queued.max.requests=48000
>
> I can't find good documentation on the effect of broker configuration on
> performance.
>


Re: kafka stream zombie state

2022-08-19 Thread Sophie Blee-Goldman
Well it sounds like your app is getting stuck somewhere in the poll loop so
it's unable to call poll
again within the session timeout, as the error message indicates -- it's a
bit misleading as it says
"Sending LeaveGroup request to coordinator" which implies it's
*currently* sending
the LeaveGroup,
but IIRC this error actually comes from the heartbeat thread -- just a long
way of clarifying that the
reason you don't see the state go into REBALANCING is because the
StreamThread is stuck and
can't rejoin the group by calling #poll

So...what now? I know your question was how to detect this, but I would
recommend first trying to
take a look into your application topology to figure out where, and *why*,
it's getting stuck (sorry for
the "q: how do I do X? answ. don't do X, do Y" StackOverflow-type response
-- happy to help with
that if we really can't resolve the underlying issue, I'll give it some
thought since I can't think of any
easy way to detect this off the top of my head)

What does your topology look like? Can you figure out what point it's
hanging? You may need to turn
on  DEBUG logging, or even TRACE, but given how infrequent/random this is
I'm guessing that's off
the table -- still, DEBUG logging at least would help.

Do you have any custom processors? Or anything in your topology that could
possibly fall into an
infinite loop? If not, I would suspect it's related to rocksdb -- but let's
start with the other stuff before
we go digging into that

Hope this helps,
Sophie

On Tue, Aug 16, 2022 at 1:06 PM Samuel Azcona 
wrote:

> Hi Guys, I'm having an issue with a kafka stream app, at some point I get a
> consumer leave group message. Exactly same issue described to another
> person here:
>
>
> https://stackoverflow.com/questions/61245480/how-to-detect-a-kafka-streams-app-in-zombie-state
>
> But the issue is that stream state is continuing reporting that the stream
> is running, but it's not consuming anything, but the stream never rejoin
> the consumer group, so my application with only one replica stop consuming.
>
> I have a health check on Kubernetes where I expose the stream state to then
> restart the pod.
> But as the kafka stream state it's always running when the consumer leaves
> the group, the app is still healthy in zombie state, so I need to manually
> go and restart the pod.
>
> Is this a bug? Or is there a way to check what is the stream consumer state
> to then expose as healt check for my application?
>
> This issue really happen randomly, usually all the Mondays. I'm using Kafka
> 2.8.1 and my app is made in kotlin.
>
> This is the message I get before zombie state, then there are no
> exceptions, errors or nothing more until I restart the pod manually.
>
> Sending LeaveGroup request to coordinator
> b-3.c4.kafka.us-east-1.amazonaws.com:9098 (id: 2147483644 rack: null)
> due to consumer poll timeout has expired. This means the time between
> subsequent calls to poll() was longer than the configured
> max.poll.interval.ms, which typically implies that the poll loop is
> spending too much time processing messages. You can address this
> either by increasing max.poll.interval.ms or by reducing the maximum
> size of batches returned in poll() with max.poll.records.
>
> Thanks for the help.
>


Re: Kafka Streams Topology State

2022-08-18 Thread Sophie Blee-Goldman
Hey Peter

Try clearing the local state -- if you have stateful tasks then by default
Streams will use rocksdb to
store records locally in directories specific to/named after that task.
This is presumably why you're
seeing errors related to "the task for peek node missing in old nodes"

You can delete the local state manually, which is generally recommended, or
by running `KafkaStreams#cleanup`
-- you just want to make sure to remove the #cleanup before you resume
running normally, in case of a restart for
whatever reason (since wiping out the local state can take a while to
restore from the changelogs)

Of course, just wiping out the local state alone will get you past any
problems related to that, but you might have
to fully reset the application (or might just have lost your current
data/state) if eg the changelog or repartition topic
names changed.

In general upgrading topologies is hard to do in a compatible way, but
there are a few tricks/features you can to
make it possible in some cases. For now we recommend naming all of your
operators to reduce the chance of
processor node names getting scrambled, and there's an experimental feature
which will allow you to break up
your topology into discrete "named topologies" which can be just a single
task, which then allows adding or removing
these named topologies without disrupting the others. It's still
technically not a public feature yet, however, so there's
no "public API" (you can still use it, it's just not public in the sense
that there's no contract for the API and it may change
across versions). Happy to expand on this more if you want but hopefully
the above is sufficient for now

cheers,
sophie

On Thu, Aug 18, 2022 at 7:28 AM Peter Cipov 
wrote:

> Hello,
>
> I am looking for  help regarding topologies and tasks in KS. Mostly where
> this information is stored outside of KS app.
>
> My case is upgrading the KS topology from v1 to v2, the topology is
> different (f.e adding peek node in v2). I made a change in code, prepared a
> jar and deployed.
>
> The issue that I observed was when old and new versions of app are up and
> running (by mistake). The task for peek node is missing in old nodes hence
> deployment crashes.
>
> My next steps were shutting down all nodes and deployment without old
> nodes. But this did not help. New deployment crashed too and I was not able
> to figure out why. The topology just stopped ingestion from the source
> topic. No error in logs.
>
> My hypothesis is that the former deployment of (KS  topology v1 and v2 at
> once) leaves some sort of ephemeral state in the kafka cluster itself.
>
> And here is my question. what kind of state except messages in repartition
> and changelog topics, consumer offsets are stored by KS in kafka cluster ?
>
> What can be the root cause from your point of view ?
>
> Thank you
> Peter
>


Re: ConsumerRecord / ProducerRecord common interface?

2021-10-22 Thread Sophie Blee-Goldman
I agree, this has always really annoyed me. This will definitely require a
KIP since
it's a public feature, but it should (hopefully!) be a fairly quick and
non-controversial one.
You can read up on the process and get started here


I'd also recommend filing a JIRA ticket and replying to this thread with
the link. Jira
is a better place to host the debate, if there is any, and discuss possible
solutions or
ask questions.

Cheers,
Sophie

On Fri, Oct 22, 2021 at 11:23 AM Steven Schlansker <
stevenschlans...@gmail.com> wrote:

> Hello Kafka friends,
>
> We are writing test code that sometimes will talk to a MockProducer
> and sometimes to a normal KafkaProducer. Therefore the test harness
> will sometimes read the MockProducer-produced records (ProducerRecord)
> directly, and sometimes will read actual ConsumerRecords through a
> real Consumer from the in-process test broker.
>
> We would like to be able to assert over a record, whether producer or
> consumer, but the classes do not have any common interface. Most of
> the properties are shared: topic, partition, headers, key, value, and
> timestamp.
>
> A straightforward improvement would be to have both ConsumerRecord and
> ProducerRecord implement a common interface, KafkaRecord, with
> accessors for these properties.
>
> I'm happy to contribute the code which should be straightforward but
> wanted to validate this idea and make sure it is a contribution that
> would be accepted before spending the time to do this. In particular
> it might require a KIP even though it's a trivial feature since the
> public API is affected.
>
> Does this sound like a good idea to implement? What would the next steps
> be?
>
> Thanks!
> Steven
>


Re: Downgrading KafkaStreams

2021-10-14 Thread Sophie Blee-Goldman
Matthias, shouldn't version probing prevent this sort of thing from
happening? That is, shouldn't
a live downgrade always be possible? I know there were some bugs in the
downgrade path for
version probing in the past, but I'm pretty sure they should have been
fixed by 2.4.1

This seems like a bug to me, no?

On Wed, Oct 13, 2021 at 9:52 AM Murilo Tavares  wrote:

> Thanks Mathias
> In this case, I have one instance running.
> Maybe it's the case that the session also needed to timeout?
> Thanks
> Murilo
>
>
>
> On Wed, 13 Oct 2021 at 12:25, Matthias J. Sax  wrote:
>
> > For this case, it seems you cannot do a rolling downgrade, but you will
> > need to stop all instances, before restarting with 2.4.1.
> >
> > -Matthias
> >
> > On 10/13/21 7:34 AM, Murilo Tavares wrote:
> > > Hi
> > > I have a large KafkaStreams topology, and for a while I have failed to
> > > upgrade it from version 2.4.1 to 2.7.0, and this time to version 2.8.1.
> > > (keeps stuck on rebalance loop)
> > > I was able to revert it from v2.7.0 back to 2.4.1 in the past, but now
> I
> > > can't rollback my code, as I get the following error.
> > > I have cleaned up the state via "streams.cleanUp()", but still can't
> get
> > > rid of the error.
> > > Any suggestion on how to downgrade it?
> > >
> > >
> > > Exception in thread "streams-app-0-StreamThread-1"
> > > org.apache.kafka.streams.errors.TaskAssignmentException: Unable to
> decode
> > > assignment data: used version: 9; latest supported version: 5
> > > at
> > >
> >
> org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo.decode(AssignmentInfo.java:306)
> > > at
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.onAssignment(StreamsPartitionAssignor.java:1091)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:391)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:421)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:340)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:471)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1267)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1231)
> > > at
> > >
> >
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1211)
> > > at
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:843)
> > > at
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:743)
> > > at
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:698)
> > > at
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:671)
> > >
> >
>


Re: Kafka Streams leave group behaviour

2021-08-18 Thread Sophie Blee-Goldman
As Boyang mentioned, Kafka Streams intentionally does not send a LeaveGroup
request
when shutting down. This is because often the shutdown is not due to a
scaling down event
but instead some transient closure, such as during a rolling bounce. In
cases where the
instance is expected to start up again shortly after, we originally wanted
to avoid that member's
tasks from being redistributed across the remaining group members since
this would disturb the
stable assignment and could cause unnecessary state migration and
restoration. We also hoped
to limit the disruption to just a single rebalance, rather than forcing the
group to rebalance once
when the member shuts down and then again when it comes back up. So it's
really an optimization
for the case in which the shutdown is temporary.

That said, many of those optimizations are no longer necessary or at least
much less useful given
recent features and improvements. For example rebalances are now
lightweight so skipping the
2nd rebalance is not as worth optimizing for, and the new assignor will
take into account the actual
underlying state for each task/partition assignment, rather than just the
previous assignment, so
the assignment should be considerably more stable across bounces and
rolling restarts.

Given that, it might be time to reconsider this optimization.
Alternatively, we could introduce another
form of the close() API that forces the member to leave the group, to be
used in event of actual scale
down rather than a transient bounce.

Filed https://issues.apache.org/jira/browse/KAFKA-13217

On Thu, Aug 12, 2021 at 6:14 PM Boyang Chen 
wrote:

> You are right Uwe, Kafka Streams won't leave group no matter dynamic or
> static membership. If you want to have fast scale down, consider trying
> static membership and use the admin command `removeMemberFromGroup` when
> you need to rescale.
>
> Boyang
>
> On Thu, Aug 12, 2021 at 4:37 PM Lerh Chuan Low 
> wrote:
>
> > I think you may have stumbled upon this:
> > https://issues.apache.org/jira/browse/KAFKA-4881. 1 thing that you could
> > try is using static membership - we have yet to try that though so can't
> > comment yet on how that might work out.
> >
> > On Thu, Aug 12, 2021 at 11:29 PM c...@uweeisele.eu 
> > wrote:
> >
> > > Hello all,
> > >
> > > I have a question about the Group Membership lifecycle of Kafka
> Streams,
> > > or more specific about when Kafka Streams does leave the consumer group
> > (in
> > > case of dynamic membership).
> > >
> > > My expectation was, that a call to the method KafkaStreams.close() also
> > > sends a LeaveGroup request to the coordination (if dynamic membership
> is
> > > used). However, its seems that this is not the case (at least in my
> case
> > > the request was not send). Only if I explicitly call
> > > KafkaStreams.removeStreamThread() a LeaveGroup request is sent to the
> > > coordinator. I used the WordCount example located in
> > > https://github.com/confluentinc/kafka-streams-examples to evaluate
> this.
> > >
> > > Is this how Kafka Streams is intended to work and if yes, what do you
> > > recommend to achieve that Kafka Streams leaves the group when shutting
> > down
> > > the application? For example, one situation where I don't want to wait
> > for
> > > the session timeout is when downscaling an application.
> > >
> > > Thanks.
> > >
> > > Best Regards,
> > > Uwe
> >
>


Re: Consumer group rebalancing

2021-07-23 Thread Sophie Blee-Goldman
Check out the GroupCoordinator logs, they say why the group is rebalancing.
Specifically the one that begins "Preparing to rebalance group"

If you search for that in the logs you shared, it shows the reason:

2021-07-22 09:26:37,202 INFO [GroupCoordinator 1]: Preparing to rebalance
> group shows_elk_Group in state PreparingRebalance with old generation 8
> (__consumer_offsets-3) (*reason: removing member
> cms-kafka-service-11a6a273-9a7c-4053-b6dd-1ca27d06690f on heartbeat
> expiration*) (kafka.coordinator.group.GroupCoordinator)
> [executor-Heartbeat]
> 2021-07-22 10:04:27,124 INFO [GroupCoordinator 1]: Preparing to rebalance
> group shows_elk_Group in state PreparingRebalance with old generation 9
> (__consumer_offsets-3) (*reason: Adding new member
> cms-kafka-service-9a363fe5-79a2-4dfb-9c69-4171abed14da with group instance
> id None*) (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-5]

2021-07-22 10:05:23,657 INFO [GroupCoordinator 1]: Preparing to rebalance
> group shows_elk_Group in state PreparingRebalance with old generation 10
> (__consumer_offsets-3) (*reason: removing member
> cms-kafka-service-0dab8855-ebe2-4e03-bdca-b1351e122b42 on heartbeat
> expiration*) (kafka.coordinator.group.GroupCoordinator)
> [executor-Heartbeat]

2021-07-22 10:13:36,957 INFO [GroupCoordinator 1]: Preparing to rebalance
> group shows_elk_Group in state PreparingRebalance with old generation 11
> (__consumer_offsets-3) (*reason: Adding new member
> cms-kafka-service-612f2458-5b05-427e-8776-0e4270b2c215 with group instance
> id None*) (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-4]

 2021-07-22 10:14:33,966 INFO [GroupCoordinator 1]: Preparing to rebalance
> group shows_elk_Group in state PreparingRebalance with old generation 12
> (__consumer_offsets-3) *(reason: removing member
> cms-kafka-service-9a363fe5-79a2-4dfb-9c69-4171abed14da on heartbeat
> expiration*) (kafka.coordinator.group.GroupCoordinator)
> [executor-Heartbeat]


Looks like a consumer is getting kicked from the group due to heartbeat
expiration and then rejoining over this time period. Unclear from the
server side logs whether it's dropping out because the group was scaled
down, or the pod running that consumer was bounced, etc, or if the consumer
was indeed alive that whole time but somehow couldn't get a heartbeat
through. You'd need to look into the client logs to figure out which of
those things it is. You can try increasing the session.timeout.ms if you
believe there was no change in group membership (ie all consumers were up
and running the whole time).

On Fri, Jul 23, 2021 at 8:22 AM shreyas shetty 
wrote:

> Hi  we have around 70-75 topics created on kafka cluster.
> And there is a consumer pod which is running in different namespace. This
> consumer pod creates separate consumer group for each of the topic. when I
> check the logs of consumer pod it shows the below error:
> ```
> GroupId: shows_elk_Group, Topic: kafka-shows connected
> GroupId: movies_elk_Group, Topic: kafka-movies connected
>
> {"level":"ERROR","timestamp":"2021-07-22T10:14:33.977Z","logger":"kafkajs","message":"[Runner]
> The group is rebalancing,
> re-joining","groupId":"shows_elk_Group","memberId":"cms-kafka-service-612f2458-5b05-427e-8776-0e4270b2c215","error":"The
> group is rebalancing, so a rejoin is needed","retryCount":0,"retryTime":92}
>
> {"level":"ERROR","timestamp":"2021-07-22T10:14:33.165Z","logger":"kafkajs","message":"[Runner]
> The group is rebalancing,
> re-joining","groupId":"movies_elk_Group","memberId":"cms-kafka-service-56cda415-2414-4054-b285-6c794edf1700","error":"The
> group is rebalancing, so a rejoin is needed","retryCount":0,"retryTime":102}
>
> ```
> I am getting same error as above for all other consumer group as well.
>
> Sharing the kafka broker logs:
> ```
>  kubectl logs my-kafka-cluster-kafka-1 -n kafka| grep shows_elk_Group
> 2021-07-22 03:54:18,466 INFO Loaded member
> MemberMetadata(memberId=cms-kafka-service-df1038e3-d5db-4e62-885f-9f8d011ae9ff,
> groupInstanceId=None, clientId=cms-kafka-service, clientHost=/10.244.2.192,
> sessionTimeoutMs=3, rebalanceTimeoutMs=6,
> supportedProtocols=List(RoundRobinAssigner), ) in group shows_elk_Group
> with generation 3. (kafka.coordinator.group.GroupMetadata$)
> [group-metadata-manager-0]
> 2021-07-22 03:54:18,471 INFO Loaded member
> MemberMetadata(memberId=cms-kafka-service-df1038e3-d5db-4e62-885f-9f8d011ae9ff,
> groupInstanceId=None, clientId=cms-kafka-service, clientHost=/10.244.2.192,
> sessionTimeoutMs=3, rebalanceTimeoutMs=6,
> supportedProtocols=List(RoundRobinAssigner), ) in group shows_elk_Group
> with generation 4. (kafka.coordinator.group.GroupMetadata$)
> [group-metadata-manager-0]
> 2021-07-22 03:54:18,471 INFO Loaded member
> MemberMetadata(memberId=cms-kafka-service-11a6a273-9a7c-4053-b6dd-1ca27d06690f,
> groupInstanceId=None, clientId=cms-kafka-service, clientHost=/10.244.2.192,
> 

Re: documentation css

2021-07-12 Thread Sophie Blee-Goldman
I see what you mean, that's pretty ugly. Can you file a bug report at
https://issues.apache.org/jira/issues/ so we can
track and follow up on this?

Thanks!

On Mon, Jul 12, 2021 at 7:35 AM Rasto Janotka 
wrote:

> Hi,
> I am using your documentation it is written very nice and clean but there
> is some small "bug in the format" in CSS/HTML and some sections (
> table/code) are quite hard to read.
>
>
> see:
>
>
>
> https://kafka.apache.org/10/documentation/streams/developer-guide/config-streams
>
> [image: Selection_999(086).jpg]
>
>
> thank you
> with regards
> Rastislav Janotka
>


Re: Consumer Group Stuck on "Completing-Rebalance" State

2021-06-23 Thread Sophie Blee-Goldman
Just to clarify, this bug actually does impact only the cooperative-sticky
assignor. The cooperative sticky assignor gets its
"ownedPartitions" input from the (possibly corrupted) SubscriptionState,
while the plain sticky assignor has to rely on
keeping track of these partitions itself, since in eager rebalancing the
"ownedPartitions" are always empty during a rebalance.
So you can safely use the regular sticky assignor to avoid this issue.

On Wed, Jun 23, 2021 at 4:38 PM Luke Chen  wrote:

> Hi Tao,
> 1. So this bug only applies to cooperative-sticky assignor?
> --> Yes, this bug only applies to sticky assignor (both eager and
> cooperative) since we will refer to the consumer's previous assignment.
>
> 2. Does assignor strategy (cooperative-sticky vs sticky vs others) really
> matter in this case?
> --> No, the assignor strategy won't affect the at most once. They are
> independent concepts.
>
> That is, to workaround this issue, please change to a non-sticky assignor
> before the bug fixed.
>
> Thank you.
> Luke
>
> Tao Huang  於 2021年6月23日 週三 下午9:34 寫道:
>
> > Thank you Sophie for sharing the details.
> >
> > So this bug only applies to cooperative-sticky assignor? Should I switch
> > to other strategy (eg: StickyAssignor) while I am waiting for the fix?
> >
> > On the other hand, my application is using "auto-commit" mechanism for
> "at
> > most once" event consuming. Does assignor strategy (cooperative-sticky vs
> > sticky vs others) really matter in this case? My understanding is that,
> > regardless which strategy is used, the members in the group have to
> rejoin
> > when re-balance happens.
> >
> > Thanks!
> >
> > Tao
> >
> > On 2021/06/23 02:01:04, Sophie Blee-Goldman  >
> > wrote:
> > > Here's the ticket: https://issues.apache.org/jira/browse/KAFKA-12984
> > >
> > > And the root cause of that itself:
> > > https://issues.apache.org/jira/browse/KAFKA-12983
> > >
> > > On Tue, Jun 22, 2021 at 6:15 PM Sophie Blee-Goldman <
> sop...@confluent.io
> > >
> > > wrote:
> > >
> > > > Hey Tao,
> > > >
> > > > We recently discovered a bug in the way that the consumer tracks
> > partition
> > > > metadata which may cause the cooperative-sticky assignor to throw
> this
> > > > exception in the case of a consumer that dropped out of the group at
> > some
> > > > point. I'm just about to file a ticket for it, and it should be fixed
> > in
> > > > the upcoming releases.
> > > >
> > > > The problem is that some consumers are claiming to own partitions
> that
> > > > they no longer actually own after having dropped out. If you can
> narrow
> > > > down the problematic consumers and restart them, it should resolve
> the
> > > > issue. I believe you should be able to tell which consumers are
> > claiming
> > > > partitions they no longer own based on the logs, but another option
> is
> > just
> > > > to restart all the consumers (or do a rolling restart until the
> problem
> > > > goes away).
> > > >
> > > > I'll follow up here with the ticket link once I've filed it.
> > > >
> > > > -Sophie
> > > >
> > > > On Tue, Jun 22, 2021 at 12:07 PM Tao Huang  >
> > > > wrote:
> > > >
> > > >> Thanks for the feedback.
> > > >>
> > > >> It seems the referred bug is on the server (Broker) side? I just
> > checked
> > > >> my Kafka Broker version, it is actually on 2.4.1. So the bug seems
> > does not
> > > >> apply to my case.
> > > >>
> > > >> Should I downgrade my client (Java library) version to 2.4.1?
> > > >>
> > > >> Thanks!
> > > >>
> > > >> On 2021/06/21 20:04:31, Ran Lupovich  wrote:
> > > >> >
> > https://issues.apache.org/jira/plugins/servlet/mobile#issue/KAFKA-12890
> > > >> >
> > > >> > Check out this jira ticket
> > > >> >
> > > >> > בתאריך יום ב׳, 21 ביוני 2021, 22:15, מאת Tao Huang ‏<
> > > >> > sandy.huang...@gmail.com>:
> > > >> >
> > > >> > > Hi There,
> > > >> > >
> > > >> > > I am experiencing intermittent issue when consumer group stuck
> on
> > > >> > > "Completing-Rebla

Re: Consumer Group Stuck on "Completing-Rebalance" State

2021-06-22 Thread Sophie Blee-Goldman
Here's the ticket: https://issues.apache.org/jira/browse/KAFKA-12984

And the root cause of that itself:
https://issues.apache.org/jira/browse/KAFKA-12983

On Tue, Jun 22, 2021 at 6:15 PM Sophie Blee-Goldman 
wrote:

> Hey Tao,
>
> We recently discovered a bug in the way that the consumer tracks partition
> metadata which may cause the cooperative-sticky assignor to throw this
> exception in the case of a consumer that dropped out of the group at some
> point. I'm just about to file a ticket for it, and it should be fixed in
> the upcoming releases.
>
> The problem is that some consumers are claiming to own partitions that
> they no longer actually own after having dropped out. If you can narrow
> down the problematic consumers and restart them, it should resolve the
> issue. I believe you should be able to tell which consumers are claiming
> partitions they no longer own based on the logs, but another option is just
> to restart all the consumers (or do a rolling restart until the problem
> goes away).
>
> I'll follow up here with the ticket link once I've filed it.
>
> -Sophie
>
> On Tue, Jun 22, 2021 at 12:07 PM Tao Huang 
> wrote:
>
>> Thanks for the feedback.
>>
>> It seems the referred bug is on the server (Broker) side? I just checked
>> my Kafka Broker version, it is actually on 2.4.1. So the bug seems does not
>> apply to my case.
>>
>> Should I downgrade my client (Java library) version to 2.4.1?
>>
>> Thanks!
>>
>> On 2021/06/21 20:04:31, Ran Lupovich  wrote:
>> > https://issues.apache.org/jira/plugins/servlet/mobile#issue/KAFKA-12890
>> >
>> > Check out this jira ticket
>> >
>> > בתאריך יום ב׳, 21 ביוני 2021, 22:15, מאת Tao Huang ‏<
>> > sandy.huang...@gmail.com>:
>> >
>> > > Hi There,
>> > >
>> > > I am experiencing intermittent issue when consumer group stuck on
>> > > "Completing-Reblalance" state. When this is happening, client throws
>> error
>> > > as below:
>> > >
>> > > 2021-06-18 13:55:41,086 ERROR io.mylab.adapter.KafkaListener
>> > > [edfKafkaListener:CIO.PandC.CIPG.InternalLoggingMetadataInfo]
>> Exception on
>> > > Kafka listener (InternalLoggingMetadataInfo) - Some partitions are
>> > > unassigned but all consumers are at maximum capacity
>> > > java.lang.IllegalStateException: Some partitions are unassigned but
>> all
>> > > consumers are at maximum capacity
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.constrainedAssign(AbstractStickyAssignor.java:248)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.assign(AbstractStickyAssignor.java:81)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.CooperativeStickyAssignor.assign(CooperativeStickyAssignor.java:64)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor.assign(AbstractPartitionAssignor.java:66)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:589)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:686)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:111)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:599)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:562)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1151)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1126)
>> > > at
>> > >
>> > >
>> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:206)
>> > > at
>> > >
>> > &g

Re: Consumer Group Stuck on "Completing-Rebalance" State

2021-06-22 Thread Sophie Blee-Goldman
Hey Tao,

We recently discovered a bug in the way that the consumer tracks partition
metadata which may cause the cooperative-sticky assignor to throw this
exception in the case of a consumer that dropped out of the group at some
point. I'm just about to file a ticket for it, and it should be fixed in
the upcoming releases.

The problem is that some consumers are claiming to own partitions that they
no longer actually own after having dropped out. If you can narrow down the
problematic consumers and restart them, it should resolve the issue. I
believe you should be able to tell which consumers are claiming partitions
they no longer own based on the logs, but another option is just to restart
all the consumers (or do a rolling restart until the problem goes away).

I'll follow up here with the ticket link once I've filed it.

-Sophie

On Tue, Jun 22, 2021 at 12:07 PM Tao Huang  wrote:

> Thanks for the feedback.
>
> It seems the referred bug is on the server (Broker) side? I just checked
> my Kafka Broker version, it is actually on 2.4.1. So the bug seems does not
> apply to my case.
>
> Should I downgrade my client (Java library) version to 2.4.1?
>
> Thanks!
>
> On 2021/06/21 20:04:31, Ran Lupovich  wrote:
> > https://issues.apache.org/jira/plugins/servlet/mobile#issue/KAFKA-12890
> >
> > Check out this jira ticket
> >
> > בתאריך יום ב׳, 21 ביוני 2021, 22:15, מאת Tao Huang ‏<
> > sandy.huang...@gmail.com>:
> >
> > > Hi There,
> > >
> > > I am experiencing intermittent issue when consumer group stuck on
> > > "Completing-Reblalance" state. When this is happening, client throws
> error
> > > as below:
> > >
> > > 2021-06-18 13:55:41,086 ERROR io.mylab.adapter.KafkaListener
> > > [edfKafkaListener:CIO.PandC.CIPG.InternalLoggingMetadataInfo]
> Exception on
> > > Kafka listener (InternalLoggingMetadataInfo) - Some partitions are
> > > unassigned but all consumers are at maximum capacity
> > > java.lang.IllegalStateException: Some partitions are unassigned but all
> > > consumers are at maximum capacity
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.constrainedAssign(AbstractStickyAssignor.java:248)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.assign(AbstractStickyAssignor.java:81)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.CooperativeStickyAssignor.assign(CooperativeStickyAssignor.java:64)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor.assign(AbstractPartitionAssignor.java:66)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:589)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:686)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:111)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:599)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:562)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1151)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1126)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:206)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:169)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:129)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:602)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:412)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:297)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:236)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1296)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237)
> > > at
> > >
> > >
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210)
> > > at io.mylab.adapter.KafkaListener.run(EdfKafkaListener.java:93)
> > > at java.lang.Thread.run(Thread.java:748)
> > >
> > > The option to exit the state is to stop some members of the consumer
> 

Re: Kafka contributor list request

2021-05-27 Thread Sophie Blee-Goldman
Done, added you to Confluence and Jira so you should be able to self-assign
tickets and create KIPs if necessary.

Welcome to Kafka :)

On Thu, May 27, 2021 at 4:28 PM Norbert Wojciechowski <
wojciechowski.norbert.git...@gmail.com> wrote:

> Hello,
>
> Can I please be assigned to Kafka contributor list on Confluence/Jira, so I
> can start contributing to Kafka and be able to work on issues?
>
> My Jira username is: erzbnif
>
> Thanks,
> Norbert
>


Re: Unable to put more than 3 consumers in a single process

2021-05-08 Thread Sophie Blee-Goldman
Hey Lahiru,

As the log message describes, the member is leaving the group because the
consumer
has been closed by your process/application. You need to track down where
in your
application code the Consumer#close method is being called, and why.

There's no technical limit on how many consumers can be in the same
process. Of course
there is a practical limit, eg if you try to start up millions of consumers
then they may struggle
to stay in the group for example due to the millions of background
heartbeat threads competing
for cpu and network resources. But I see no reason why a modern machine
would be unable
to handle just 4 consumers, and anyways the log message you provided makes
it clear that
the consumer is only leaving the group due to being manually closed.

Hope that helps,
Sophie

On Thu, May 6, 2021 at 7:29 AM Lahiru Chandima 
wrote:

>  Hi,
>
> I am trying to place multiple kafka consumers of a single consumer group in
> the same process. But, I am unable to put more than 3 consumers in the same
> process.The topic which the consumer group tries to read has 10
> partitions.When I try to attach the 4th consumer, it leaves the group after
> the first poll, with below message. No partitions get assigned to the
> consumer.
>
> [ForkJoinPool.commonPool-worker-3] INFO
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator - [Consumer
> clientId=consumer-group65-4, groupId=group65] Member
> consumer-group65-4-b148f7ec-b852-46ed-8514-c3af3bd78395 sending LeaveGroup
> request to coordinator my-cluster:9092 (id: 2147483645 rack: null) due to
> the consumer is being closed
>
> What is the reason for this? Is there any way to put more than 3 consumers
> in the same process?
>
> Thank you
> --
> Lahiru Chandima
>


[ANNOUNCE] Apache Kafka 2.6.2

2021-04-21 Thread Sophie Blee-Goldman
The Apache Kafka community is pleased to announce the release for Apache
Kafka 2.6.2

This is a bug fix release and it includes fixes and improvements from
35 JIRAs, including a few critical bugs.

All of the changes in this release can be found in the release notes:
https://www.apache.org/dist/kafka/2.6.2/RELEASE_NOTES.html


You can download the source and binary release (Scala 2.12
and 2.13) from:
https://kafka.apache.org/downloads#2.6.2

---


Apache Kafka is a distributed streaming platform with four core APIs:


** The Producer API allows an application to publish a stream of records to
one or more Kafka topics.

** The Consumer API allows an application to subscribe to one or more
topics and process the stream of records produced to them.

** The Streams API allows an application to act as a stream processor,
consuming an input stream from one or more topics and producing an
output stream to one or more output topics, effectively transforming the
input streams to output streams.

** The Connector API allows building and running reusable producers or
consumers that connect Kafka topics to existing applications or data
systems. For example, a connector to a relational database might
capture every change to a table.


With these APIs, Kafka can be used for two broad classes of application:

** Building real-time streaming data pipelines that reliably get data
between systems or applications.

** Building real-time streaming applications that transform or react
to the streams of data.


Apache Kafka is in use at large and small companies worldwide, including
Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank,
Target, The New York Times, Uber, Yelp, and Zalando, among others.

A big thank you for the following 26 contributors to this release!

A. Sophie Blee-Goldman, Andy Wilkinson, Bob Barrett, Bruno Cadonna,
Chia-Ping Tsai, Chris Egerton, dengziming, Greg Harris, Ismael Juma, Jim
Galasyn, John Roesler, Julien Jean Paul Sirocchi, Lee Dongjin, Luke Chen,
Manikumar Reddy, Matthias J. Sax, Michael G. Noll, Mickael Maison, Rajini
Sivaram, Ramesh Krishnan M, Randall Hauch, Sharath Bhat, Stanislav
Vodetskyi, Walker Carlson, Yilong Chang

We welcome your help and feedback. For more information on how to
report problems, and to get involved, visit the project website at
https://kafka.apache.org/

Thank you!


Cheers,

Sophie


Re: Deletion a specific consumer of a consumer group through the Kafka admin client API.

2021-04-09 Thread Sophie Blee-Goldman
Hey Mazen,

There is technically an AdminClient API which allows you to remove a single
member from the group:

Admin.removeMembersFromConsumerGroup(String groupId,
RemoveMembersFromConsumerGroupOptions options);

but I don't think this will help you. For one thing, this API only works
for removing members of a consumer group that uses static membership. The
*RemoveMembersFromConsumerGroupOptions* parameter only allows you to
specify the consumer to remove via the groupInstanceId, which defines a
static member.

Perhaps more importantly, this API does not *force* the member to
permanently be expelled from the group, rather it just informs the broker
that this particular consumer has actually already left the group and
therefore it should be kicked out. With static membership, it can take a
while for the broker to notice a member who has actually left the group vs
one who has temporarily "disappeared", eg due to a pod restart or a rolling
bounce. So this API was added to let users explicitly remove a member from
the group when that consumer app has been taken down in order to let its
partitions be reassigned to other members as quickly as possible.

If you tried to use that API on a static member who was still actively
running, I'm pretty sure it would just rejoin the group the next time it
called poll(). I think this hints at the motivation for not providing a
"forcibly remove member from group" API in the first place -- consumer
groups generally work the other way around, ie the existence of individual
consumers calling poll() is what determines who is/isn't in the group, and
the broker is just there to keep track so it distribute partitions only
among the live members. The group coordinator isn't the "source of truth"
for which consumers are actually in the group, that would be the consumers
themselves. Does that make sense?

But I see where you're coming from, so maybe there's another way to do what
you're trying to do. I would focus less on trying to figure out pod names
and more on how to get consumers to spin up/down when you want to scale
in/out -- after all k8s and the physical pods are just a detail of the
underlying architecture, and should probably be abstracted away from the
logic of your Consumer application. Instead of killing the pod, can you
just stop/exit the process? For example let's say you just want to remove a
member if it goes below some threshold consumption rate.

public static void main() {
Consumer consumer = new KafkaConsumer();
int consumptionRate;
while (consumptionRate > threshold) {

consumer.poll();
// do some processing

// update the consumption rate
   consumptionRate = computeConsumptionRate();
}
// done -- just exit the JVM
}

If you had something slightly more complicated in mind, like removing the
consumer with the worst consumption rate after each rebalance, then you
could use the Subscription to have each consumer report its consumption
rate to the group leader. Then the leader figures out the worst one and
sends a signal back in its Assignment to inform that Consumer to shut down
after this rebalance (and, presumably, don't assign any partitions to the
one shutting down)

On Fri, Apr 9, 2021 at 6:04 AM Mazen Ezzeddine <
mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:

> Dear all,
>
>
>
> The kafka admin client API enables the deletion of a consumer group
> through a logic like the one shown below
>
>
>
> DeleteConsumerGroupsResult deleteConsumerGroupsResult =
> adminClient.deleteConsumerGroups(Arrays.asList(consumerGroupToBeDeleted));
>
>
>
> However, is there any way/API through which the admin client can delete a
> specific consumer of a consumer group (e.g., say, the one which has the
> minimum consumption rate )…
>
>
>
>
>
> My kafka cluster is running on Kubernetes and I can use a Kubernetes
> client to dynamically delete a particular pod (eventually a Kafka
> consumer). However, naming of consumer pods in Kubernetes is
>
> different than the actual consumer names in a Kafka cluster that are used
> by an admin client to resolve and access consumers. And hence, the
> association between Kubernetes kafka consumer pods
>
>  naming and the actual consumer names used (by the admin client)  inside
> kafka cluster looks difficult…
>
>
>
> Any hint on that issue?
>
>
>
> Between, the decision to implement a whole consumer group deletion API on
> the admin client but not an individual consumer in the group? is just a
> design decision or there is any technical/protocol
>
> issues that enforce not offering such an API?
>
>
>
> Thank you.
>
>
>


[VOTE] 2.6.2 RC1

2021-04-08 Thread Sophie Blee-Goldman
Hello Kafka users, developers and client-developers,

This is the second candidate for release of Apache Kafka 2.6.2.

Apache Kafka 2.6.2 is a bugfix release and fixes 35 issues since the 2.6.1
release. Please see the release notes for more information.

Release notes for the 2.6.2 release:
https://home.apache.org/~ableegoldman/kafka-2.6.2-rc1/RELEASE_NOTES.html

*** Please download, test and vote by Tuesday, April 13th, 9am PST

Kafka's KEYS file containing PGP keys we use to sign the release:
https://kafka.apache.org/KEYS

* Release artifacts to be voted upon (source and binary):
https://home.apache.org/~ableegoldman/kafka-2.6.2-rc1/

* Maven artifacts to be voted upon:
https://repository.apache.org/content/groups/staging/org/apache/kafka/

* Javadoc:
https://home.apache.org/~ableegoldman/kafka-2.6.2-rc1/javadoc/

* Tag to be voted upon (off 2.6 branch) is the 2.6.2 tag:
https://github.com/apache/kafka/releases/tag/2.6.2-rc1

* Documentation:
https://kafka.apache.org/26/documentation.html

* Protocol:
https://kafka.apache.org/26/protocol.html

* Successful Jenkins builds for the 2.6 branch:
Unit/integration tests:
https://ci-builds.apache.org/job/Kafka/job/kafka-2.6-jdk8/114/
System tests:
https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4458/

/**

Thanks,
Sophie


Re: [ANNOUNCE] New Committer: Bruno Cadonna

2021-04-07 Thread Sophie Blee-Goldman
Congrats!

On Wed, Apr 7, 2021 at 6:32 PM Luke Chen  wrote:

> Congrats Bruno!!
>
> Luke
>
> On Thu, Apr 8, 2021 at 9:18 AM Matthias J. Sax  wrote:
>
> > Congrats Bruno! Very well deserved!
> >
> >
> > -Matthias
> >
> > On 4/7/21 3:51 PM, Bill Bejeck wrote:
> > > Congrats Bruno! Well deserved.
> > >
> > > Bill
> > >
> > > On Wed, Apr 7, 2021 at 6:34 PM Guozhang Wang 
> wrote:
> > >
> > >> Hello all,
> > >>
> > >> I'm happy to announce that Bruno Cadonna has accepted his invitation
> to
> > >> become an Apache Kafka committer.
> > >>
> > >> Bruno has been contributing to Kafka since Jan. 2019 and has made 99
> > >> commits and more than 80 PR reviews so far:
> > >>
> > >> https://github.com/apache/kafka/commits?author=cadonna
> > >>
> > >> He worked on a few key KIPs on Kafka Streams:
> > >>
> > >> * KIP-471: Expose RocksDB Metrics in Kafka Streams
> > >> * KIP-607: Add Metrics to Kafka Streams to Report Properties of
> RocksDB
> > >> * KIP-662: Throw Exception when Source Topics of a Streams App are
> > Deleted
> > >>
> > >> Besides all the code contributions and reviews, he's also done a
> handful
> > >> for the community: multiple Kafka meetup talks in Berlin and Kafka
> > Summit
> > >> talks, an introductory class to Kafka at Humboldt-Universität zu
> Berlin
> > >> seminars, and have co-authored a paper on Kafka's stream processing
> > >> semantics in this year's SIGMOD conference (
> > >> https://en.wikipedia.org/wiki/SIGMOD). Bruno has also been quite
> > active on
> > >> SO channels and AK mailings.
> > >>
> > >> Please join me to congratulate Bruno for all the contributions!
> > >>
> > >> -- Guozhang
> > >>
> > >
> >
>


Re: [kafka-clients] [VOTE] 2.6.2 RC0

2021-03-31 Thread Sophie Blee-Goldman
Hey Georg,

I'm currently working out the issues with the licensing that Justin brought
up earlier along with the
other release manager(s). I expect to be back on track with a new RC under
vote by the beginning
of next week.

Here's the ticket for anyone interested in following along:
https://issues.apache.org/jira/browse/KAFKA-12593

Sorry for the delay. I appreciate everyone's patience
Sophie

On Wed, Mar 31, 2021 at 11:19 AM Georg Friedrich <
georg.friedr...@webfleet.com> wrote:

> Hi Sophie,
>
> Is there a new RC yet?
> Thanks for driving the release.
>
> Kind regards
> Georg
>
> -----Original Message-
> From: Sophie Blee-Goldman 
> Sent: Friday, March 19, 2021 11:44 PM
> To: Bruno Cadonna 
> Cc: dev ; users@kafka.apache.org;
> kafka-clie...@googlegroups.com
> Subject: Re: [kafka-clients] [VOTE] 2.6.2 RC0
>
> Thanks Bruno. I agree this qualifies as a blocker since it was a
> regression in 2.6 and may result in data loss. I'll roll a new RC with the
> fix
>
> On Fri, Mar 19, 2021 at 7:03 AM 'Bruno Cadonna' via kafka-clients <
> kafka-clie...@googlegroups.com> wrote:
>
> > Hi Sophie,
> >
> > Correction to my last e-mail: The bug does not break eos, but it
> > breaks at-least-once.
> >
> > Bruno
> >
> > On 19.03.21 14:54, Bruno Cadonna wrote:
> > > Hi Sophie,
> > >
> > > Please have a look at the following bug report:
> > >
> > > https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fis
> > > sues.apache.org%2Fjira%2Fbrowse%2FKAFKA-12508data=04%7C01%7Cgeo
> > > rg.friedrich%40webfleet.com%7C8d33e66f7ba44fb9dd3508d8eb29abc1%7Ce64
> > > 8a6341151497c97970f975bddecc0%7C0%7C1%7C637517911498453914%7CUnknown
> > > %7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiL
> > > CJXVCI6Mn0%3D%7C3000sdata=WgVKtlGmytznQJfFwEHzyjLrNgCmG9tM%2BQs
> > > ASEsKfSw%3Dreserved=0
> > >
> > > I set its priority to blocker since the bug might break
> > > at-least-once and exactly-once processing guarantees.
> > >
> > > Feel free to set it back to major, if you think that it is not a
> blocker.
> > >
> > > Best,
> > > Bruno
> > >
> > > On 12.03.21 19:47, 'Sophie Blee-Goldman' via kafka-clients wrote:
> > >> Hello Kafka users, developers and client-developers,
> > >>
> > >> This is the first candidate for release of Apache Kafka 2.6.2.
> > >>
> > >> Apache Kafka 2.6.2 is a bugfix release and fixes 30 issues since
> > >> the
> > >> 2.6.1 release. Please see the release notes for more information.
> > >>
> > >> Release notes for the 2.6.2 release:
> > >>
> > https://eur01.safelinks.protection.outlook.com/?url=https:%2F%2Fhome.a
> > pache.org%2F~ableegoldman%2Fkafka-2.6.2-rc0%2FRELEASE_NOTES.htmld
> > ata=04%7C01%7Cgeorg.friedrich%40webfleet.com%7C8d33e66f7ba44fb9dd3508d
> > 8eb29abc1%7Ce648a6341151497c97970f975bddecc0%7C0%7C1%7C637517911498453
> > 914%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTi
> > I6Ik1haWwiLCJXVCI6Mn0%3D%7C3000sdata=WGf1CFpIHfMbWogRS2VxBfZYVso9
> > xmQ6t2r7qTcHrqY%3Dreserved=0
> > >> <
> > https://eur01.safelinks.protection.outlook.com/?url=https:%2F%2Fhome.a
> > pache.org%2F~ableegoldman%2Fkafka-2.6.2-rc0%2FRELEASE_NOTES.htmld
> > ata=04%7C01%7Cgeorg.friedrich%40webfleet.com%7C8d33e66f7ba44fb9dd3508d
> > 8eb29abc1%7Ce648a6341151497c97970f975bddecc0%7C0%7C1%7C637517911498453
> > 914%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTi
> > I6Ik1haWwiLCJXVCI6Mn0%3D%7C3000sdata=WGf1CFpIHfMbWogRS2VxBfZYVso9
> > xmQ6t2r7qTcHrqY%3Dreserved=0>
> > >>
> > >>
> > >> *** Please download, test and vote by Friday, March 19th, 9am PST
> > >>
> > >> Kafka's KEYS file containing PGP keys we use to sign the release:
> > >> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fk
> > >> afka.apache.org%2FKEYSdata=04%7C01%7Cgeorg.friedrich%40webflee
> > >> t.com%7C8d33e66f7ba44fb9dd3508d8eb29abc1%7Ce648a6341151497c97970f97
> > >> 5bddecc0%7C0%7C1%7C637517911498453914%7CUnknown%7CTWFpbGZsb3d8eyJWI
> > >> joiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C300
> > >> 0sdata=Y57BmucAl7XwL059okLRA%2FLLHzry2rSOZl11iQygEu0%3Dre
> > >> served=0
> > >> <https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2F
> > >> kafka.apache.org%2FKEYSdata=04%7C01%7Cgeorg.friedrich%40webfle
> > >> et.com%7C8d33e66f7ba44fb9dd3508d8eb29abc1%7Ce

Re: Redis as state store

2021-03-23 Thread Sophie Blee-Goldman
>
> it seems to me that local state stores might prove more performant in the
> event of above
> mentioned failures of streams application

Note, the scenario I was describing with deleting all record from the
remote store in event of
failure is *not* what we do today -- I was describing a potential solution
to the problem of EOS
with remote storage. As of the current code, local storage is actually the
one which would wipe
out the state stores and need to restore from scratch. Granted, I'm
actually not sure how this
would be handled with EOS -- most likely we would still end up restoring
from the changelog,
but we just wouldn't wipe out the state stores.

But the point here is that EOS with remote storage is broken -- if your use
case requires
exactly-once semantics, you *must* use local state stores. If you don't
care about occasionally
processing a record twice, or consider steady performance more important
than this, then just
don't use EOS in the first place.

Note: Streams doesn't provide an RPC layer itself, it's up to you to
implement, but yes Streams
does support query state stores through RPC. But this feature (Interactive
Queries/IQ) is for
querying the state from outside the Kafka Streams application, it's not
intended to be used
from within a processor. In general remote calls inside a processor are
somewhat discouraged,
since it's difficult to handle failed calls. And remote calls will also
always break EOS, if that
matters to you.

All that said, I guess you *could* use IQ to query the state stores on
another instance from within
a processor, but I highly doubt this would solve the problem you're facing:
if the GlobalKTable
isn't syncing fast enough for you then I wouldn't expect updates in another
instance to be
available when you need them. This depends on what you're trying to do and
the flow of data
on the input topics, but I get the sense there's probably a better way to
do what you're trying to
achieve. I'm not sure why you would want a GlobalKTable for example -- why
not just have
one application write the results to an output topic, and then let the
downstream application
read in that topic as a table? You should be able to have full control over
things that way

On Sun, Mar 21, 2021 at 5:58 AM Pushkar Deole  wrote:

> Thanks Sophie... i was just thinking what would be a good options for us,
> whether local state stores or redis state store, and it seems to me that
> local state stores might prove more performant in the event of above
> mentioned failures of streams application.
>
> The main reason we are thinking of moving to redis state stores is because:
> we want each application to have access to state saved by other stream
> application instance. We tried to use a GlobalKTable backed by topic and
> each instance would save to that topic which is then synchronized to
> GlobalKTable in each application instance. However making GlobalKTable in
> each application instance took around 100ms and before that time the next
> event might need to get processed by application instance in which case it
> did not have proper state.
>
> I was also looking at some options today available with local state store
> and came across that kafka also provides an RPC layer on top of state store
> which allows a steam application to query state stored in local state store
> of another stream application instance. Is that correct? If so then we can
> try that option instead of redis state store. Let me know what you think
>
> On Sun, Mar 21, 2021 at 6:38 AM Sophie Blee-Goldman
>  wrote:
>
> > Yes, if you have to restore from the changelog from scratch then this
> will
> > definitely impact
> > the application's performance. This is the current state of things for
> EOS
> > applications that use
> > some kind of local storage such as the in-memory or rocksdb state
> > stores.The point of EOS is
> > to be 100% correct, not to maximize performance -- it's a tradeoff and
> you
> > need to decide what
> > characteristics are most important for the specific application and use
> > case.
> >
> > That said, obviously better performance is always a good thing when it's
> > possible to do without
> > sacrificing processing semantics. That's why I proposed to buffer
> updates;
> > if we can avoid dirtying
> > the store in the first place, then there's no need to wipe out all the
> > state and rebuild from the changelog
> > from scratch. So yes, this was intended as an alternative proposal which
> > would improve the performance
> > for any EOS application regardless of whether it uses local or remote
> > storage.
> >
> > But as with all things, this has tradeoffs of its own: for one thing it's
> > probably a significantly larger
> > effort to implement, so if we want to cor

Re: Redis as state store

2021-03-20 Thread Sophie Blee-Goldman
Yes, if you have to restore from the changelog from scratch then this will
definitely impact
the application's performance. This is the current state of things for EOS
applications that use
some kind of local storage such as the in-memory or rocksdb state
stores.The point of EOS is
to be 100% correct, not to maximize performance -- it's a tradeoff and you
need to decide what
characteristics are most important for the specific application and use
case.

That said, obviously better performance is always a good thing when it's
possible to do without
sacrificing processing semantics. That's why I proposed to buffer updates;
if we can avoid dirtying
the store in the first place, then there's no need to wipe out all the
state and rebuild from the changelog
from scratch. So yes, this was intended as an alternative proposal which
would improve the performance
for any EOS application regardless of whether it uses local or remote
storage.

But as with all things, this has tradeoffs of its own: for one thing it's
probably a significantly larger
effort to implement, so if we want to correct the EOS + remote storage
situation quickly then this
approach would not be the best way to go. Also, buffering updates of course
requires additional
resources (ie storage and/or memory), so some users may actually prefer to
take an occasional
performance hit to keep their app lightweight.

Anyways, these are just some thoughts on how to improve the current
situation. Maybe there are
even more options to address this problem which haven't been considered
yet. Let us know if you
have a better idea :)

On Fri, Mar 19, 2021 at 11:50 PM Pushkar Deole  wrote:

> Thanks Sophie... that answers my question, however still worried about some
> other aspects:
>
> 1. If redis is to be restored from changelog topic: what would happen if i
> have 3 stream applications and 1 instance went down ... will other 2
> instances halt until entire existing state from redis is wiped out and
> entire state is restored back from changelog topic? If so then it would
> have a significant performance hit especially when this happens during
> heavy traffic hours
>
> 2. Will #1 be solved by the 2nd alternative that you mentioned in the
> comment i.e 'An alternative is to just start buffering updates in-memory
> (or in rocksdb, this could be configurable) and then avoid dirtying the
> remote storage in the first place as we would only flush the data out to it
> during a commit'  It looks to me that this won't need rebuilding entire
> state store because changelog is disabled, and this alternative would avoid
> making the state store inconsistent in first place, thus saving wipe out
> and rebuild ? If so then this also doesn't need to halt other stream
> applications and would prove much more better approach from performance
> point of view. Is that correct?
>
> On Sat, Mar 20, 2021 at 2:25 AM Sophie Blee-Goldman
>  wrote:
>
> > Hey Pushkar, yes, the data will still be backed by a changelog topic
> unless
> > the
> > user explicitly disables logging for that state store. The fault
> tolerance
> > mechanism
> > of Kafka Streams is based on changelogging, therefore there are no
> > correctness
> > guarantees if you decide to disable it.
> >
> > That said, I'm guessing many users do in fact disable the changelog when
> > plugging
> > in a remote store with it's own fault tolerance guarantees -- is that
> what
> > you're getting
> > at? We could definitely build in better support for that case, as either
> an
> > additional
> > optimization on top of KAFKA-12475
> > <https://issues.apache.org/jira/browse/KAFKA-12475> or as an alternative
> > implementation to fix the
> > underlying EOS problem. Check out my latest comment on the ticket here
> > <
> >
> https://issues.apache.org/jira/browse/KAFKA-12475?focusedCommentId=17305191=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17305191
> > >
> >
> > Does that address your question?
> >
> > On Fri, Mar 19, 2021 at 10:14 AM Pushkar Deole 
> > wrote:
> >
> > > Hello Sophie,
> > >
> > > may be i am missing something here, however can you let me know how a
> > redis
> > > based state store will be wiped off the inconsistent state in case
> stream
> > > application dies in the middle of processing e.g. stream application
> > > consumed from source topic, processed source event and saved state to
> > redis
> > > and before producing event on destination topic, the stream application
> > had
> > > error.
> > > If this occurs with a rocksDB or in-memory state store, it will be
> > rebuilt
> > > from changelog topic, however for red

Re: Kafka Streams And Partitioning

2021-03-19 Thread Sophie Blee-Goldman
Ah ok, I think I was envisioning a different use case from your initial
description of the problem.
If everything that you want to group together is already correctly
partitioned, then you won't need
a repartitioning step. If I understand correctly, you have something like
this in mind:

 builder
.stream("input-topic")
.selectKey((key, value) -> key + extractKeyData(value)); // eg A --> A:B
.groupByKey()
.aggregate(...);
.to("output-topic");  // keys are of the form A:B, but should be
partitioned by A

If that looks about right, then there are two things to watch out for:
1) To keep partitioning on A instead of A:B, you'll need to provide a
custom Partitioner when writing
to the output topic. See Produced#streamPartitioner
2) Since you have a key-changing operation (selectKey) upstream of a
stateful operation (aggregate),
Streams will automatically infer that a repartitioning step is required
and insert one for you.
Unfortunately there's currently no way to force Streams not to do that,
even when you know the
data is going to be partitioned correctly -- there is a ticket for this
but it has yet to be picked up by
anyone. See https://issues.apache.org/jira/browse/KAFKA-4835
I think the only workaround at this point would be to implement the
aggregation step yourself
using the low-level Processor API. Streams will only handle the
repartitioning for DSL operators.

You can still use the DSL for everything else, and mix in the PAPI by
using a transformer. Streams
does not insert repartitions before a transformer since it can't infer
whether or not the operation is
stateful. I know re-implementing the aggregator is a hassle but you
should be able to just copy and
paste much of the existing aggregation code. Check out the
KStreamAggregateProcessor class.

This would look something like this:

 builder
.stream("input-topic")
.selectKey((key, value) -> key + extractKeyData(value)); // eg A --> A:B
.transform(myAggregateProcessorSupplier); // supplier returns a new
processor which implements the aggregation
.to("output-topic", Produced.streamPartitioner(myStreamPartitioner));
// myStreamPartitioner extracts and partitions based on A

Obviously this situation is not ideal -- if you're interested in improving
things, feel free to pick up KAFKA-4835
<https://issues.apache.org/jira/browse/KAFKA-4835>


On Wed, Mar 17, 2021 at 8:19 PM Gareth Collins 
wrote:

> Hi Sophie,
>
> Thanks very much for the response!
>
> So if I understand correctly it will be impossible to avoid the repartition
> topic?
>
> e.g. my original message may have key = A...and will be partitioned on A.
>
> But in my Kafka Streams app, I will want to aggregate on A:B or A:C or A:D
> (B, C or D come from extra key values in the data)...but continue to
> partition on A. Then later
> read via REST all values for A. So to make this work I have to have a
> repartition topic even though I am not really repartitioning (i.e. all
> records for A should still be processed
> together). Is my understanding correct?
>
> So WindowedStreamPartitioner is a special case for avoiding the repartition
> topic?
>
> thanks in advance,
> Gareth
>
> On Wed, Mar 17, 2021 at 7:59 PM Sophie Blee-Goldman
>  wrote:
>
> > Hey Gareth,
> >
> > Kafka Streams state store partitioning is based on the partitioning of
> the
> > upstream input topics.
> > If you want your RocksDB stores to be partitioned based on the prefix of
> a
> > key, then you should
> > make sure the input topic feeding into it uses whatever partitioning
> > strategy you had in mind.
> >
> > If the source topics are user input topics and you have control over the
> > production to these topics,
> > then just use a custom partitioner to produce to them. If you don't have
> > control over them, you can
> > insert an intermediate/repartition topic between the input topics and the
> > subtopology with the RocksDB.
> > Check out the KStream#repartitioned operator, it accepts a Repartitioned
> > which itself accepts a
> > StreamPartitioner that you can use to control the partitioning.
> >
> > You can check out the class WindowedStreamPartitioner for an example:
> this
> > is how we handle the
> > WindowStore case that you pointed out.
> >
> >
> >
> > On Mon, Mar 15, 2021 at 11:45 AM Gareth Collins <
> > gareth.o.coll...@gmail.com>
> > wrote:
> >
> > > Hi,
> > >
> > > This may be a newbie question but is it possible to control the
> > > partitioning of a RocksDB KeyValueStore in Kafka Streams?
> > >
> > > For example, I perhaps only want to partition based on a 

Re: [kafka-clients] [VOTE] 2.7.1 RC0

2021-03-19 Thread Sophie Blee-Goldman
Hey Mickael, I just merged the fix back to 2.7 so you should be good to go

Thanks for the PR Bruno!

On Fri, Mar 19, 2021 at 9:34 AM Mickael Maison 
wrote:

> Thanks Bruno,
>
> That indeed sounds like a blocker.
>
> I'm closing this vote, I'll build a new RC once a fix is merged into 2.7
>
> On Fri, Mar 19, 2021 at 2:04 PM Bruno Cadonna
>  wrote:
> >
> > Hi Mickael,
> >
> > Correction to my last e-mail: The bug does not break eos, but it breaks
> > at-least-once.
> >
> > Bruno
> >
> >
> > On 19.03.21 14:54, Bruno Cadonna wrote:
> > > Hi Mickael,
> > >
> > > Please have a look at the following bug report:
> > >
> > > https://issues.apache.org/jira/browse/KAFKA-12508
> > >
> > > I set its priority to blocker since the bug might break at-least-once
> > > and exactly-once processing guarantees.
> > >
> > > Feel free to set it back to major, if you think that it is not a
> blocker.
> > >
> > > Best,
> > > Bruno
> > >
> > >
> > > On 19.03.21 12:26, Mickael Maison wrote:
> > >> Hello Kafka users, developers and client-developers,
> > >>
> > >> This is the first candidate for release of Apache Kafka 2.7.1.
> > >>
> > >> Apache Kafka 2.7.1 is a bugfix release and 40 issues have been fixed
> > >> since 2.7.0.
> > >>
> > >> Release notes for the 2.7.1 release:
> > >> https://home.apache.org/~mimaison/kafka-2.7.1-rc0/RELEASE_NOTES.html
> > >>
> > >> *** Please download, test and vote by Friday, March 26, 5pm PST
> > >>
> > >> Kafka's KEYS file containing PGP keys we use to sign the release:
> > >> https://kafka.apache.org/KEYS
> > >>
> > >> * Release artifacts to be voted upon (source and binary):
> > >> https://home.apache.org/~mimaison/kafka-2.7.1-rc0/
> > >>
> > >> * Maven artifacts to be voted upon:
> > >>
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >>
> > >> * Javadoc:
> > >> https://home.apache.org/~mimaison/kafka-2.7.1-rc0/javadoc/
> > >>
> > >> * Tag to be voted upon (off 2.7 branch) is the 2.7.1 tag:
> > >> https://github.com/apache/kafka/releases/tag/2.7.1-rc0
> > >>
> > >> * Documentation:
> > >> https://kafka.apache.org/27/documentation.html
> > >>
> > >> * Protocol:
> > >> https://kafka.apache.org/27/protocol.html
> > >>
> > >> * Successful Jenkins builds for the 2.7 branch:
> > >> Unit/integration tests:
> > >> https://ci-builds.apache.org/job/Kafka/job/kafka-2.7-jdk8/135/
> > >>
> > >> /**
> > >>
> > >> Thanks,
> > >> Mickael
> > >>
>
> --
> You received this message because you are subscribed to the Google Groups
> "kafka-clients" group.
> To unsubscribe from this group and stop receiving emails from it, send an
> email to kafka-clients+unsubscr...@googlegroups.com.
> To view this discussion on the web visit
> https://groups.google.com/d/msgid/kafka-clients/CA%2BOCqna8CaMZa%2BiK7u6nv1jNv9q9cXrHYfh5E%3DXLYQ41GWmheA%40mail.gmail.com
> .
>


Re: [kafka-clients] [VOTE] 2.6.2 RC0

2021-03-19 Thread Sophie Blee-Goldman
Thanks Bruno. I agree this qualifies as a blocker since it was a regression
in 2.6 and
may result in data loss. I'll roll a new RC with the fix

On Fri, Mar 19, 2021 at 7:03 AM 'Bruno Cadonna' via kafka-clients <
kafka-clie...@googlegroups.com> wrote:

> Hi Sophie,
>
> Correction to my last e-mail: The bug does not break eos, but it breaks
> at-least-once.
>
> Bruno
>
> On 19.03.21 14:54, Bruno Cadonna wrote:
> > Hi Sophie,
> >
> > Please have a look at the following bug report:
> >
> > https://issues.apache.org/jira/browse/KAFKA-12508
> >
> > I set its priority to blocker since the bug might break at-least-once
> > and exactly-once processing guarantees.
> >
> > Feel free to set it back to major, if you think that it is not a blocker.
> >
> > Best,
> > Bruno
> >
> > On 12.03.21 19:47, 'Sophie Blee-Goldman' via kafka-clients wrote:
> >> Hello Kafka users, developers and client-developers,
> >>
> >> This is the first candidate for release of Apache Kafka 2.6.2.
> >>
> >> Apache Kafka 2.6.2 is a bugfix release and fixes 30 issues since the
> >> 2.6.1 release. Please see the release notes for more information.
> >>
> >> Release notes for the 2.6.2 release:
> >>
> https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/RELEASE_NOTES.html
> >> <
> https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/RELEASE_NOTES.html>
> >>
> >>
> >> *** Please download, test and vote by Friday, March 19th, 9am PST
> >>
> >> Kafka's KEYS file containing PGP keys we use to sign the release:
> >> https://kafka.apache.org/KEYS <https://kafka.apache.org/KEYS>
> >>
> >> * Release artifacts to be voted upon (source and binary):
> >> https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/
> >> <https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/>
> >>
> >> * Maven artifacts to be voted upon:
> >> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >> <https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> >>
> >> * Javadoc:
> >> https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/javadoc/
> >> <https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/javadoc/>
> >>
> >> * Tag to be voted upon (off 2.6 branch) is the 2.6.2 tag:
> >> https://github.com/apache/kafka/releases/tag/2.6.2-rc0
> >> <https://github.com/apache/kafka/releases/tag/2.6.2-rc0>
> >>
> >> * Documentation:
> >> https://kafka.apache.org/26/documentation.html
> >> <https://kafka.apache.org/26/documentation.html>
> >>
> >> * Protocol:
> >> https://kafka.apache.org/26/protocol.html
> >> <https://kafka.apache.org/26/protocol.html>
> >>
> >> * Successful Jenkins builds for the 2.6 branch:
> >> Unit/integration tests:
> >> https://ci-builds.apache.org/job/Kafka/job/kafka-2.6-jdk8/105/
> >> <https://ci-builds.apache.org/job/Kafka/job/kafka-2.6-jdk8/105/>
> >> System tests:
> >> https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4435/
> <https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4435/>
> >>
> >>
> >> /**
> >>
> >> Thanks,
> >> Sophie
> >>
> >> --
> >> You received this message because you are subscribed to the Google
> >> Groups "kafka-clients" group.
> >> To unsubscribe from this group and stop receiving emails from it, send
> >> an email to kafka-clients+unsubscr...@googlegroups.com
> >> <mailto:kafka-clients+unsubscr...@googlegroups.com>.
> >> To view this discussion on the web visit
> >>
> https://groups.google.com/d/msgid/kafka-clients/CAFLS_9jkHGsj42DT7Og3%3Dov9RbO%3DbEAQX55h0L6YKHJQR9qJpw%40mail.gmail.com
> >> <
> https://groups.google.com/d/msgid/kafka-clients/CAFLS_9jkHGsj42DT7Og3%3Dov9RbO%3DbEAQX55h0L6YKHJQR9qJpw%40mail.gmail.com?utm_medium=email_source=footer>.
>
> >>
>
> --
> You received this message because you are subscribed to the Google Groups
> "kafka-clients" group.
> To unsubscribe from this group and stop receiving emails from it, send an
> email to kafka-clients+unsubscr...@googlegroups.com.
> To view this discussion on the web visit
> https://groups.google.com/d/msgid/kafka-clients/b1ea7ea3-f7aa-1ccb-eedd-6f2b4c254d1e%40confluent.io
> .
>


Re: Redis as state store

2021-03-19 Thread Sophie Blee-Goldman
Hey Pushkar, yes, the data will still be backed by a changelog topic unless
the
user explicitly disables logging for that state store. The fault tolerance
mechanism
of Kafka Streams is based on changelogging, therefore there are no
correctness
guarantees if you decide to disable it.

That said, I'm guessing many users do in fact disable the changelog when
plugging
in a remote store with it's own fault tolerance guarantees -- is that what
you're getting
at? We could definitely build in better support for that case, as either an
additional
optimization on top of KAFKA-12475
<https://issues.apache.org/jira/browse/KAFKA-12475> or as an alternative
implementation to fix the
underlying EOS problem. Check out my latest comment on the ticket here
<https://issues.apache.org/jira/browse/KAFKA-12475?focusedCommentId=17305191=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17305191>

Does that address your question?

On Fri, Mar 19, 2021 at 10:14 AM Pushkar Deole  wrote:

> Hello Sophie,
>
> may be i am missing something here, however can you let me know how a redis
> based state store will be wiped off the inconsistent state in case stream
> application dies in the middle of processing e.g. stream application
> consumed from source topic, processed source event and saved state to redis
> and before producing event on destination topic, the stream application had
> error.
> If this occurs with a rocksDB or in-memory state store, it will be rebuilt
> from changelog topic, however for redis state store, how it will wiped off
> the state ? are we saying here that the data stored in redis will still be
> backed by changelog topic and redis will be restored from backed topic in
> case of stream application error?
>
> On Tue, Mar 16, 2021 at 12:18 AM Sophie Blee-Goldman
>  wrote:
>
> > This certainly does seem like a flaw in the Streams API, although of
> course
> > Streams is just
> > in general not designed for use with remote anything (API calls, stores,
> > etc)
> >
> > That said, I don't see any reason why we *couldn't* have better support
> for
> > remote state stores.
> > Note that there's currently no deleteAll method on the store interface,
> and
> > not all databases
> > necessarily support that. But we could add a default implementation which
> > just calls delete(key)
> > on all keys in the state store, and for the RocksDB-based state stores we
> > still wipe out the state
> > as usual (and recommend the same for any custom StateStore which is local
> > rather than remote).
> > Obviously falling back on individual delete(key) operations for all the
> > data in the entire store will
> > have worse performance, but that's better than silently breaking EOS when
> > deleteAll is not available
> > on a remote store.
> >
> > Would you be interested in doing a KIP for this? We should also file a
> JIRA
> > with the above explanation,
> > so that other users of remote storage are aware of this limitation. And
> > definitely document this somewhere
> >
> >
> > On Mon, Mar 15, 2021 at 8:04 AM Bruno Cadonna  >
> > wrote:
> >
> > > Hi Alex,
> > >
> > > I guess wiping out the state directory is easier code-wise, faster,
> > > and/or at the time of development the developers did not design for
> > > remote state stores. But I do actually not know the exact reason.
> > >
> > > Off the top of my head, I do not know how to solve this for remote
> state
> > > stores. Using the uncaught exception handler is not good, because a
> > > computing node could fail without giving the JVM the opportunity to
> > > throw an exception.
> > >
> > > In your tests, try to increase the commit interval to a high value and
> > > see if you get inconsistencies. You should get an inconsistency if the
> > > state store maintains counts for keys and after the last commit before
> > > the failure, the Streams app puts an event with a new key K with value
> 1
> > > into the state store. After failover, Streams would put the same event
> > > with key K again into the state store. If the state store deleted all
> of
> > > its data, Streams would put again value 1, but if the state store did
> > > not delete all data, Streams would put value 2 which is wrong because
> it
> > > would count the same event twice.
> > >
> > > Best,
> > > Bruno
> > >
> > >
> > > On 15.03.21 15:20, Alex Craig wrote:
> > > > Bruno,
> > > > Thanks for the info!  that makes sense.  Of course now I have more
> > > > questio

Re: Kafka Streams And Partitioning

2021-03-17 Thread Sophie Blee-Goldman
Hey Gareth,

Kafka Streams state store partitioning is based on the partitioning of the
upstream input topics.
If you want your RocksDB stores to be partitioned based on the prefix of a
key, then you should
make sure the input topic feeding into it uses whatever partitioning
strategy you had in mind.

If the source topics are user input topics and you have control over the
production to these topics,
then just use a custom partitioner to produce to them. If you don't have
control over them, you can
insert an intermediate/repartition topic between the input topics and the
subtopology with the RocksDB.
Check out the KStream#repartitioned operator, it accepts a Repartitioned
which itself accepts a
StreamPartitioner that you can use to control the partitioning.

You can check out the class WindowedStreamPartitioner for an example: this
is how we handle the
WindowStore case that you pointed out.



On Mon, Mar 15, 2021 at 11:45 AM Gareth Collins 
wrote:

> Hi,
>
> This may be a newbie question but is it possible to control the
> partitioning of a RocksDB KeyValueStore in Kafka Streams?
>
> For example, I perhaps only want to partition based on a prefix of a key
> rather than the full key. I assume something similar must be done for the
> WindowStore to partition without the window start time and sequence number
> (otherwise window entries could be spread across partitions)?
>
> Sort of like the window store, I am wanting to be able to retrieve all
> values with a certain key prefix from the KeyValueStore with one read
> operation.
>
> Is this possible?
>
> thanks in advance,
> Gareth Collins
>


Re: Slightly Modified Sticky Assignor.

2021-03-16 Thread Sophie Blee-Goldman
Just FYI, I filed a ticket to improve the Consumer#committed API. You can
track this on https://issues.apache.org/jira/browse/KAFKA-12485

On Tue, Mar 16, 2021 at 2:16 AM Mazen Ezzeddine <
mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:

> Great, thanks so much for the detailed answer.
>
> Best,
> ________
> From: Sophie Blee-Goldman 
> Sent: Tuesday, March 16, 2021 5:58 AM
> To: users@kafka.apache.org 
> Subject: Re: Slightly Modified Sticky Assignor.
>
> Hey Mazen,
>
> The easiest way to approach this is probably to pass in a reference to the
> associated Consumer and
> then just call one of the *Consumer#committed *methods which return the
> OffsetAndMetadata.
>
> But I'm guessing your underling question may be about how to get that
> reference to the Consumer in
> the first place. There really isn't very good support for this in the
> ConsumerPartitionAssignor interface
> since it relies on reflection to instantiate the assignor with the default
> constructor. I would recommend
> making your custom ConsumerPartitionAssignor implement the Configurable
> interface, and then use
> the configs you pass in to the Consumer to ultimately get a handle on it.
> Since you have to provide the
> configs to construct the Consumer in the first place, you might need a
> layer of indirection: for example
>
> class ConsumerProvider {
> private Consumer consumer;
>
> public void setConsumer(Consumer consumer);
>
> public Consumer getConsumer();
> }
>
> // main code
> ConsumerProvider provider = new ConsumerProvider;
> consumerConfig.put("MY_CONSUMER_PROVIDER", provider);
> Consumer consumer = new KafkaConsumer(consumerConfig, ...);
> provider.setConsumer(consumer);
>
> class MyAssignor implements ConsumerPArtitionAssignor, Configurable {
>
> private ConsumerProvider;
>
> @Override
> public void configure(configs) {
> this.consumerProvider = configs.get("MY_CONSUMER_PROVIDER");
> }
>
> @Override
> ByteBuffer subscriptionUserData(topics) {
> offsets = consumerProvider.getConsumer().committed(...);
> }
> }
>
> Just a warning, I haven't actually tested this out, but the general idea of
> using configs should work.
>
> I know you said "seamless" and this is anything but :/ Maybe I'm tired and
> missing something obvious, but
> clearly there's room for improvement here. You can file a JIRA ticket to
> improve the partition assignor
> experience and make it easier to set up (and even work on this yourself if
> you're interested)
>
> Unfortunately you generally want to keep the subscriptionUserData() method
> pretty light, and this
> method will make a remote call to the server. To be honest, I'm not totally
> sure why that is the case, since
> the Consumer should know what offsets it's committed for which
> partitions...maybe someone else can
> jump in on the choice behind that. This has come up before, so it's worth
> investigating whether we can
> just return the cached offsets if they're available and only make a remote
> call for *Consumer#committed* if
> absolutely necessary. I'll try to follow up on that
>
> On Tue, Mar 9, 2021 at 9:26 AM Mazen Ezzeddine <
> mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:
>
> > Hi all,
> >
> > I am implementing a custom partition assignor slightly different than the
> > sticky assignor  assignor. As known, in the sticky assignor each consumer
> > sends the set of owned partitions as part of its subscription message.
> This
> > happens in the subscriptionUserData by calling the
> > serializeTopicPartitionAssignment method etc…
> >
> > Kindly, my question is that what is the most seamless way to get offset
> > information (e.g., last committed offset) for each owned partition from
> > within the subscriptionUserData method or generally from within the
> > stickyAssignor class, preferably using public APIs.
> >
> > Thank you.
> >
>


Re: Kafka custom partition - consumers assignor with custom per partition user/custom data

2021-03-15 Thread Sophie Blee-Goldman
I believe I already answered your question in another channel, but just to
follow up in this thread in case anyone else is interested in the answer:

You can override the *ConsumerPartitionAssignor.onAssignment(Assignment,
ConsumerGroupMetadata)*  method to get an update on the currently
assigned partitions after each rebalance. The *Assignment* parameter
contains two fields: the assigned partitions, and the serialized assignment
userdata, if any.

On Sat, Mar 6, 2021 at 6:34 AM Mazen Ezzeddine <
mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:

> Hi all,
>
> I am implementing a custom  consumers to topics/partitions assignor in
> Kafka. To this end, I am overriding the AbstractPartitionAssignor which in
> turn implements the  ConsumerPartitionAssignor interface.
>
> As part of the custom assignor, I want to send a single (float)
> information about each partition of each topic that the consumer subscribes
> to.
>
> I am aware that I can send custom data to the assignor by overriding the
> default method
> ByteBuffer subscriptionUserData(Set topics) .
>
> However, the issue is that from the method signature above I can not get
> the list of partitions assigned to the underlined consumer for each of the
> topics that the consumer registers for.
>
> On the other hand, I can see that the subscription class sent by each of
> the consumers to the group coordinator has list of the owned partitions per
> consumer.
> public static final class Subscription {
> private final List topics;
> private final ByteBuffer userData;
> private final List ownedPartitions;
> private Optional groupInstanceId;
>
>
> Any hint on how I can send custom per partition data to the group
> coordinator through the method ByteBuffer subscriptionUserData(Set
> topics), or using any other way that relies only on the kafka public APIs.
>
> Thank you.
>
>


Re: Slightly Modified Sticky Assignor.

2021-03-15 Thread Sophie Blee-Goldman
Hey Mazen,

The easiest way to approach this is probably to pass in a reference to the
associated Consumer and
then just call one of the *Consumer#committed *methods which return the
OffsetAndMetadata.

But I'm guessing your underling question may be about how to get that
reference to the Consumer in
the first place. There really isn't very good support for this in the
ConsumerPartitionAssignor interface
since it relies on reflection to instantiate the assignor with the default
constructor. I would recommend
making your custom ConsumerPartitionAssignor implement the Configurable
interface, and then use
the configs you pass in to the Consumer to ultimately get a handle on it.
Since you have to provide the
configs to construct the Consumer in the first place, you might need a
layer of indirection: for example

class ConsumerProvider {
private Consumer consumer;

public void setConsumer(Consumer consumer);

public Consumer getConsumer();
}

// main code
ConsumerProvider provider = new ConsumerProvider;
consumerConfig.put("MY_CONSUMER_PROVIDER", provider);
Consumer consumer = new KafkaConsumer(consumerConfig, ...);
provider.setConsumer(consumer);

class MyAssignor implements ConsumerPArtitionAssignor, Configurable {

private ConsumerProvider;

@Override
public void configure(configs) {
this.consumerProvider = configs.get("MY_CONSUMER_PROVIDER");
}

@Override
ByteBuffer subscriptionUserData(topics) {
offsets = consumerProvider.getConsumer().committed(...);
}
}

Just a warning, I haven't actually tested this out, but the general idea of
using configs should work.

I know you said "seamless" and this is anything but :/ Maybe I'm tired and
missing something obvious, but
clearly there's room for improvement here. You can file a JIRA ticket to
improve the partition assignor
experience and make it easier to set up (and even work on this yourself if
you're interested)

Unfortunately you generally want to keep the subscriptionUserData() method
pretty light, and this
method will make a remote call to the server. To be honest, I'm not totally
sure why that is the case, since
the Consumer should know what offsets it's committed for which
partitions...maybe someone else can
jump in on the choice behind that. This has come up before, so it's worth
investigating whether we can
just return the cached offsets if they're available and only make a remote
call for *Consumer#committed* if
absolutely necessary. I'll try to follow up on that

On Tue, Mar 9, 2021 at 9:26 AM Mazen Ezzeddine <
mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:

> Hi all,
>
> I am implementing a custom partition assignor slightly different than the
> sticky assignor  assignor. As known, in the sticky assignor each consumer
> sends the set of owned partitions as part of its subscription message. This
> happens in the subscriptionUserData by calling the
> serializeTopicPartitionAssignment method etc…
>
> Kindly, my question is that what is the most seamless way to get offset
> information (e.g., last committed offset) for each owned partition from
> within the subscriptionUserData method or generally from within the
> stickyAssignor class, preferably using public APIs.
>
> Thank you.
>


Re: [VOTE] 2.6.2 RC0

2021-03-15 Thread Sophie Blee-Goldman
Thanks Luke. I'll make sure to bump this in the kafka-site repo once the
release is finalized.

On Mon, Mar 15, 2021 at 8:58 PM Luke Chen  wrote:

> Hi Sophie,
> A small doc update for 2.6.2. I think we missed it.
> https://github.com/apache/kafka/pull/10328
>
> Thanks.
> Luke
>
> On Tue, Mar 16, 2021 at 11:12 AM Guozhang Wang  wrote:
>
> > Hi Sophie,
> >
> > I've reviewed the javadocs / release notes / documentations, and they
> LGTM.
> >
> > +1.
> >
> >
> > Guozhang
> >
> > On Fri, Mar 12, 2021 at 10:48 AM Sophie Blee-Goldman
> >  wrote:
> >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the first candidate for release of Apache Kafka 2.6.2.
> > >
> > > Apache Kafka 2.6.2 is a bugfix release and fixes 30 issues since the
> > 2.6.1
> > > release. Please see the release notes for more information.
> > >
> > > Release notes for the 2.6.2 release:
> > >
> https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Friday, March 19th, 9am PST
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > https://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/javadoc/
> > >
> > > * Tag to be voted upon (off 2.6 branch) is the 2.6.2 tag:
> > > https://github.com/apache/kafka/releases/tag/2.6.2-rc0
> > >
> > > * Documentation:
> > > https://kafka.apache.org/26/documentation.html
> > >
> > > * Protocol:
> > > https://kafka.apache.org/26/protocol.html
> > >
> > > * Successful Jenkins builds for the 2.6 branch:
> > > Unit/integration tests:
> > > https://ci-builds.apache.org/job/Kafka/job/kafka-2.6-jdk8/105/
> > > System tests:
> > >
> https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4435/
> > >
> > > /**
> > >
> > > Thanks,
> > > Sophie
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


Re: Redis as state store

2021-03-15 Thread Sophie Blee-Goldman
Yep, that fell off my radar.  Here we go:
https://issues.apache.org/jira/browse/KAFKA-12475

On Mon, Mar 15, 2021 at 8:09 PM Guozhang Wang  wrote:

> Hey Sophie,
>
> Maybe we can first create a JIRA ticket for this?
>
> On Mon, Mar 15, 2021 at 3:09 PM Sophie Blee-Goldman
>  wrote:
>
> > Sounds good! I meant anyone who is interested :)
> >
> > Let me know if you have any questions after digging in to this
> >
> > On Mon, Mar 15, 2021 at 2:39 PM Alex Craig 
> wrote:
> >
> > >  Hey Sophie, not sure if you meant me or not but I'd be happy to take a
> > > stab at creating a KIP for this.  I want to spend some time digging
> into
> > > more of how this works first, but will then try to gather my thoughts
> and
> > > get something created.
> > >
> > > Alex
> > >
> > > On Mon, Mar 15, 2021 at 1:48 PM Sophie Blee-Goldman
> > >  wrote:
> > >
> > > > This certainly does seem like a flaw in the Streams API, although of
> > > course
> > > > Streams is just
> > > > in general not designed for use with remote anything (API calls,
> > stores,
> > > > etc)
> > > >
> > > > That said, I don't see any reason why we *couldn't* have better
> support
> > > for
> > > > remote state stores.
> > > > Note that there's currently no deleteAll method on the store
> interface,
> > > and
> > > > not all databases
> > > > necessarily support that. But we could add a default implementation
> > which
> > > > just calls delete(key)
> > > > on all keys in the state store, and for the RocksDB-based state
> stores
> > we
> > > > still wipe out the state
> > > > as usual (and recommend the same for any custom StateStore which is
> > local
> > > > rather than remote).
> > > > Obviously falling back on individual delete(key) operations for all
> the
> > > > data in the entire store will
> > > > have worse performance, but that's better than silently breaking EOS
> > when
> > > > deleteAll is not available
> > > > on a remote store.
> > > >
> > > > Would you be interested in doing a KIP for this? We should also file
> a
> > > JIRA
> > > > with the above explanation,
> > > > so that other users of remote storage are aware of this limitation.
> And
> > > > definitely document this somewhere
> > > >
> > > >
> > > > On Mon, Mar 15, 2021 at 8:04 AM Bruno Cadonna
> >  > > >
> > > > wrote:
> > > >
> > > > > Hi Alex,
> > > > >
> > > > > I guess wiping out the state directory is easier code-wise, faster,
> > > > > and/or at the time of development the developers did not design for
> > > > > remote state stores. But I do actually not know the exact reason.
> > > > >
> > > > > Off the top of my head, I do not know how to solve this for remote
> > > state
> > > > > stores. Using the uncaught exception handler is not good, because a
> > > > > computing node could fail without giving the JVM the opportunity to
> > > > > throw an exception.
> > > > >
> > > > > In your tests, try to increase the commit interval to a high value
> > and
> > > > > see if you get inconsistencies. You should get an inconsistency if
> > the
> > > > > state store maintains counts for keys and after the last commit
> > before
> > > > > the failure, the Streams app puts an event with a new key K with
> > value
> > > 1
> > > > > into the state store. After failover, Streams would put the same
> > event
> > > > > with key K again into the state store. If the state store deleted
> all
> > > of
> > > > > its data, Streams would put again value 1, but if the state store
> did
> > > > > not delete all data, Streams would put value 2 which is wrong
> because
> > > it
> > > > > would count the same event twice.
> > > > >
> > > > > Best,
> > > > > Bruno
> > > > >
> > > > >
> > > > > On 15.03.21 15:20, Alex Craig wrote:
> > > > > > Bruno,
> > > > > > Thanks for the info!  that makes sense.  Of course now I have
> more
> > > > > > questions.  :)  Do you 

Re: Redis as state store

2021-03-15 Thread Sophie Blee-Goldman
Sounds good! I meant anyone who is interested :)

Let me know if you have any questions after digging in to this

On Mon, Mar 15, 2021 at 2:39 PM Alex Craig  wrote:

>  Hey Sophie, not sure if you meant me or not but I'd be happy to take a
> stab at creating a KIP for this.  I want to spend some time digging into
> more of how this works first, but will then try to gather my thoughts and
> get something created.
>
> Alex
>
> On Mon, Mar 15, 2021 at 1:48 PM Sophie Blee-Goldman
>  wrote:
>
> > This certainly does seem like a flaw in the Streams API, although of
> course
> > Streams is just
> > in general not designed for use with remote anything (API calls, stores,
> > etc)
> >
> > That said, I don't see any reason why we *couldn't* have better support
> for
> > remote state stores.
> > Note that there's currently no deleteAll method on the store interface,
> and
> > not all databases
> > necessarily support that. But we could add a default implementation which
> > just calls delete(key)
> > on all keys in the state store, and for the RocksDB-based state stores we
> > still wipe out the state
> > as usual (and recommend the same for any custom StateStore which is local
> > rather than remote).
> > Obviously falling back on individual delete(key) operations for all the
> > data in the entire store will
> > have worse performance, but that's better than silently breaking EOS when
> > deleteAll is not available
> > on a remote store.
> >
> > Would you be interested in doing a KIP for this? We should also file a
> JIRA
> > with the above explanation,
> > so that other users of remote storage are aware of this limitation. And
> > definitely document this somewhere
> >
> >
> > On Mon, Mar 15, 2021 at 8:04 AM Bruno Cadonna  >
> > wrote:
> >
> > > Hi Alex,
> > >
> > > I guess wiping out the state directory is easier code-wise, faster,
> > > and/or at the time of development the developers did not design for
> > > remote state stores. But I do actually not know the exact reason.
> > >
> > > Off the top of my head, I do not know how to solve this for remote
> state
> > > stores. Using the uncaught exception handler is not good, because a
> > > computing node could fail without giving the JVM the opportunity to
> > > throw an exception.
> > >
> > > In your tests, try to increase the commit interval to a high value and
> > > see if you get inconsistencies. You should get an inconsistency if the
> > > state store maintains counts for keys and after the last commit before
> > > the failure, the Streams app puts an event with a new key K with value
> 1
> > > into the state store. After failover, Streams would put the same event
> > > with key K again into the state store. If the state store deleted all
> of
> > > its data, Streams would put again value 1, but if the state store did
> > > not delete all data, Streams would put value 2 which is wrong because
> it
> > > would count the same event twice.
> > >
> > > Best,
> > > Bruno
> > >
> > >
> > > On 15.03.21 15:20, Alex Craig wrote:
> > > > Bruno,
> > > > Thanks for the info!  that makes sense.  Of course now I have more
> > > > questions.  :)  Do you know why this is being done outside of the
> state
> > > > store API?  I assume there are reasons why a "deleteAll()" type of
> > > function
> > > > wouldn't work, thereby allowing a state store to purge itself?  And
> > maybe
> > > > more importantly, is there a way to achieve a similar behavior with a
> > 3rd
> > > > party store?  I'm not sure if hooking into the uncaught exception
> > handler
> > > > might be a good way to purge/drop a state store in the event of a
> fatal
> > > > error?  I did setup a MongoDB state store recently as part of a POC
> and
> > > was
> > > > testing it with EOS enabled.  (forcing crashes to occur and checking
> > that
> > > > the result of my aggregation was still accurate)  I was unable to
> cause
> > > > inconsistent data in the mongo store (which is good!), though of
> > course I
> > > > may just have been getting lucky.  Thanks again for your help,
> > > >
> > > > Alex
> > > >
> > > > On Mon, Mar 15, 2021 at 8:59 AM Pushkar Deole 
> > > wrote:
> > > >
> > > >> Bruno,
> > > >>
> > > >> i 

Re: Redis as state store

2021-03-15 Thread Sophie Blee-Goldman
This certainly does seem like a flaw in the Streams API, although of course
Streams is just
in general not designed for use with remote anything (API calls, stores,
etc)

That said, I don't see any reason why we *couldn't* have better support for
remote state stores.
Note that there's currently no deleteAll method on the store interface, and
not all databases
necessarily support that. But we could add a default implementation which
just calls delete(key)
on all keys in the state store, and for the RocksDB-based state stores we
still wipe out the state
as usual (and recommend the same for any custom StateStore which is local
rather than remote).
Obviously falling back on individual delete(key) operations for all the
data in the entire store will
have worse performance, but that's better than silently breaking EOS when
deleteAll is not available
on a remote store.

Would you be interested in doing a KIP for this? We should also file a JIRA
with the above explanation,
so that other users of remote storage are aware of this limitation. And
definitely document this somewhere


On Mon, Mar 15, 2021 at 8:04 AM Bruno Cadonna 
wrote:

> Hi Alex,
>
> I guess wiping out the state directory is easier code-wise, faster,
> and/or at the time of development the developers did not design for
> remote state stores. But I do actually not know the exact reason.
>
> Off the top of my head, I do not know how to solve this for remote state
> stores. Using the uncaught exception handler is not good, because a
> computing node could fail without giving the JVM the opportunity to
> throw an exception.
>
> In your tests, try to increase the commit interval to a high value and
> see if you get inconsistencies. You should get an inconsistency if the
> state store maintains counts for keys and after the last commit before
> the failure, the Streams app puts an event with a new key K with value 1
> into the state store. After failover, Streams would put the same event
> with key K again into the state store. If the state store deleted all of
> its data, Streams would put again value 1, but if the state store did
> not delete all data, Streams would put value 2 which is wrong because it
> would count the same event twice.
>
> Best,
> Bruno
>
>
> On 15.03.21 15:20, Alex Craig wrote:
> > Bruno,
> > Thanks for the info!  that makes sense.  Of course now I have more
> > questions.  :)  Do you know why this is being done outside of the state
> > store API?  I assume there are reasons why a "deleteAll()" type of
> function
> > wouldn't work, thereby allowing a state store to purge itself?  And maybe
> > more importantly, is there a way to achieve a similar behavior with a 3rd
> > party store?  I'm not sure if hooking into the uncaught exception handler
> > might be a good way to purge/drop a state store in the event of a fatal
> > error?  I did setup a MongoDB state store recently as part of a POC and
> was
> > testing it with EOS enabled.  (forcing crashes to occur and checking that
> > the result of my aggregation was still accurate)  I was unable to cause
> > inconsistent data in the mongo store (which is good!), though of course I
> > may just have been getting lucky.  Thanks again for your help,
> >
> > Alex
> >
> > On Mon, Mar 15, 2021 at 8:59 AM Pushkar Deole 
> wrote:
> >
> >> Bruno,
> >>
> >> i tried to explain this in 'kafka user's language through above
> mentioned
> >> scenario, hope i put it properly -:) and correct me if i am wrong
> >>
> >> On Mon, Mar 15, 2021 at 7:23 PM Pushkar Deole 
> >> wrote:
> >>
> >>> This is what I understand could be the issue with external state store:
> >>>
> >>> kafka stream application consumes source topic, does processing, stores
> >>> state to kafka state store (this is backed by topic) and before
> producing
> >>> event on destination topic, the application fails with some issue. In
> >>> this case, the transaction has failed, so kafka guarantees either all
> or
> >>> none, means offset written to source topic, state written to state
> store
> >>> topic, output produced on destination topic... all of these happen or
> >> none
> >>> of these and in this failure scenario it is none of these.
> >>>
> >>> Assume you have redis state store, and you updated the state into redis
> >>> and stream application failed. Now, you have source topic and
> destination
> >>> topic consistent i.e. offset is not committed to source topic and
> output
> >>> not produced on destination topic, but you redis state store is
> >>> inconsistent with that since it is external state store and kafka can't
> >>> guarantee rollback ot state written there
> >>>
> >>> On Mon, Mar 15, 2021 at 6:30 PM Alex Craig 
> >> wrote:
> >>>
>  " Another issue with 3rd party state stores could be violation of
>  exactly-once guarantee provided by kafka streams in the event of a
> >> failure
>  of streams application instance"
> 
>  I've heard this before but would love to know more about how a custom
>  state
> 

Re: Rebalancing and scaling of consumers on kubernetes, instanteous scale to x consumer replicas ==> x rebalancing?

2021-03-15 Thread Sophie Blee-Goldman
Hey Mazen,

There's not necessarily one rebalance per new consumer, in theory if all
100 consumers are started up at the same time then there
may be just a single rebalance. It really depends on the timing -- for
example in the log snippet you provided, you can see that the
first member joined at 13:57:34 and the rebalance completed ~3 seconds
later at 13:57:37. Then the second member is seen joining
at 14:07:43, which is just over 10 minutes later. I think you need to
investigate why there's such a long delay between the first and
second consumers joining the group.


On Mon, Mar 15, 2021 at 7:58 AM Mazen Ezzeddine <
mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:

> Hi all,
>
> I have a kafka consumer  pod running on kubernetes, I executed the command
> kubectl scale consumerName --replicas=2,  and as shown in the logs below
> two seperate rebalancing processes were trigerred, so if the number of
> consumer replicas scaled = 100, one hundred seperate rebalancing are going
> to be trigerred.  is that accurate? am I missing something? any workaroud
> to trigger a single rebalancing regardless of the number of replicas in the
> scale command.
>
>
> group coordinator logs
> =
>
>
> 2021-03-15 13:57:34,230 INFO [GroupCoordinator 1]: Preparing to rebalance
> group debugconsumerlag in state PreparingRebalance with old generation 0
> (__consumer_offsets-31) (reason: Adding new member
> consumer-debugconsumerlag-1-1a577d6c-7389-4217-883f-89535032ae02 with group
> instance id None) (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-5]
> 2021-03-15 13:57:37,266 INFO [GroupCoordinator 1]: Stabilized group
> debugconsumerlag generation 1 (__consumer_offsets-31)
> (kafka.coordinator.group.GroupCoordinator) [executor-Rebalance]
> 2021-03-15 13:57:37,784 INFO [GroupCoordinator 1]: Assignment received
> from leader for group debugconsumerlag for generation 1
> (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-3]
> 2021-03-15 14:07:43,822 INFO [GroupCoordinator 1]: Preparing to rebalance
> group debugconsumerlag in state PreparingRebalance with old generation 1
> (__consumer_offsets-31) (reason: Adding new member
> consumer-debugconsumerlag-1-e2e57bf6-6cbc-4dba-81d4-d7e58219c23f with group
> instance id None) (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-1]
> 2021-03-15 14:07:46,530 INFO [GroupCoordinator 1]: Stabilized group
> debugconsumerlag generation 2 (__consumer_offsets-31)
> (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-1]
> 2021-03-15 14:07:46,675 INFO [GroupCoordinator 1]: Assignment received
> from leader for group debugconsumerlag for generation 2
> (kafka.coordinator.group.GroupCoordinator)
> [data-plane-kafka-request-handler-3]
>
> Kind regards,
>


[VOTE] 2.6.2 RC0

2021-03-12 Thread Sophie Blee-Goldman
Hello Kafka users, developers and client-developers,

This is the first candidate for release of Apache Kafka 2.6.2.

Apache Kafka 2.6.2 is a bugfix release and fixes 30 issues since the 2.6.1
release. Please see the release notes for more information.

Release notes for the 2.6.2 release:
https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/RELEASE_NOTES.html

*** Please download, test and vote by Friday, March 19th, 9am PST

Kafka's KEYS file containing PGP keys we use to sign the release:
https://kafka.apache.org/KEYS

* Release artifacts to be voted upon (source and binary):
https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/

* Maven artifacts to be voted upon:
https://repository.apache.org/content/groups/staging/org/apache/kafka/

* Javadoc:
https://home.apache.org/~ableegoldman/kafka-2.6.2-rc0/javadoc/

* Tag to be voted upon (off 2.6 branch) is the 2.6.2 tag:
https://github.com/apache/kafka/releases/tag/2.6.2-rc0

* Documentation:
https://kafka.apache.org/26/documentation.html

* Protocol:
https://kafka.apache.org/26/protocol.html

* Successful Jenkins builds for the 2.6 branch:
Unit/integration tests:
https://ci-builds.apache.org/job/Kafka/job/kafka-2.6-jdk8/105/
System tests:
https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4435/

/**

Thanks,
Sophie


Re: Rebalancing stuck, never finishes

2021-02-26 Thread Sophie Blee-Goldman
Peter,

It does seem like KAFKA-9752 is the most likely suspect, although if your
clients
were upgraded to 2.6.1 then I don't believe they would be on an early
enough version
of the JoinGroup to run into this. I'm not 100% sure though, it may be a
good idea
to leave a comment on that ticket and ping Jason directly since he
implemented the fix

Murilo,

I agree that your problem is not likely to be KAFKA-9752, since that was
caused by
KAFKA-9232 and that code is not present  in 2.2.1. But maybe you're hitting
up on the
issue which  KAFKA-9232 was originally intended to fix? In any case, 2.2.1
is quite old now
so there may be other known bugs which have since been fixed.

I know it's not always possible/easy, but I would still recommend to
upgrade your brokers to a
more recent version if you can.


On Fri, Feb 26, 2021 at 7:19 AM Murilo Tavares  wrote:

> Just to provide a bit more detail, I noticed Peter's pattern:
> "Rebalance failed. org.apache.kafka.common.errors.DisconnectException:
> null"
> "(Re-)joining group"
>
> But I also get a different pattern, interchangeably:
> Group coordinator broker-1:9092 (id: 2147483646 rack: null) is unavailable
> or invalid due to cause: null.isDisconnected: true. Rediscovery will be
> attempted.
> Followed by
> Discovered group coordinator broker-1:9092 (id: 2147483646 rack: null)
>
>
>
> On Fri, 26 Feb 2021 at 09:59, Murilo Tavares  wrote:
>
> > Hi
> > I got the same behaviour yesterday while trying to upgrade my
> KafkaStreams
> > app from 2.4.1 to 2.7.0. Our brokers are on 2.2.1.
> >
> > Looking at KAFKA-9752 it mentions the cause being two other tickets:
> > https://issues.apache.org/jira/browse/KAFKA-7610
> > https://issues.apache.org/jira/browse/KAFKA-9232
> >
> > Although the first ticket seems fixed in 2.2.0, the latter was just fixed
> > in 2.2.3, so my brokers shouldn't have the code for KAFKA-9232.
> > But what I don't understand is that KAFKA-9752 says:
> > "Note that this is only possible if 1) we have a consumer using an old
> > JoinGroup version, 2) the consumer times out and disconnects from its
> > initial JoinGroup request."
> > In this case, I guess my consumer is not using an old JoinGroup, as my
> > consumers (KafkaStreams) are on 2.7.0...
> >
> > Thanks
> > Murilo
> >
> > On Fri, 26 Feb 2021 at 06:06, Péter Sinóros-Szabó
> >  wrote:
> >
> >> Hey Sophie,
> >>
> >> thanks for the link, I was checking that ticket, but I was not sure if
> it
> >> is relevant for our case.
> >> Eventually we "fixed" our problem with reducing the session.timeout.ms
> >> (it
> >> was set to a high value for other reasons).
> >>
> >> But today, in another service, we faced the same problem when upgrading
> >> the
> >> Kafka Client from 2.5.1 to 2.6.1. We are still using 2.4.1 on the
> brokers.
> >> Do you think the same problem (KAFKA-9752) might cause this problem too?
> >> It's hard to judge just based on the description of that ticket.
> >>
> >> Thanks,
> >> Peter
> >>
> >
>


Re: Rebalancing stuck, never finishes

2021-02-25 Thread Sophie Blee-Goldman
Hey Peter,

It does sound like you may have hit
https://issues.apache.org/jira/browse/KAFKA-9752

You will need to upgrade your brokers in order to get the fix, since it's a
broker-side issue

On Tue, Feb 9, 2021 at 2:48 AM Péter Sinóros-Szabó
 wrote:

> Hi,
>
> I have an application running with 6 instances of it on Kubernetes. All 6
> instances (pods) are the same, using the same consumer group id.
> Recently we see that when the application is restarted (rolling restart on
> K8s), the triggered rebalancing sometimes doesn't finish at all and the
> Kafka Client stucks in rebalancing. Occasionally it finishes after 30-60
> minutes, sometimes it doesn't.
>
> If it is stuck, then if we stop the application and wait until
> kafka-consumer-groups.sh doesn't show the group, and then we restart the
> application, then the initial rebalancing finishes just fine and all is
> good... until some hours or days later a rolling restart restarts it all
> again.
>
> I grabbed some logs from the time when it was continuously rebalancing.
> Logs are mixed from 6 pods, but all pods have the same logs. (Kafka brokers
> seem like running on localhost, but that's not true, traffic is routed on a
> service mesh...)
>
> 2021-02-05T17:00:18.261422532Z:  fin-df8d589bd-95bsz: INFO: Camel (camel-1)
> thread #2 - KafkaConsumer[topicX]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-3, groupId=fin] Group coordinator localhost:9204 (id:
> 2147482641 rack: null) is unavailable or invalid
> 2021-02-05T17:00:18.261454952Z:  fin-df8d589bd-95bsz: INFO: Camel (camel-1)
> thread #2 - KafkaConsumer[topicX]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-3, groupId=fin] Rebalance failed.:
> org.apache.kafka.common.errors.DisconnectException: null
>
> 2021-02-05T17:00:18.499108799Z:  fin-df8d589bd-85zf9: INFO: Camel (camel-1)
> thread #42 - KafkaConsumer[topicY]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-43, groupId=fin] Discovered group coordinator
> localhost:9204 (id: 2147482641 rack: null)
> 2021-02-05T17:00:18.499300612Z:  fin-df8d589bd-85zf9: INFO: Camel (camel-1)
> thread #42 - KafkaConsumer[topicY]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-43, groupId=fin] (Re-)joining group
>
> No more logs from Kafka Consumer, it seems that the rebalancing doesn't
> finish at all, I don't see logs in any of the pods about the partition
> assignments being calculated, so my _guess_ is that the rebalancing stucks
> in PreparingRebalance phase and never progress from there.
>
> --- About 2 minutes 10 seconds later (sometimes I see a difference here of
> 1 minutes 10 seconds).
>
> 2021-02-05T17:02:29.615402388Z:  fin-df8d589bd-95bsz: INFO:
> kafka-coordinator-heartbeat-thread | fin:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-9, groupId=fin] Group coordinator localhost:9204 (id:
> 2147482641 rack: null) is unavailable or invalid, will attempt rediscovery
> 2021-02-05T17:02:29.615520075Z:  fin-df8d589bd-95bsz: INFO: Camel (camel-1)
> thread #28 - KafkaConsumer[twcard.plastic.events.finance.reconciliation]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-29, groupId=fin] Rebalance failed.:
> org.apache.kafka.common.errors.RebalanceInProgressException: The group is
> rebalancing, so a rejoin is needed.
>
> --- This last line may has a difference reason for rebalance too:
> "Rebalance failed.: org.apache.kafka.common.errors.DisconnectException:
> null"
>
> 2021-02-05T17:02:29.74932507Z:  fin-df8d589bd-j8mw6: INFO: Camel (camel-1)
> thread #2 - KafkaConsumer[topicX]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-3, groupId=fin] Discovered group coordinator
> localhost:9204 (id: 2147482641 rack: null)
> 2021-02-05T17:02:29.749488204Z:  fin-df8d589bd-j8mw6: INFO: Camel (camel-1)
> thread #2 - KafkaConsumer[topicX]:
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator: [Consumer
> clientId=consumer-fin-3, groupId=fin] (Re-)joining group
>
> ... and the same repeats forever.
>
> Kafka Client version: 2.6.x
> Kafka Broker version: 2.4.1
>
>
> What can be the reason for this failing rebalance?
>
> I found this bug on 2.4.1, is it possible that I hit this issue?
> https://issues.apache.org/jira/browse/KAFKA-9752
> "Consumer rebalance can be stuck after new member timeout with old
> JoinGroup version"
>
>
> Thanks for the help,
> Peter
>


Re: Create a new consumer for a consumer group from within the assign method of the rebalancing interface (AbstractPartitionAssignor)

2021-02-25 Thread Sophie Blee-Goldman
If you create a new consumer inside the assign method (and assuming you
actually
start polling with it so that it sends the JoinGroup request), then yes, it
would need a
new rebalance to accommodate this consumer. The group coordinator will
inform all the
existing members to rejoin the group so that the rebalance can proceed with
the latest
up-to-date view of the current group.

On Wed, Feb 24, 2021 at 7:58 AM Mazen Ezzeddine <
mazen.ezzedd...@etu.univ-cotedazur.fr> wrote:

> I am running a Kafka cluster on Kubernetes. I am implementing a custom
> PartitionAssignor to personalize the way topic partitions are assigned to
> existing consumers in the consumer group. To this end, I am overriding the
> method Map assign( Cluster metadata, Map Subscription> subscriptions)
>
>
> If inside the assign method I dynamically created a new consumer through
> the Kubernetes client APIs, how would the rebalancing protocol behave in
> such case. Precisely, when the newly created consumer send a joinGroup
> request to the group coordinator(while the rebalancing process is still in
> progress), would the current in progress rebalancing completes, and then a
> new rebalance process is triggered to accommodate for the newly created
> consumer?
>
>
> Thanks.
>
>


Re: [VOTE] 2.7.0 RC6

2020-12-17 Thread Sophie Blee-Goldman
Thanks for driving this release! I built from the tag and ran the tests,
and verified the signatures.

+1 (binding)

Sophie

On Thu, Dec 17, 2020 at 3:53 PM Jakub Scholz  wrote:

> +1 (non-binding) ... I used the binaries (Scala 2.12) and the staged Maven
> artifacts - all seems to work fine. Thanks.
>
> Jakub
>
> On Wed, Dec 16, 2020 at 3:53 PM Bill Bejeck  wrote:
>
> > Hello Kafka users, developers and client-developers,
> >
> > This is the seventh candidate for release of Apache Kafka 2.7.0.
> >
> > * Configurable TCP connection timeout and improve the initial metadata
> > fetch
> > * Enforce broker-wide and per-listener connection creation rate (KIP-612,
> > part 1)
> > * Throttle Create Topic, Create Partition and Delete Topic Operations
> > * Add TRACE-level end-to-end latency metrics to Streams
> > * Add Broker-side SCRAM Config API
> > * Support PEM format for SSL certificates and private key
> > * Add RocksDB Memory Consumption to RocksDB Metrics
> > * Add Sliding-Window support for Aggregations
> >
> > This release also includes a few other features, 53 improvements, and 91
> > bug fixes.
> >
> > *** Please download, test and vote by Monday, December 21, 12 PM ET
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > https://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > https://home.apache.org/~bbejeck/kafka-2.7.0-rc6/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > https://home.apache.org/~bbejeck/kafka-2.7.0-rc6/javadoc/
> >
> > * Tag to be voted upon (off 2.7 branch) is the 2.7.0 tag:
> > https://github.com/apache/kafka/releases/tag/2.7.0-rc6
> >
> > * Documentation:
> > https://kafka.apache.org/27/documentation.html
> >
> > * Protocol:
> > https://kafka.apache.org/27/protocol.html
> >
> > * Successful Jenkins builds for the 2.7 branch:
> > Unit/integration tests:
> >
> >
> https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-2.7-jdk8/detail/kafka-2.7-jdk8/81/
> >
> > Thanks,
> > Bill
> >
>


Re: Guidance to downgrade the stateful Kafka stream application?

2020-12-17 Thread Sophie Blee-Goldman
Hey Ming,

There should not be any issues in downgrading from 2.5 to 2.2, and if
you stayed on the eager protocol then you can do the downgrade in a
single rolling bounce.

It sounds like your main concern here is with the RocksDB version bump,
and whether there would be any problems reading/opening a newer version
of rocksdb using an older version? AFAICT this should not be the case,
especially if you haven't made any config changes or tried to leverage new
features that are only present in the newer version of rocksdb. We also
only bumped the rocksdb version by a small amount. There was no major
version change or any compatibility-breakage that we are aware of.

If you're paranoid, or if you do happen to run into an issue after
downgrading,
you can always just wipe out the local state stores. Streams will rebuild
all
local rocksdb instances from scratch from the changelog topic.

Best,
Sophie

On Thu, Dec 17, 2020 at 11:59 AM Ming Liu  wrote:

> Hi Team,
> I can't find any documentation or guidance on the expectation of the
> downgrade of stateful Kafka stream application (which has different rocksdb
> versions embedded).
> For example, if we upgrade from 2.2 to 2.5 (with binary upgrade only
> and using the same eager protocol) and somehow found some problem and
> downgrade to 2.2. What is the expectation?
>
> Thanks!
> Ming
>


Re: Kafka Streams application stuck rebalancing on startup

2020-11-05 Thread Sophie Blee-Goldman
Ok I looked into this a bit and found the bug. I'll open a PR with the fix
sometime today:
https://issues.apache.org/jira/browse/KAFKA-10689

I also think we can do a better job of surfacing issues like this,
rather than letting the
application silently spin without making progress. I left some thoughts on
the JIRA
ticket and will try to incorporate one of them into the fix as well.

On Mon, Nov 2, 2020 at 9:38 PM Sophie Blee-Goldman 
wrote:

> Ok I tried to reproduce the issue with a minimal example, and saw the same
> results.
> It seems like there's something weird going on with that exact topology
> that's causing
> it to get stuck during the assignment. Maybe it's causing an unexpected
> cycle in the
> topology that the assignor can't handle? Pretty weird that even removing
> the windowedBy
> fixes the issue, since a topology with a windowed aggregation is pretty
> much isomorphic
> to one with just a regular aggregation.
>
> Can you create a JIRA ticket for this and include your observations + link
> to the example?
> It's definitely a bug, and we'll need to look into this to understand
> what's going wrong here.
>
> Sorry for the trouble, but thanks for bring it to our attention
>
> On Wed, Oct 28, 2020 at 12:24 PM Alex Jablonski <
> ajablon...@thoughtworks.com> wrote:
>
>> This block:
>>
>> @EmbeddedKafka(
>> topics = {
>> "WordCounts", "WordsForNumbers", "OutputTopic"
>> }
>> )
>>
>> starts up an embedded Kafka in the test and creates the 3 topics (2
>> input and 1 output). By default it creates them with 2 partitions
>> each, but changing to 1 partition didn't alter the endless-rebalancing
>> behavior.
>>
>> We also see the endless rebalancing behavior in a real Kafka cluster,
>> using input and output topics that have already been created (and are
>> readily consumed from and written to).
>>
>>
>>
>>
>> On Wed, Oct 28, 2020 at 12:45 PM Sophie Blee-Goldman > >
>> wrote:
>>
>> > Yeah there's definitely something weird going on (assuming this is the
>> full
>> > logs over that
>> > time period). The last thing we see logged from the StreamThread is this
>> > message from
>> > around the start of the task assignment process:
>> >
>> > 2020-10-28 12:22:37.879 DEBUG 27226 --- [-StreamThread-1]
>> > o.a.k.s.p.i.StreamsPartitionAssignor : stream-thread
>> >
>> >
>> [demo-application-81060bdc-c8cc-4350-85f8-d238267e264e-StreamThread-1-consumer]
>> > Constructed client metadata
>> > {81060bdc-c8cc-4350-85f8-d238267e264e=ClientMetadata{hostInfo=null,
>> >
>> >
>> consumers=[demo-application-81060bdc-c8cc-4350-85f8-d238267e264e-StreamThread-1-consumer-976853d9-06ad-4515-abf3-2a7398c12006],
>> > state=[activeTasks: ([]) standbyTasks: ([]) assignedTasks: ([])
>> > prevActiveTasks: ([]) prevStandbyTasks: ([]) prevAssignedTasks: ([])
>> > prevOwnedPartitionsByConsumerId: ([]) capacity: 1]}} from the member
>> > subscriptions.
>> >
>> >
>> >
>> > which is at 12:22:37. Then there's nothing else from Streams until at
>> least
>> > 12:25:00,
>> > where the logs end. Not sure what it could be doing inside the assignor
>> for
>> > 2+ minutes
>> > without ever reaching another...how many partitions are on the input
>> > topics? Are you
>> > sure the input topics have been pre-created before starting the app,
>> with
>> > the correct
>> > names, etc?
>> >
>> > On Wed, Oct 28, 2020 at 10:29 AM Alex Jablonski <
>> > ajablon...@thoughtworks.com>
>> > wrote:
>> >
>> > > Hi Sophie,
>> > >
>> > > Thanks for your questions! Responses inline below. Also, I realized I
>> > > linked to the gradle file, not the interesting bits of the example.
>> This
>> > > <
>> >
>> https://github.com/ajablonski/streams-issue-demo/blob/master/src/main/java/com/github/ajablonski/StreamsConfiguration.java
>> > >
>> > > is the configuration and this
>> > > <
>> >
>> https://github.com/ajablonski/streams-issue-demo/blob/master/src/test/java/com/github/ajablonski/StreamsConfigurationTest.java
>> > >
>> > > is the test.
>> > >
>> > > On Tue, Oct 27, 2020 at 10:11 PM Sophie Blee-Goldman <
>> > sop...@confluent.io>
>> > > wrote:
>> > >
>> > >> >
>> > >> > We've 

Re: Kafka Streams application stuck rebalancing on startup

2020-11-02 Thread Sophie Blee-Goldman
Ok I tried to reproduce the issue with a minimal example, and saw the same
results.
It seems like there's something weird going on with that exact topology
that's causing
it to get stuck during the assignment. Maybe it's causing an unexpected
cycle in the
topology that the assignor can't handle? Pretty weird that even removing
the windowedBy
fixes the issue, since a topology with a windowed aggregation is pretty
much isomorphic
to one with just a regular aggregation.

Can you create a JIRA ticket for this and include your observations + link
to the example?
It's definitely a bug, and we'll need to look into this to understand
what's going wrong here.

Sorry for the trouble, but thanks for bring it to our attention

On Wed, Oct 28, 2020 at 12:24 PM Alex Jablonski 
wrote:

> This block:
>
> @EmbeddedKafka(
> topics = {
> "WordCounts", "WordsForNumbers", "OutputTopic"
> }
> )
>
> starts up an embedded Kafka in the test and creates the 3 topics (2
> input and 1 output). By default it creates them with 2 partitions
> each, but changing to 1 partition didn't alter the endless-rebalancing
> behavior.
>
> We also see the endless rebalancing behavior in a real Kafka cluster,
> using input and output topics that have already been created (and are
> readily consumed from and written to).
>
>
>
>
> On Wed, Oct 28, 2020 at 12:45 PM Sophie Blee-Goldman 
> wrote:
>
> > Yeah there's definitely something weird going on (assuming this is the
> full
> > logs over that
> > time period). The last thing we see logged from the StreamThread is this
> > message from
> > around the start of the task assignment process:
> >
> > 2020-10-28 12:22:37.879 DEBUG 27226 --- [-StreamThread-1]
> > o.a.k.s.p.i.StreamsPartitionAssignor : stream-thread
> >
> >
> [demo-application-81060bdc-c8cc-4350-85f8-d238267e264e-StreamThread-1-consumer]
> > Constructed client metadata
> > {81060bdc-c8cc-4350-85f8-d238267e264e=ClientMetadata{hostInfo=null,
> >
> >
> consumers=[demo-application-81060bdc-c8cc-4350-85f8-d238267e264e-StreamThread-1-consumer-976853d9-06ad-4515-abf3-2a7398c12006],
> > state=[activeTasks: ([]) standbyTasks: ([]) assignedTasks: ([])
> > prevActiveTasks: ([]) prevStandbyTasks: ([]) prevAssignedTasks: ([])
> > prevOwnedPartitionsByConsumerId: ([]) capacity: 1]}} from the member
> > subscriptions.
> >
> >
> >
> > which is at 12:22:37. Then there's nothing else from Streams until at
> least
> > 12:25:00,
> > where the logs end. Not sure what it could be doing inside the assignor
> for
> > 2+ minutes
> > without ever reaching another...how many partitions are on the input
> > topics? Are you
> > sure the input topics have been pre-created before starting the app, with
> > the correct
> > names, etc?
> >
> > On Wed, Oct 28, 2020 at 10:29 AM Alex Jablonski <
> > ajablon...@thoughtworks.com>
> > wrote:
> >
> > > Hi Sophie,
> > >
> > > Thanks for your questions! Responses inline below. Also, I realized I
> > > linked to the gradle file, not the interesting bits of the example.
> This
> > > <
> >
> https://github.com/ajablonski/streams-issue-demo/blob/master/src/main/java/com/github/ajablonski/StreamsConfiguration.java
> > >
> > > is the configuration and this
> > > <
> >
> https://github.com/ajablonski/streams-issue-demo/blob/master/src/test/java/com/github/ajablonski/StreamsConfigurationTest.java
> > >
> > > is the test.
> > >
> > > On Tue, Oct 27, 2020 at 10:11 PM Sophie Blee-Goldman <
> > sop...@confluent.io>
> > > wrote:
> > >
> > >> >
> > >> > We've been able to get the crucial factors that cause this behavior
> > >> down to
> > >> > a particular combination
> > >>
> > >> What do you mean by this -- that you only see this when all four of
> > those
> > >> operators
> > >> are at play? Or do you see it with any of them.
> > >>
> > >
> > > We see this when all four operators are in play. If you change the
> sample
> > > streams configuration to not do that final foreign key join, or not use
> > > custom serdes for example, I don't see the stuck-state issue (the
> > > application transitions to running state just fine).
> > >
> > >
> > >>
> > >> I guess the first thing to narrow down is whether it's actually
> > >> rebalancing
> > >> or just
> > >> restoring within this t

Re: Kafka Streams application stuck rebalancing on startup

2020-10-28 Thread Sophie Blee-Goldman
Yeah there's definitely something weird going on (assuming this is the full
logs over that
time period). The last thing we see logged from the StreamThread is this
message from
around the start of the task assignment process:

2020-10-28 12:22:37.879 DEBUG 27226 --- [-StreamThread-1]
o.a.k.s.p.i.StreamsPartitionAssignor : stream-thread
[demo-application-81060bdc-c8cc-4350-85f8-d238267e264e-StreamThread-1-consumer]
Constructed client metadata
{81060bdc-c8cc-4350-85f8-d238267e264e=ClientMetadata{hostInfo=null,
consumers=[demo-application-81060bdc-c8cc-4350-85f8-d238267e264e-StreamThread-1-consumer-976853d9-06ad-4515-abf3-2a7398c12006],
state=[activeTasks: ([]) standbyTasks: ([]) assignedTasks: ([])
prevActiveTasks: ([]) prevStandbyTasks: ([]) prevAssignedTasks: ([])
prevOwnedPartitionsByConsumerId: ([]) capacity: 1]}} from the member
subscriptions.



which is at 12:22:37. Then there's nothing else from Streams until at least
12:25:00,
where the logs end. Not sure what it could be doing inside the assignor for
2+ minutes
without ever reaching another...how many partitions are on the input
topics? Are you
sure the input topics have been pre-created before starting the app, with
the correct
names, etc?

On Wed, Oct 28, 2020 at 10:29 AM Alex Jablonski 
wrote:

> Hi Sophie,
>
> Thanks for your questions! Responses inline below. Also, I realized I
> linked to the gradle file, not the interesting bits of the example. This
> <https://github.com/ajablonski/streams-issue-demo/blob/master/src/main/java/com/github/ajablonski/StreamsConfiguration.java>
> is the configuration and this
> <https://github.com/ajablonski/streams-issue-demo/blob/master/src/test/java/com/github/ajablonski/StreamsConfigurationTest.java>
> is the test.
>
> On Tue, Oct 27, 2020 at 10:11 PM Sophie Blee-Goldman 
> wrote:
>
>> >
>> > We've been able to get the crucial factors that cause this behavior
>> down to
>> > a particular combination
>>
>> What do you mean by this -- that you only see this when all four of those
>> operators
>> are at play? Or do you see it with any of them.
>>
>
> We see this when all four operators are in play. If you change the sample
> streams configuration to not do that final foreign key join, or not use
> custom serdes for example, I don't see the stuck-state issue (the
> application transitions to running state just fine).
>
>
>>
>> I guess the first thing to narrow down is whether it's actually
>> rebalancing
>> or just
>> restoring within this time (the REBALANCING state is somewhat
>> misleadingly-named).
>> If this is a completely new app then it's probably not restoring, but if
>> this app had
>> already been running and building up state before hitting this issue then
>> that's probably
>> the reason. It's not at all uncommon for restoration to take more than 30
>> seconds.
>>
>
> This is happening with the app in a completely new state -- in the test,
> for example, there's no pre-loaded data when we're asserting that the app
> should eventually get to RUNNING, and none of the internal topics exist.
>
>
>> If it really is rebalancing this entire time, then you need to look into
>> the logs to figure
>> out why. I don't see anything obviously wrong with your particular
>> application, and even
>> if there was it should never result in endless rebalances like this. How
>> many instances
>> of the application are you running?
>>
>
> In our actual application, we have 3 instances, but in the tests in that
> sample project, there's only 1.
>
> The logs that we're getting right before the application gets "stuck" are
> below. The one that seems most concerning to my uninformed eyes is "Member
> demo-application-714a21af-5fe5-4b9c-8450-53033309a406-StreamThread-1-consumer-cabbd9ce-83a7-4691-8599-b2ffe77da282
> in group demo-application has failed". I've attached some DEBUG level logs
> too, though nothing was obvious to me that would better explain the hanging
> behavior.
>
> 2020-10-28 12:11:19.823  INFO 27127 --- [-StreamThread-1]
> o.a.k.c.c.internals.AbstractCoordinator  : [Consumer
> clientId=demo-application-714a21af-5fe5-4b9c-8450-53033309a406-StreamThread-1-consumer,
> groupId=demo-application] Discovered group coordinator localhost:50343 (id:
> 2147483647 rack: null)
> 2020-10-28 12:11:19.825  INFO 27127 --- [-StreamThread-1]
> o.a.k.c.c.internals.AbstractCoordinator  : [Consumer
> clientId=demo-application-714a21af-5fe5-4b9c-8450-53033309a406-StreamThread-1-consumer,
> groupId=demo-application] (Re-)joining group
> 2020-10-28 12:11:19.842  WARN 27127 --- [-StreamThread-1]
> org.apache.kafka.clients.NetworkClient   : [Consumer
> clientId=de

Re: Kafka Streams application stuck rebalancing on startup

2020-10-27 Thread Sophie Blee-Goldman
>
> We've been able to get the crucial factors that cause this behavior down to
> a particular combination

What do you mean by this -- that you only see this when all four of those
operators
are at play? Or do you see it with any of them.

I guess the first thing to narrow down is whether it's actually rebalancing
or just
restoring within this time (the REBALANCING state is somewhat
misleadingly-named).
If this is a completely new app then it's probably not restoring, but if
this app had
already been running and building up state before hitting this issue then
that's probably
the reason. It's not at all uncommon for restoration to take more than 30
seconds.

If it really is rebalancing this entire time, then you need to look into
the logs to figure
out why. I don't see anything obviously wrong with your particular
application, and even
if there was it should never result in endless rebalances like this. How
many instances
of the application are you running?

Cheers,
Sophie

On Thu, Oct 15, 2020 at 10:01 PM Alex Jablonski 
wrote:

> Hey there!
>
> My team and I have run across a bit of a jam in our application where,
> given a particular setup, our Kafka Streams application never seems to
> start successfully, instead just getting stuck in the REBALANCING state.
> We've been able to get the crucial factors that cause this behavior down to
> a particular combination of (1) grouping, (2) windowing, (3) aggregating,
> and (4) foreign-key joining, with some of those steps specifying Serdes
> besides the default.
>
> It's probably more useful to see a minimal example, so there's one here
>  >.
> The underlying Kafka Streams version is 2.5.1. The first test should show
> the application eventually transition to running state, but it doesn't
> within the 30 second timeout I've set. Interestingly, getting rid of the
> 'Grouped.with' argument to the 'groupBy' function and the
> 'Materialized.with' in 'aggregate' in the 'StreamsConfiguration' lets the
> application transition to "RUNNING", though without the correct Serdes
> that's not too valuable.
>
> There might be a cleaner way to organize the particular flow in the toy
> example, but is there something fundamentally wrong with the approach laid
> out in that application that would cause Streams to be stuck in
> REBALANCING? I'd appreciate any advice folks could give!
>
> Thanks!
> Alex Jablonski
>


Re: Kafka Streams RocksDB CPU usage

2020-10-27 Thread Sophie Blee-Goldman
You might want to start with a lower commit interval, if you can handle some
additional latency. I would bet that the frequent flushing is a major part
of your
problem: not just the act of flushing itself, but the consequences for the
structure
of the data in each rocksdb. If you end up flushing unfilled memtables then
you'll
end up with a large number of small L0 files that then have to be
compacted, and
until they are this can make the iterators/seeks less effective. Also it
means the
memtable is less effective as a write cache so you miss out on some
immediate
deduplication of updates to the same key.

There's been some recent work to decouple flushing from committing, so
starting
in 2.7 you shouldn't have to choose between low latency and cache/rocksdb
performance. This release is currently in progress but I'd recommend
checking
it out when you can.

I'm not sure what version you're using but in 2.5 we added some RocksDB
metrics
that could be useful for further insight. I think they're all recorded at
the DEBUG
level. Might be worth investigating.

We also recently added some additional metrics to expose properties of
RocksDB,
 which will also be available in the upcoming 2.7 release.

Cheers,
Sophie

On Tue, Oct 27, 2020 at 1:49 PM Giselle van Dongen <
giselle.vandon...@ugent.be> wrote:

> Hi all,
>
>
> We have a Kafka Streams job which has high CPU utilization. When profiling
> the job, we saw that this was for a large part due to RocksDB methods:
> flush, seek, put, get, iteratorCF. We use the default settings for our
> RocksDB state store. Which configuration parameters are most important to
> tune to lower CPU usage? Most documentation focuses on memory as the
> bottleneck.
>
>
> Our job does a join and window step. The commit interval is 1 second. We
> enabled caching and the cache is 512MB large. We have 6 instances of 6 CPU
> and 30 GB RAM.
>
>
>
> Thank you for any help!
>
>


Re: Stream stopped running due to TimeoutException: Timeout expired after 60000milliseconds while awaiting InitProducerId

2020-10-19 Thread Sophie Blee-Goldman
Hey Pushkar,

If I remember correctly there were a number of broker-side bugs that were
uncovered over the course of this ticket, any of which could result in the
exception you've seen. I'm *pretty* sure that they should all be fixed in
2.5.0,
but it's possible that some of the fixes only made it into 2.5.1. I'd
recommend
upgrading to 2.5.1 when you can (even if it doesn't fix this
particular issue,
it's always a good idea to upgrade to the bugfix version when it's released)

There's also this one other client-side bug that I'm aware of, which could
result in the TimeoutException you saw. It should only affect applications
with just
a single broker in the bootstrap servers list (and
max.in.flight.connections set
to 1). If that applies to you, you can work around the issue for now by
adding
another broker to the bootstrap list. The fix is on track to make it into
the
upcoming 2.7 release. See KAFKA-10520


On Sun, Oct 18, 2020 at 11:16 PM Pushkar Deole  wrote:

> Hi All,
>
> Recently we moved kafka to azure lab and we are seeing these exceptions
> quite often. Strangely only some of the stream apps get this error and stop
> working, while other stream app run fine.
> Came across this issue on kafka issues list
> https://issues.apache.org/jira/browse/KAFKA-8803 and the problem we are
> facing matches to great extent along with all symptoms we are experiencing.
>
> Sophie, one of the comments says that you are leading investigation here,
> so do you have any suggestions?
>
> We are using kafka broker 2.5.0
> kafka client and streams are 2.5.1
>


Re: Kafka stream error - Consumer is not subscribed to any topics or assigned any partitions

2020-09-14 Thread Sophie Blee-Goldman
>
> if we are just creating a global state store (GlobalKTable
> for instance) from a topic, then that is what you are calling as
> global-only topology.


Exactly. If there is no KStream or KTable in your Streams topology,
just a GlobalKTable, then that would be a "global-only" topology

On Mon, Sep 14, 2020 at 7:46 AM John Roesler  wrote:

> Hi Pushkar,
>
> I'd recommend always keeping Streams and the Clients at the
> same version, since we build, test, and release them
> together. FWIW, I think there were some bugfixes for the
> clients in 2.5.1 anyway.
>
> Thanks,
> -John
>
> On Mon, 2020-09-14 at 20:08 +0530, Pushkar Deole wrote:
> > Sophie, one more question: will just upgrading kafka-streams jar to 2.5.1
> > will work or we need to other jars also to be upgraded to 2.5.1 e.g.
> > kafka-clients etc. ?
> >
> > On Mon, Sep 14, 2020 at 7:16 PM Pushkar Deole 
> wrote:
> >
> > > Thanks Sophie... if we are just creating a global state store
> > > (GlobalKTable for instance) from a topic, then that is what you are
> calling
> > > as global-only topology. In our application that is what we are doing
> and
> > > there is no source topic for the stream to process data from, i mean
> there
> > > is however it is done through a consumer-producer kind of design and
> not
> > > through stream topology.
> > >
> > > On Fri, Sep 11, 2020 at 10:58 PM Sophie Blee-Goldman <
> sop...@confluent.io>
> > > wrote:
> > >
> > > > You should upgrade to 2.5.1, it contains a fix for this.
> > > >
> > > > Technically the "fix" is just to automatically set the
> num.stream.threads
> > > > to 0
> > > > when a global-only topology is detected, so setting this manually
> would
> > > > accomplish the same thing. But the fix also includes a tweak of the
> > > > KafkaStreams state machine to make sure it reaches the RUNNING state
> > > > even with no stream threads. So if you use a state listener, you'll
> want
> > > > to
> > > > use 2.5.1
> > > >
> > > > It's always a good idea to upgrade when a new bugfix version is
> released
> > > > anyway
> > > >
> > > > On Fri, Sep 11, 2020 at 5:15 AM Pushkar Deole 
> > > > wrote:
> > > >
> > > > > Hi All,
> > > > >
> > > > > I upgraded from Kafka streams 2.4 to 2.5.0 and one of the
> applications
> > > > > suddenly stopped working with the error message:
> > > > >
> > > > > Exception in thread
> > > > >
> "DsiApplication-0fcde033-dab2-431c-9d82-76e85fcb4c91-StreamThread-1"
> > > > > java.lang.IllegalStateException: Consumer is not subscribed to any
> > > > topics
> > > > > or assigned any partitions
> > > > > at
> > > > >
> > > > >
> > > >
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1228)
> > > > > at
> > > > >
> > > > >
> > > >
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216)
> > > > > at
> > > > >
> > > > >
> > > >
> org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:853)
> > > > > at
> > > > >
> > > > >
> > > >
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:753)
> > > > > at
> > > > >
> > > > >
> > > >
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:697)
> > > > > at
> > > > >
> > > > >
> > > >
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:670)
> > > > > This application uses streams just to create a global state store
> from a
> > > > > topic in order to create a global state store as a cache for
> static data
> > > > > across application instances and the stream doesn't consume from
> any
> > > > input
> > > > > topic. Came across following thread on stackoverflow
> > > > >
> > > > >
> > > >
> https://stackoverflow.com/questions/61342530/kafka-streams-2-5-0-requires-input-topic
> > > > > Matthias, I see you have answered some queries there, so would
> like to
> > > > > confirm if setting num.stream.threads to 0 will solve this issue?
> > > > >
>
>


Re: Streams constantly reblancing,

2020-09-11 Thread Sophie Blee-Goldman
That definitely sounds pretty odd if no new logs are showing up on the
Streams side. This is just a guess, but is there any chance the source
topics aren't being found during startup? I was just reminded of a bug in
an old-ish version where Streams would gracefully shutdown but remain
in the REBALANCING state when it couldn't detect the source topics.

What version are you using?

On Thu, Sep 3, 2020 at 8:11 AM Steve Jones  wrote:

> There are zero events on the stream in the case with the log files, I've
> had the same behaviour when running with 6 events.  Its constantly
> reporting rebalancing but from a CPU perspective nothing appears to
> actually be doing anything.
>
> I've run it for several hours just to see if something happened (checking
> every 30 seconds) and there are no log events from Kafka during those
> times.  So the log file (filter.log) shows the logs for about an hour and a
> quarter run, my confusion comes from the fact that looking via JMX and at
> the CPU logs it appears that nothing much is actually being done on the
> Kafka side but there aren't any exceptions.
>
> Help and hints appreciated.
>
> Steve
>
> On Wed, 2 Sep 2020 at 12:27, Sophie Blee-Goldman 
> wrote:
>
> > Hey steve,
> >
> > I guess the first question I have is, is the consumer group actually
> > rebalancing
> > or does KafkaStreams just appear to be "stuck" in the REBALANCING state?
> If
> > this application has been run before and there is a large amount of data
> in
> > the
> > changelog topics, it might take a while to rebuild the local state stores
> > from the
> > changelog. During this time it would appear to be in REBALANCING, but
> it's
> > actually
> > just restoring.
> >
> > Just one possibility of many. How long did you let it run without seeing
> it
> > stabilize?
> > It's difficult to diagnose from just the small snippet of logs above, so
> I
> > would
> > try and take a look at the full picture over more than just a minute.
> There
> > *should*
> > be something slightly more helpful logged if it really is stuck in a
> cycle
> > of endless
> > rebalances (eg timing out on the session interval)
> >
> > On Wed, Sep 2, 2020 at 11:52 AM Steve Jones 
> > wrote:
> >
> > > Just an addition to this in case someone can help, I'm seeing zero
> thread
> > > activity in the rebalancing but I'm also not seeing anything being
> logged
> > > by Kafka to indicate its doing anything on rebalancing (Logging level
> set
> > > to ALL), producer and consumer code works fine but streams just appears
> > to
> > > hang.
> > >
> > > On Mon, 31 Aug 2020 at 12:49, Steve Jones 
> > wrote:
> > >
> > >> When running an application on the Mac it works fine, when running
> > >> exactly the same app and config on the Raspberry Pi it constantly says
> > it
> > >> is "Rebalancing" the streams
> > >>
> > >> 2020-08-31 12:47:11 INFO
> > >> org.apache.kafka.common.utils.AppInfoParser$AppInfo  Kafka
> > version:
> > >> 2.6.0
> > >>
> > >> ''2020-08-31 12:47:11 INFO
> > >> org.apache.kafka.common.utils.AppInfoParser$AppInfo  Kafka
> > commitId:
> > >> 62abe01bee039651
> > >>
> > >> ''2020-08-31 12:47:11 INFO
> > >> org.apache.kafka.common.utils.AppInfoParser$AppInfo  Kafka
> > >> startTimeMs: 1598903231499
> > >>
> > >> ''2020-08-31 12:47:11 WARNING org.apache.kafka.streams.StreamsConfig
> > >> checkIfUnexpectedUserSpecifiedConsumerConfig Unexpected user-specified
> > >> consumer config: enable.auto.commit found. User setting (true) will be
> > >> ignored and the Streams default setting (false) will be used
> > >>
> > >> ''2020-08-31 12:47:11 INFO org.apache.kafka.streams.KafkaStreams
> > setState
> > >> stream-client [pi-test-84721b40-dfa1-4848-b3de-5c7561048403] State
> > >> transition from CREATED to REBALANCING
> > >>
> > >> ''2020-08-31 12:47:11 INFO
> > >> com.businesssoa.home.events.filter.dispatcher.kafka.SensorPipe start
> > Pipe
> > >> STARTED
> > >>
> > >> ''2020-08-31 12:47:11 INFO
> > >> org.apache.kafka.streams.processor.internals.StreamThread run
> > stream-thread
> > >> [pi-test-84721b40-dfa1-4848-b3de-5c7561048403-StreamThread-1] Starting
> > >>
> > >> ''2020-08-31 12:47:11 INFO
> > >> org.apache.kafka.streams.processor.internals.St

Re: Kafka stream error - Consumer is not subscribed to any topics or assigned any partitions

2020-09-11 Thread Sophie Blee-Goldman
You should upgrade to 2.5.1, it contains a fix for this.

Technically the "fix" is just to automatically set the num.stream.threads
to 0
when a global-only topology is detected, so setting this manually would
accomplish the same thing. But the fix also includes a tweak of the
KafkaStreams state machine to make sure it reaches the RUNNING state
even with no stream threads. So if you use a state listener, you'll want to
use 2.5.1

It's always a good idea to upgrade when a new bugfix version is released
anyway

On Fri, Sep 11, 2020 at 5:15 AM Pushkar Deole  wrote:

> Hi All,
>
> I upgraded from Kafka streams 2.4 to 2.5.0 and one of the applications
> suddenly stopped working with the error message:
>
> Exception in thread
> "DsiApplication-0fcde033-dab2-431c-9d82-76e85fcb4c91-StreamThread-1"
> java.lang.IllegalStateException: Consumer is not subscribed to any topics
> or assigned any partitions
> at
>
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1228)
> at
>
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216)
> at
>
> org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:853)
> at
>
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:753)
> at
>
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:697)
> at
>
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:670)
>
> This application uses streams just to create a global state store from a
> topic in order to create a global state store as a cache for static data
> across application instances and the stream doesn't consume from any input
> topic. Came across following thread on stackoverflow
>
> https://stackoverflow.com/questions/61342530/kafka-streams-2-5-0-requires-input-topic
>
> Matthias, I see you have answered some queries there, so would like to
> confirm if setting num.stream.threads to 0 will solve this issue?
>


Re: Streams constantly reblancing,

2020-09-02 Thread Sophie Blee-Goldman
Hey steve,

I guess the first question I have is, is the consumer group actually
rebalancing
or does KafkaStreams just appear to be "stuck" in the REBALANCING state? If
this application has been run before and there is a large amount of data in
the
changelog topics, it might take a while to rebuild the local state stores
from the
changelog. During this time it would appear to be in REBALANCING, but it's
actually
just restoring.

Just one possibility of many. How long did you let it run without seeing it
stabilize?
It's difficult to diagnose from just the small snippet of logs above, so I
would
try and take a look at the full picture over more than just a minute. There
*should*
be something slightly more helpful logged if it really is stuck in a cycle
of endless
rebalances (eg timing out on the session interval)

On Wed, Sep 2, 2020 at 11:52 AM Steve Jones  wrote:

> Just an addition to this in case someone can help, I'm seeing zero thread
> activity in the rebalancing but I'm also not seeing anything being logged
> by Kafka to indicate its doing anything on rebalancing (Logging level set
> to ALL), producer and consumer code works fine but streams just appears to
> hang.
>
> On Mon, 31 Aug 2020 at 12:49, Steve Jones  wrote:
>
>> When running an application on the Mac it works fine, when running
>> exactly the same app and config on the Raspberry Pi it constantly says it
>> is "Rebalancing" the streams
>>
>> 2020-08-31 12:47:11 INFO
>> org.apache.kafka.common.utils.AppInfoParser$AppInfo  Kafka version:
>> 2.6.0
>>
>> ''2020-08-31 12:47:11 INFO
>> org.apache.kafka.common.utils.AppInfoParser$AppInfo  Kafka commitId:
>> 62abe01bee039651
>>
>> ''2020-08-31 12:47:11 INFO
>> org.apache.kafka.common.utils.AppInfoParser$AppInfo  Kafka
>> startTimeMs: 1598903231499
>>
>> ''2020-08-31 12:47:11 WARNING org.apache.kafka.streams.StreamsConfig
>> checkIfUnexpectedUserSpecifiedConsumerConfig Unexpected user-specified
>> consumer config: enable.auto.commit found. User setting (true) will be
>> ignored and the Streams default setting (false) will be used
>>
>> ''2020-08-31 12:47:11 INFO org.apache.kafka.streams.KafkaStreams setState
>> stream-client [pi-test-84721b40-dfa1-4848-b3de-5c7561048403] State
>> transition from CREATED to REBALANCING
>>
>> ''2020-08-31 12:47:11 INFO
>> com.businesssoa.home.events.filter.dispatcher.kafka.SensorPipe start Pipe
>> STARTED
>>
>> ''2020-08-31 12:47:11 INFO
>> org.apache.kafka.streams.processor.internals.StreamThread run stream-thread
>> [pi-test-84721b40-dfa1-4848-b3de-5c7561048403-StreamThread-1] Starting
>>
>> ''2020-08-31 12:47:11 INFO
>> org.apache.kafka.streams.processor.internals.StreamThread setState
>> stream-thread [pi-test-84721b40-dfa1-4848-b3de-5c7561048403-StreamThread-1]
>> State transition from CREATED to STARTING
>>
>> ''2020-08-31 12:47:11 INFO
>> org.apache.kafka.clients.consumer.KafkaConsumer subscribe [Consumer
>> clientId=pi-test-84721b40-dfa1-4848-b3de-5c7561048403-StreamThread-1-consumer,
>> groupId=pi-test] Subscribed to topic(s): filtered-topic, reading-topic
>>
>> ''2020-08-31 12:47:11 INFO com.businesssoa.home.events.filter.App
>> waitTillStarted Status is REBALANCING , 1
>>
>> ''2020-08-31 12:47:12 INFO org.apache.kafka.clients.Metadata update
>> [Producer
>> clientId=pi-test-84721b40-dfa1-4848-b3de-5c7561048403-StreamThread-1-producer]
>> Cluster ID: 1AOC2O8zSqq7nuQKACXFxQ
>>
>> ''2020-08-31 12:47:12 INFO org.apache.kafka.clients.Metadata update
>> [Consumer
>> clientId=pi-test-84721b40-dfa1-4848-b3de-5c7561048403-StreamThread-1-consumer,
>> groupId=pi-test] Cluster ID: 1AOC2O8zSqq7nuQKACXFxQ
>>
>> ''2020-08-31 12:47:41 INFO com.businesssoa.home.events.filter.App
>> waitTillStarted Status is REBALANCING , 2
>>
>> ''2020-08-31 12:48:11 INFO com.businesssoa.home.events.filter.App
>> waitTillStarted Status is REBALANCING , 3
>>
>> Any guidance on how I can debug the streams as to why its constantly
>> rebalancing?
>>
>> Steve
>>
>>


Re: JNI linker issue on ARM (Raspberry PI)

2020-08-24 Thread Sophie Blee-Goldman
Yeah, if you weren't already running it, try upgrading Streams to 2.6. They
recently
added support for "all" platforms to rocksdb and we updated the dependency
to
get this fix in 2.6. See KAFKA-9225


If you already were running 2.6, then, that's unfortunate. You might have
some luck
asking the rocksdb folks if all else fails

On Mon, Aug 24, 2020 at 5:46 PM John Roesler  wrote:

> Hi Steve,
>
> Which version of Streams is this? I vaguely recall that we updated to a
> version of Rocks that’s compiled for ARM, and I think some people have used
> it on ARM, but I might be misremembering.
>
> I’m afraid I can’t be much help in debugging this, but maybe some others
> on the list have more context. If all else fails, you can probably narrow
> it down to the Java RocksDB library. If you create a standalone java
> program using the same rocks dependency that we do, then you can pretty
> confidently raise it with the Rocks folks.
>
> If you want to side-step the issue while debugging this, plugging in a
> different store implementation (like the in-memory one) would probably
> work, although I’m sure memory is scarce on a Raspberry Pi.
>
> I hope this helps,
> -John
>
> On Mon, Aug 24, 2020, at 19:26, Steve Jones wrote:
> > I'm trying to install Kafka Streams on a Raspberry PI, it works fine as a
> > broker, works fine as both a producer and consumer, but when I try and
> run
> > streams on the PI rather than on the Mac there is a linker issue:
> >
> > Exception in thread
> > "main-broker-f53264a1-0c70-445f-bf3f-bf634a9a1ed2-StreamThread-1"
> > java.lang.UnsatisfiedLinkError:
> /tmp/librocksdbjni15158764823832728522.so:
> > /tmp/librocksdbjni15158764823832728522.so: cannot open shared object
> file:
> > No such file or directory (Possible cause: can't load IA 32 .so on a ARM
> > platform)
> >
> > at java.base/java.lang.ClassLoader$NativeLibrary.load0(Native Method)
> >
> > at
> java.base/java.lang.ClassLoader$NativeLibrary.load(ClassLoader.java:2452)
> >
> > at
> >
> java.base/java.lang.ClassLoader$NativeLibrary.loadLibrary(ClassLoader.java:2508)
> >
> > at java.base/java.lang.ClassLoader.loadLibrary0(ClassLoader.java:2704)
> >
> > at java.base/java.lang.ClassLoader.loadLibrary(ClassLoader.java:2637)
> >
> >
> > I've recompiled rocksdb on the RaspberryPI and added that to the loadpath
> > but the same error occurs.  I've done the Google search but not found
> > anything around streams on ARM (Raspberry PI) and what needs to be
> > recompiled/linked for this to work.
> >
> >
> > Help appreciated.
> >
> >
> > Steve Jones
> >
>


Re: Partition assignment not well distributed over threads

2020-07-29 Thread Sophie Blee-Goldman
Hey Giselle,

How many stream threads is each instance configured with? If the total
number of threads
across all instances exceeds the total number of tasks, then some threads
won't get any
assigned tasks. There's a known bug where tasks might not get evenly
distributed over all
instances in this scenario, as Streams would only attempt to balance the
tasks over the
threads. See KAFKA-9173 .
Luckily, this should be fixed in 2.6 which is just about to be
released.

Instances that joined later, or restarted, would be more likely to have
these threads with no
assigned tasks due to the stickiness optimization, as you guessed.

If the problem you've run into is due to running more stream threads than
tasks, I would
recommend just decreasing the number of threads per instance to get a
balanced assignment.
This won't hurt performance in any way since those extra threads would have
just been sitting
idle anyways. Or better yet, upgrade to 2.6.

Regarding the colocation question: no, the assignment doesn't take that
into account at the
moment. Typically Streams applications won't be running on the same machine
as the broker.
Clearly it has been difficult enough to optimize for two things at the same
time, stickiness and
balance, without introducing a third :)

On Wed, Jul 29, 2020 at 4:58 AM Giselle Van Dongen <
giselle.vandon...@klarrio.com> wrote:

> We have a Kafka Streams (2.4) app consisting of 5 instances. It reads from
> a Kafka topic with 20 partitions (5 brokers).
>
> We notice that the partition assignment does not always lead to well
> distributed load over the different threads. We notice this at startup as
> well as after a recovery of a failed thread.
>
> 1. At startup, some instances get a significantly lower load and sometimes
> even no load. It seems like instances that come up slightly later get no
> partitions assigned (because of sticky assignment?).
>
> 2. When one thread (container) dies and comes back it often does not
> receive any or very few partitions to work on. We assume this has to do
> with the sticky assignment. Is there any way we can make this distribution
> more equal?
>
> I was also wondering whether Kafka Streams takes into account colocation
> of Kafka brokers with stream processing threads when assigning partitions.
> Do partitions on brokers get assigned to the streams thread that is
> colocated with it on the same machine?
>


Re: Kafka Streams - partition assignment for the input topic

2020-03-23 Thread Sophie Blee-Goldman
I don't think it has anything to do with your specific topology, but it
might be
that the "stickiness" is overriding the "data parallelism balance" in the
current
assignment algorithm. There are a lot of different factors to optimize for,
so we
end up making tradeoffs with a rough hierarchy of these factors:

1) task balance (number of tasks)
2) stickiness (instance had this task before, helps with state restoration)
3) data parallelism (distributing tasks of the same subtopology across
different instances)

In other words, within the constraint that instances should have an equal
number of
total tasks (proportionate to their number of stream thread, differing by
at most 1), the
next highest criteria is "which instance had this task last". So, if you
bring up your
instances one at a time, the first one will end up with all the tasks
initially. When you
bring up the second, we will go through the tasks and assign them to their
previous
owner if possible, ie, until that previous owner is full (the balance
constraint). Since
we assign tasks in subtopology order, eg t1p0, t1p1, t2p0, t2p1, t3 ... the
"first" tasks
will presumably all end up on your first instance. It sounds like in your
case the "first"
tasks are the ones for the input topic.

I hope that made sense and didn't just confuse things further -- if you're
interested in
the assignment code I'm referring to take a look at StickyTaskAssignor. The
good news
is that KIP-441 will definitely address this limitation of the current
assignment algorithm.

On Mon, Mar 23, 2020 at 7:57 AM Stephen Young
 wrote:

> Thanks for your help Sophie and Matthias.
>
> In my cloud environment I'm using kafka version 2.2.1. I've tested this
> locally with 2.4.1 and I can see the same issue with 3 local instances. As
> I add more local instances I start to see better balancing.
>
> I was wondering if the issue could be because my kafka streams app reads
> the input topic as a ktable. I want this so it is simple for producers to
> send deletes and updates to records in the topic (by nullifying keys etc)
> and also so my streams app automatically recalculates various aggregations
> in all of the sub-topologies. Could this be the cause of the problem?
>
> Stephen
>
> On Fri, 20 Mar 2020 at 17:33, Sophie Blee-Goldman 
> wrote:
>
> > Although it's not the main objective, one side effect of KIP-441 should
> be
> > improved balance of the final stable assignment. By warming up standbys
> > before switching them over to active tasks we can achieve stickiness
> > without
> > sacrificing balance in the followup rebalance.
> >
> > This work is targeted for the next release, so if you do still observe
> > issues in
> > newer versions I'd recommend trying out 2.6 when it comes out.
> >
> > You can read up on the details and track the progress of this KIP in the
> > KIP document:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-441:+Smooth+Scaling+Out+for+Kafka+Streams
> > JIRA: https://issues.apache.org/jira/browse/KAFKA-6145?src=confmacro
> >
> > Cheers,
> > Sophie
> >
> > On Fri, Mar 20, 2020 at 10:20 AM Matthias J. Sax 
> wrote:
> >
> > > Partition assignment, or move specific "task placement" for Kafka
> > > Streams, is a hard-coded algorithm (cf.
> > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
> > > ).
> > > The algorithm actually tires to assign different tasks from the same
> > > sub-topology to different instances and thus, your 6 input topic
> > > partitions should ideally get balanced over your 3 instance (ie, 2
> each,
> > > one for each thread).
> > >
> > > However, the algorithm needs to trade-off load balancing and stickiness
> > > (to avoid unnecessary, expensive state migration) and thus, the
> > > placement strategy is best effort only. Also, in older versions there
> > > was some issue that got fixed in newer version (ie, 2.0.x and newer).
> > > Not sure what version you are on (as you linked to 1.0 docs, maybe
> > > upgrade resolves your issue?).
> > >
> > > Compare:
> > >
> > >  - https://issues.apache.org/jira/browse/KAFKA-6039
> > >  - https://issues.apache.org/jira/browse/KAFKA-7144
> > >
> > > If you still observe issues in never version, please comment on the
> > > tickets ofr create a new ticket describing the problem. Or even better,
> > > do a PR to help improving the "task placement" algorithm. :)
> > >
> > >
> > > -M

Re: Kafka Streams - partition assignment for the input topic

2020-03-20 Thread Sophie Blee-Goldman
Although it's not the main objective, one side effect of KIP-441 should be
improved balance of the final stable assignment. By warming up standbys
before switching them over to active tasks we can achieve stickiness without
sacrificing balance in the followup rebalance.

This work is targeted for the next release, so if you do still observe
issues in
newer versions I'd recommend trying out 2.6 when it comes out.

You can read up on the details and track the progress of this KIP in the
KIP document:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-441:+Smooth+Scaling+Out+for+Kafka+Streams
JIRA: https://issues.apache.org/jira/browse/KAFKA-6145?src=confmacro

Cheers,
Sophie

On Fri, Mar 20, 2020 at 10:20 AM Matthias J. Sax  wrote:

> Partition assignment, or move specific "task placement" for Kafka
> Streams, is a hard-coded algorithm (cf.
>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
> ).
> The algorithm actually tires to assign different tasks from the same
> sub-topology to different instances and thus, your 6 input topic
> partitions should ideally get balanced over your 3 instance (ie, 2 each,
> one for each thread).
>
> However, the algorithm needs to trade-off load balancing and stickiness
> (to avoid unnecessary, expensive state migration) and thus, the
> placement strategy is best effort only. Also, in older versions there
> was some issue that got fixed in newer version (ie, 2.0.x and newer).
> Not sure what version you are on (as you linked to 1.0 docs, maybe
> upgrade resolves your issue?).
>
> Compare:
>
>  - https://issues.apache.org/jira/browse/KAFKA-6039
>  - https://issues.apache.org/jira/browse/KAFKA-7144
>
> If you still observe issues in never version, please comment on the
> tickets ofr create a new ticket describing the problem. Or even better,
> do a PR to help improving the "task placement" algorithm. :)
>
>
> -Matthias
>
>
> On 3/20/20 6:47 AM, Stephen Young wrote:
> > Thanks Guozhang. That's really helpful!
> >
> > Are you able to explain a bit more about how it would work for my use
> case? As I understand it this 'repartition' method enables us to
> materialize a stream to a new topic with a custom partitioning strategy.
> >
> > But my problem is not how the topic is partitioned. My issue is that the
> partitions of the source topic need to be spread equally amongst all the
> available threads. How could 'repartition' help with this?
> >
> > Stephen
> >
> > On 2020/03/19 23:20:54, Guozhang Wang  wrote:
> >> Hi Stephen,
> >>
> >> We've deprecated the partition-grouper API due to its drawbacks in
> >> upgrading compatibility (consider if you want to change the
> num.partitions
> >> while evolving your application), and instead we're working on KIP-221
> for
> >> the same purpose of your use case:
> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-221%3A+Enhance+DSL+with+Connecting+Topic+Creation+and+Repartition+Hint
> >>
> >>
> >> Guozhang
> >>
> >> On Wed, Mar 18, 2020 at 7:48 AM Stephen Young
> >>  wrote:
> >>
> >>> I have a question about partition assignment for a kafka streams app.
> As I
> >>> understand it the more complex your topology is the greater the number
> of
> >>> internal topics kafka streams will create. In my case the app has 8
> graphs
> >>> in the topology. There are 6 partitions for each graph (this matches
> the
> >>> number of partitions of the input topic). So there are 48 partitions
> that
> >>> the app needs to handle. These get balanced equally across all 3
> servers
> >>> where the app is running (each server also has 2 threads so there are 6
> >>> available instances of the app).
> >>>
> >>> The problem for me is that the partitions of the input topic have the
> >>> heaviest workload. But these 6 partitions are not distributed evenly
> >>> amongst the instances. They are just considered 6 partitions amongst
> the 48
> >>> the app needs to balance. But this means if a server gets most or all
> of
> >>> these 6 partitions, it ends up exhausting all of the resources on that
> >>> server.
> >>>
> >>> Is there a way of equally balancing these 6 specific partitions
> amongst the
> >>> available instances? I thought writing a custom partition grouper might
> >>> help here:
> >>>
> >>>
> >>>
> https://kafka.apache.org/10/documentation/streams/developer-guide/config-streams.html#partition-grouper
> >>>
> >>> But the advice seems to be to not do this otherwise you risk breaking
> the
> >>> app.
> >>>
> >>> Thanks!
> >>>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
>
>


Re: "Skipping record for expired segment" in InMemoryWindowStore

2020-02-11 Thread Sophie Blee-Goldman
:
> >
> > Hey all,
> >
> > Sorry for the confusion. Bruno set me straight offline.
> >
> > Previously, we had metrics for each reason for skipping records, and the
> > rationale was that you would monitor the metrics and only turn to the
> logs
> > if you needed to *debug* unexpected record skipping. Note that skipping
> > records by itself isn't a cause for concern, since this is exactly what
> Streams
> > is designed to do in a number of situations.
> >
> > However, during the KIP-444 discussion, the decision was reversed, and we
> > decided to just log one "roll-up" metric for all skips and increase the
> log
> > messages to warning level for debuggability. This particularly makes
> sense
> > because you otherwise would have to restart the application to change the
> > log level if you needed to figure out why the single skipped-record
> metric
> > is non-zero. And then you may not even observe it again.
> >
> > I either missed the memo on that discussion, or participated in it and
> then
> > forgot it even happened. I'm not sure I want to look back at the thread
> to
> > find out.
> >
> > Anyway, I've closed the PR I opened to move it back to debug. We should
> > still try to help figure out the root cause of this particular email
> thread,
> > though.
> >
> > Thanks,
> > -John
> >
> > On Mon, Feb 10, 2020, at 12:20, Sophie Blee-Goldman wrote:
> > > While I agree that seems like it was probably a refactoring mistake,
> I'm
> > > not
> > > convinced it isn't the right thing to do. John, can you reiterate the
> > > argument
> > > for setting it to debug way back when?
> > >
> > > I would actually present this exact situation as an argument for
> keeping it
> > > as
> > > warn, since something indeed seems fishy here that was only surfaced
> > > through this warning. That said, maybe the metric is the more
> appropriate
> > > way to bring attention to this: not sure if it's info or debug level
> > > though, or
> > > how likely it is that anyone really pays attention to it?
> > >
> > > On Mon, Feb 10, 2020 at 9:53 AM John Roesler 
> wrote:
> > >
> > > > Hi,
> > > >
> > > > I’m sorry for the trouble. It looks like it was a mistake during
> > > >
> > > > https://github.com/apache/kafka/pull/6521
> > > >
> > > > Specifically, while addressing code review comments to change a bunch
> of
> > > > other logs from debugs to warnings, that one seems to have been
> included by
> > > > accident:
> > > >
>
> https://github.com/apache/kafka/commit/ac27e8578f69d60a56ba28232d7e96c76957f66c
> > > >
> > > > I’ll see if I can fix it today.
> > > >
> > > > Regarding Bruno's thoughts, there was a pretty old decision to
> capture the
> > > > "skipped records" as a metric for visibility and log it at the debug
> level
> > > > for debuggability. We decided that "warning" wasn't the right level
> because
> > > > Streams is operating completely as specified.
> > > >
> > > > However, I do agree that it doesn't seem right to see more skipped
> records
> > > > during start-up; I would expect to see exactly the same records
> skipped
> > > > during start-up as during regular processing, since the skipping
> logic is
> > > > completely deterministic and based on the sequence of timestamps your
> > > > records have in the topic.  Maybe you just notice it more during
> startup?
> > > > I.e., if there are 1000 warning logs spread over a few months, then
> you
> > > > don't notice it, but when you see them all together at start-up, it's
> more
> > > > concerning?
> > > >
> > > > Thanks,
> > > > -John
> > > >
> > > >
> > > > On Mon, Feb 10, 2020, at 10:15, Bruno Cadonna wrote:
> > > > > Hi,
> > > > >
> > > > > I am pretty sure this was intentional. All skipped records log
> > > > > messages are on WARN level.
> > > > >
> > > > > If a lot of your records are skipped on app restart with this log
> > > > > message on WARN-level, they were also skipped with the log message
> on
> > > > > DEBUG-level. You simply did not know about it before. With an
> > > > > in-memory window store, this message is logged when

Re: "Skipping record for expired segment" in InMemoryWindowStore

2020-02-10 Thread Sophie Blee-Goldman
While I agree that seems like it was probably a refactoring mistake, I'm
not
convinced it isn't the right thing to do. John, can you reiterate the
argument
for setting it to debug way back when?

I would actually present this exact situation as an argument for keeping it
as
warn, since something indeed seems fishy here that was only surfaced
through this warning. That said, maybe the metric is the more appropriate
way to bring attention to this: not sure if it's info or debug level
though, or
how likely it is that anyone really pays attention to it?

On Mon, Feb 10, 2020 at 9:53 AM John Roesler  wrote:

> Hi,
>
> I’m sorry for the trouble. It looks like it was a mistake during
>
> https://github.com/apache/kafka/pull/6521
>
> Specifically, while addressing code review comments to change a bunch of
> other logs from debugs to warnings, that one seems to have been included by
> accident:
> https://github.com/apache/kafka/commit/ac27e8578f69d60a56ba28232d7e96c76957f66c
>
> I’ll see if I can fix it today.
>
> Regarding Bruno's thoughts, there was a pretty old decision to capture the
> "skipped records" as a metric for visibility and log it at the debug level
> for debuggability. We decided that "warning" wasn't the right level because
> Streams is operating completely as specified.
>
> However, I do agree that it doesn't seem right to see more skipped records
> during start-up; I would expect to see exactly the same records skipped
> during start-up as during regular processing, since the skipping logic is
> completely deterministic and based on the sequence of timestamps your
> records have in the topic.  Maybe you just notice it more during startup?
> I.e., if there are 1000 warning logs spread over a few months, then you
> don't notice it, but when you see them all together at start-up, it's more
> concerning?
>
> Thanks,
> -John
>
>
> On Mon, Feb 10, 2020, at 10:15, Bruno Cadonna wrote:
> > Hi,
> >
> > I am pretty sure this was intentional. All skipped records log
> > messages are on WARN level.
> >
> > If a lot of your records are skipped on app restart with this log
> > message on WARN-level, they were also skipped with the log message on
> > DEBUG-level. You simply did not know about it before. With an
> > in-memory window store, this message is logged when a window with a
> > start time older than the current stream time minus the retention
> > period is put into the window store, i.e., the window is NOT inserted
> > into the window stroe. If you get a lot of them on app restart, you
> > should have a look at the timestamps of your records and the retention
> > of your window store. If those values do not explain the behavior,
> > please try to find a minimal example that shows the issue and post it
> > here on the mailing list.
> >
> > On Mon, Feb 10, 2020 at 2:27 PM Samek, Jiří 
> wrote:
> > >
> > > Hi,
> > >
> > > in
> > >
> https://github.com/apache/kafka/commit/9f5a69a4c2d6ac812ab6134e64839602a0840b87#diff-a5cfe68a5931441eff5f00261653dd10R134
> > >
> > > log level of "Skipping record for expired segment" was changed from
> debug
> > > to warn. Was it intentional change? Should it be somehow handled by
> user?
> > > How can user handle it? I am getting a lot of these on app restart.
> >
>


Re: Reducing streams startup bandwidth usage

2019-12-12 Thread Sophie Blee-Goldman
Thanks for collecting all these metrics. It might be that as the length of
the lists
increases over time, the cache is able to hold fewer unique keys and
eventually has to
start evicting things. This would explain why the cache hit rate starts to
decrease, and
likely why latency starts to go up. Whenever a dirty entry is
evicted/flushed from the cache
it gets sent to the changelog (and underlying state store), so these
evictions might be the
cause of the increased load.

The fluctuations you're seeing (ie it starts and stops "working") could
just be the window
closing. After that, the list size would go back down to zero, and the
cache would suddenly
have free space again.

Does that seem to make sense with what you're seeing?

On Tue, Dec 10, 2019 at 7:04 PM Alessandro Tagliapietra <
tagliapietra.alessan...@gmail.com> wrote:

> Just an update since it has been happening again now and I have some more
> metrics to show, the topology is this:
>
> Topologies:
>Sub-topology: 0
> Source: KSTREAM-SOURCE-00 (topics: [sensors])
>   --> KSTREAM-TRANSFORMVALUES-01
> Processor: KSTREAM-TRANSFORMVALUES-01 (stores:
> [new-data-store])
>   --> KSTREAM-FLATMAPVALUES-02
>   <-- KSTREAM-SOURCE-00
> Processor: KSTREAM-FLATMAPVALUES-02 (stores: [])
>   --> KSTREAM-TRANSFORMVALUES-03
>   <-- KSTREAM-TRANSFORMVALUES-01
> Processor: KSTREAM-TRANSFORMVALUES-03 (stores:
> [LastValueStore])
>   --> KSTREAM-FILTER-04
>   <-- KSTREAM-FLATMAPVALUES-02
> Processor: KSTREAM-FILTER-04 (stores: [])
>   --> KSTREAM-AGGREGATE-05
>   <-- KSTREAM-TRANSFORMVALUES-03
> Processor: KSTREAM-AGGREGATE-05 (stores: [aggregate-store])
>   --> KTABLE-TOSTREAM-06
>   <-- KSTREAM-FILTER-04
> Processor: KTABLE-TOSTREAM-06 (stores: [])
>   --> KSTREAM-TRANSFORM-07
>   <-- KSTREAM-AGGREGATE-05
> Processor: KSTREAM-TRANSFORM-07 (stores: [suppress-store])
>   --> KSTREAM-MAP-08
>   <-- KTABLE-TOSTREAM-06
> Processor: KSTREAM-MAP-08 (stores: [])
>   --> KSTREAM-PRINTER-09, KSTREAM-SINK-10
>   <-- KSTREAM-TRANSFORM-07
> Processor: KSTREAM-PRINTER-09 (stores: [])
>   --> none
>   <-- KSTREAM-MAP-08
> Sink: KSTREAM-SINK-10 (topic: sensors-output)
>   <-- KSTREAM-MAP-08
>
>  - https://imgur.com/R3Pqypo this shows that the input source topic has
> the
> same rate of messages
>  - https://imgur.com/BTwq09p this is the number of records processed by
> each processor node, at first there are 3 processor nodes
> kstream-transformvalues-3, kstream-filter-4, kstream-aggregate-5 processing
> 4/5k records/min, then ktable-tostream-6 and kstream-transform-7 rump up
> and the previous ones slow down due the higher load
>  - https://imgur.com/5eXpf8l the state stores cache rate starts to
> decrease
>  - https://imgur.com/dwFOb2g put and fetch operations of the window store
> almost remain the same (maybe lowers due higher load)
>  - https://imgur.com/1XZmMW5 commit latency increases
>  - https://imgur.com/UdBpOVU commit rate stays almost the same
>  - https://imgur.com/UJ3JB4f process latency increases
>  - https://imgur.com/55YVmy2 process rate stays the same
>  - https://imgur.com/GMJ3eGV sent records increase because of aggregate
> and
> suppress store changelog records
>  - https://imgur.com/XDm2kX6 sent bytes for those changelog topics
> increase
>
> (full album https://imgur.com/a/tXlJJEO)
>
> Any other metric that might be important?
>
> It seems that the issue is between the aggregate and Ktable.toStream()
>
> After a restart as expected usage go back to normal values
>
> --
> Alessandro Tagliapietra
>
>
> On Mon, Dec 9, 2019 at 7:22 PM Alessandro Tagliapietra <
> tagliapietra.alessan...@gmail.com> wrote:
>
> > You're saying that with a 100ms commit interval, caching won't help
> > because it would still send the compacted changes to the changelog every
> > 100ms?
> >
> > Regarding the custom state store I'll look into that because I didn't go
> > much further than transformers and stores in my kafka experience so I'll
> > need to understand better what that implies.
> >
> > Yeah I only have one window per key in the store.
> >
> > The only thing I don't understand is why cache works 80% of the time and
> > then suddenly the changelog sent bytes increase 90x.
> > I mean, if cache wasn't working, why enabling i

Re: Reducing streams startup bandwidth usage

2019-12-09 Thread Sophie Blee-Goldman
Alright, well I see why you have so much data being sent to the changelog
if each
update involves appending to a list and then writing in the whole list. And
with 340
records/minute I'm actually not sure how the cache could really help at all
when it's
being flushed every 100ms.

Here's kind of a wild idea, if you really only need append semantics: what
if you wrote
a custom StateStore that wrapped the normal RocksDBStore (or
RocksDBWindowStore)
and did the append for you under the hood? The changelogging layer sits
between the
layer that you would call #put on in your transformer and the final layer
that actually writes
to the underlying storage engine. If you insert an extra layer and modify
your transformer
to only call put on the new data (rather than the entire list) then only
this new data will get
sent to the changelog. Your custom storage layer will know it's actually
append semantics,
and add the new data to the existing list before sending it on to RocksDB.

Since you only ever have one window per key in the store (right?) you just
need to make
sure that nothing from the current window gets deleted prematurely. You'd
want to turn off
compaction on the changelog and caching on the store of course, and maybe
give the
changelog some extra retention time to be safe.

Obviously I haven't thoroughly verified this alternative, but it seems like
this approach (or
something to its effect) could help you cut down on the changelog data.
WDYT?

On Mon, Dec 9, 2019 at 4:35 PM Alessandro Tagliapietra <
tagliapietra.alessan...@gmail.com> wrote:

> Hi Sophie,
>
> Just to give a better context, yes we use EOS and the problem happens in
> our aggregation store.
> Basically when windowing data we append each record into a list that's
> stored in the aggregation store.
> We have 2 versions, in production we use the kafka streams windowing API,
> in staging we manually calculate the window end timestamp and aggregate
> using that timestamp.
>
> To give you an example of the staging code, it's a simple transformer that:
>  - if incoming data fits in the same window as the data in store, append
> the data to the existing store list overwriting the same key and nothing is
> sent downstream
>  - if incoming data has a timestamp smaller than the existing store data,
> discard the record
>  - if incoming data has a timestamp bigger than the existing store data,
> send the stored list downstream and store the new window data into the
> store
>
> This way we don't use multiple keys (kafka streams instead uses a store
> where each key is stream-key + window key) as we overwrite the store data
> using the same key over and over.
> So what I would expect is that since we're overwriting the same keys there
> isn't more  and more data to be cached as the number of keys are always the
> same and we don't really need to cache more data over time.
>
> To respond to your questions:
>  - yes when I say that cache "stopped/started" working I mean that at some
> point the store started sending more and more data to che changelog topic
> and then suddenly stopped again even without a restart (a restart always
> fixes the problem).
>  - Yes there are no density changes in the input stream, I've checked the
> number of records sent to the stream input topic and there is a variation
> of ~10-20 records per minute on an average of 340 records per minute. Most
> of the records are also generated by simulators with very predictable
> output rate.
>
> In the meantime I've enabled reporting of debug metrics (so including cache
> hit ratio) to hopefully get better insights the next time it happens.
>
> Thank you in advance
>
> --
> Alessandro Tagliapietra
>
> On Mon, Dec 9, 2019 at 3:57 PM Sophie Blee-Goldman 
> wrote:
>
> > It's an LRU cache, so once it gets full new records will cause older ones
> > to be evicted (and thus sent
> > downstream). Of course this should only apply to records of a different
> > key, otherwise it will just cause
> > an update of that key in the cache.
> >
> > I missed that you were using EOS, given the short commit interval it's
> hard
> > to see those effects.
> > When you say that it stopped working and then appeared to start working
> > again, are you just
> > referring to the amount of data being sent to the changelog? And you can
> > definitely rule out differences
> > in the density of updates in the input stream?
> >
> >
> >
> > On Mon, Dec 9, 2019 at 12:26 PM Alessandro Tagliapietra <
> > tagliapietra.alessan...@gmail.com> wrote:
> >
> > > Hi Sophie,
> > >
> > > thanks fo helping.
> > >
> > > By eviction of older records you mean they get flushed to the changelog
&g

Re: Reducing streams startup bandwidth usage

2019-12-09 Thread Sophie Blee-Goldman
It's an LRU cache, so once it gets full new records will cause older ones
to be evicted (and thus sent
downstream). Of course this should only apply to records of a different
key, otherwise it will just cause
an update of that key in the cache.

I missed that you were using EOS, given the short commit interval it's hard
to see those effects.
When you say that it stopped working and then appeared to start working
again, are you just
referring to the amount of data being sent to the changelog? And you can
definitely rule out differences
in the density of updates in the input stream?



On Mon, Dec 9, 2019 at 12:26 PM Alessandro Tagliapietra <
tagliapietra.alessan...@gmail.com> wrote:

> Hi Sophie,
>
> thanks fo helping.
>
> By eviction of older records you mean they get flushed to the changelog
> topic?
> Or the cache is just full and so all new records go to the changelog topic
> until the old ones are evicted?
>
> Regarding the timing, what timing do you mean? Between when the cache stops
> and starts working again? We're using EOS os I believe the commit interval
> is every 100ms.
>
> Regards
>
> --
> Alessandro Tagliapietra
>
>
>
> On Mon, Dec 9, 2019 at 12:15 PM Sophie Blee-Goldman 
> wrote:
>
> > It might be that the cache appears to "stop working" because it gets
> full,
> > and each
> > new update causes an eviction (of some older record). This would also
> > explain the
> > opposite behavior, that it "starts working" again after some time without
> > being restarted,
> > since the cache is completely flushed on commit. Does the timing seem to
> > align with your
> > commit interval (default is 30s)?
> >
> > On Mon, Dec 9, 2019 at 12:03 AM Alessandro Tagliapietra <
> > tagliapietra.alessan...@gmail.com> wrote:
> >
> > > And it seems that for some reason after a while caching works again
> > > without a restart of the streams application
> > >
> > > [image: Screen Shot 2019-12-08 at 11.59.30 PM.png]
> > >
> > > I'll try to enable debug metrics and see if I can find something useful
> > > there.
> > > Any idea is appreciated in the meantime :)
> > >
> > > --
> > > Alessandro Tagliapietra
> > >
> > > On Sun, Dec 8, 2019 at 12:54 PM Alessandro Tagliapietra <
> > > tagliapietra.alessan...@gmail.com> wrote:
> > >
> > >> It seems that even with caching enabled, after a while the sent bytes
> > >> stil go up
> > >>
> > >> [image: Screen Shot 2019-12-08 at 12.52.31 PM.png]
> > >>
> > >> you can see the deploy when I've enabled caching but it looks like
> it's
> > >> still a temporary solution.
> > >>
> > >> --
> > >> Alessandro Tagliapietra
> > >>
> > >>
> > >> On Sat, Dec 7, 2019 at 10:08 AM Alessandro Tagliapietra <
> > >> tagliapietra.alessan...@gmail.com> wrote:
> > >>
> > >>> Could be, but since we have a limite amount of input keys (~30),
> > >>> windowing generates new keys but old ones are never touched again
> > since the
> > >>> data per key is in order, I assume it shouldn't be a big deal for it
> to
> > >>> handle 30 keys
> > >>> I'll have a look at cache metrics and see if something pops out
> > >>>
> > >>> Thanks
> > >>>
> > >>> --
> > >>> Alessandro Tagliapietra
> > >>>
> > >>>
> > >>> On Sat, Dec 7, 2019 at 10:02 AM John Roesler 
> > >>> wrote:
> > >>>
> > >>>> Hmm, that’s a good question. Now that we’re talking about caching, I
> > >>>> wonder if the cache was just too small. It’s not very big by
> default.
> > >>>>
> > >>>> On Sat, Dec 7, 2019, at 11:16, Alessandro Tagliapietra wrote:
> > >>>> > Ok I'll check on that!
> > >>>> >
> > >>>> > Now I can see that with caching we went from 3-4MB/s to 400KB/s,
> > that
> > >>>> will
> > >>>> > help with the bill.
> > >>>> >
> > >>>> > Last question, any reason why after a while the regular windowed
> > >>>> stream
> > >>>> > starts sending every update instead of caching?
> > >>>> > Could it be because it doesn't have any more memory available? Any
> > >>>> other
> > >>>> > possible reason?
> &g

Re: Reducing streams startup bandwidth usage

2019-12-09 Thread Sophie Blee-Goldman
It might be that the cache appears to "stop working" because it gets full,
and each
new update causes an eviction (of some older record). This would also
explain the
opposite behavior, that it "starts working" again after some time without
being restarted,
since the cache is completely flushed on commit. Does the timing seem to
align with your
commit interval (default is 30s)?

On Mon, Dec 9, 2019 at 12:03 AM Alessandro Tagliapietra <
tagliapietra.alessan...@gmail.com> wrote:

> And it seems that for some reason after a while caching works again
> without a restart of the streams application
>
> [image: Screen Shot 2019-12-08 at 11.59.30 PM.png]
>
> I'll try to enable debug metrics and see if I can find something useful
> there.
> Any idea is appreciated in the meantime :)
>
> --
> Alessandro Tagliapietra
>
> On Sun, Dec 8, 2019 at 12:54 PM Alessandro Tagliapietra <
> tagliapietra.alessan...@gmail.com> wrote:
>
>> It seems that even with caching enabled, after a while the sent bytes
>> stil go up
>>
>> [image: Screen Shot 2019-12-08 at 12.52.31 PM.png]
>>
>> you can see the deploy when I've enabled caching but it looks like it's
>> still a temporary solution.
>>
>> --
>> Alessandro Tagliapietra
>>
>>
>> On Sat, Dec 7, 2019 at 10:08 AM Alessandro Tagliapietra <
>> tagliapietra.alessan...@gmail.com> wrote:
>>
>>> Could be, but since we have a limite amount of input keys (~30),
>>> windowing generates new keys but old ones are never touched again since the
>>> data per key is in order, I assume it shouldn't be a big deal for it to
>>> handle 30 keys
>>> I'll have a look at cache metrics and see if something pops out
>>>
>>> Thanks
>>>
>>> --
>>> Alessandro Tagliapietra
>>>
>>>
>>> On Sat, Dec 7, 2019 at 10:02 AM John Roesler 
>>> wrote:
>>>
 Hmm, that’s a good question. Now that we’re talking about caching, I
 wonder if the cache was just too small. It’s not very big by default.

 On Sat, Dec 7, 2019, at 11:16, Alessandro Tagliapietra wrote:
 > Ok I'll check on that!
 >
 > Now I can see that with caching we went from 3-4MB/s to 400KB/s, that
 will
 > help with the bill.
 >
 > Last question, any reason why after a while the regular windowed
 stream
 > starts sending every update instead of caching?
 > Could it be because it doesn't have any more memory available? Any
 other
 > possible reason?
 >
 > Thank you so much for your help
 >
 > --
 > Alessandro Tagliapietra
 >
 >
 > On Sat, Dec 7, 2019 at 9:14 AM John Roesler 
 wrote:
 >
 > > Ah, yes. Glad you figured it out!
 > >
 > > Caching does not reduce EOS guarantees at all. I highly recommend
 using
 > > it. You might even want to take a look at the caching metrics to
 make sure
 > > you have a good hit ratio.
 > >
 > > -John
 > >
 > > On Sat, Dec 7, 2019, at 10:51, Alessandro Tagliapietra wrote:
 > > > Never mind I've found out I can use `.withCachingEnabled` on the
 store
 > > > builder to achieve the same thing as the windowing example as
 > > > `Materialized.as` turns that on by default.
 > > >
 > > > Does caching in any way reduces the EOS guarantees?
 > > >
 > > > --
 > > > Alessandro Tagliapietra
 > > >
 > > >
 > > > On Sat, Dec 7, 2019 at 1:12 AM Alessandro Tagliapietra <
 > > > tagliapietra.alessan...@gmail.com> wrote:
 > > >
 > > > > Seems my journey with this isn't done just yet,
 > > > >
 > > > > This seems very complicated to me but I'll try to explain it as
 best I
 > > can.
 > > > > To better understand the streams network usage I've used
 prometheus
 > > with
 > > > > the JMX exporter to export kafka metrics.
 > > > > To check the amount of data we use I'm looking at the increments
 > > > > of kafka_producer_topic_metrics_byte_total and
 > > > > kafka_producer_producer_topic_metrics_record_send_total,
 > > > >
 > > > > Our current (before the change mentioned above) code looks like
 this:
 > > > >
 > > > > // This transformers just pairs a value with the previous one
 storing
 > > the
 > > > > temporary one in a store
 > > > > val pairsStream = metricStream
 > > > >   .transformValues(ValueTransformerWithKeySupplier {
 PairTransformer()
 > > },
 > > > > "LastValueStore")
 > > > >   .filter { _, value: MetricSequence? -> value != null }
 > > > >
 > > > > // Create a store to store suppressed windows until a new one is
 > > received
 > > > > val suppressStoreSupplier =
 > > > >
 > >
 Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("suppress-store"),
 > > > > ..
 > > > >
 > > > > // Window and aggregate data in 1 minute intervals
 > > > > val aggregatedStream = pairsStream
 > > > >   .groupByKey()
 > > > >   .windowedBy(TimeWindows.of(Duration.ofMinutes(1)))
 > > > >   

Re: Very low bytesout for internal repartition topic, is it related to RocksDB?

2019-10-22 Thread Sophie Blee-Goldman
I'm less familiar with that part of the code but that sounds correct to me.
You're default request timeout is 300 seconds though, is that right? Seems
like it should be large enough in most scenarios. Did you see any network
outages around that time?

On Wed, Oct 16, 2019 at 10:30 AM Xiyuan Hu  wrote:

> Hi Sophie,
>
> A follow up questions, I set the cache to 0 and after 12 hours
> running, some nodes turn dead with error messages: task [1_0] Abort
> sending since an error caught with a previous record (key
> 333283323209294164cf16bb333c14a3506001b8fec3XXX3\x00\x00\x01m\xD5\x155\x80
> value [B@33574f49 timestamp null) to topic
> TEST-KTABLE-SUPPRESS-STATE-STORE-09-changelog due to
> org.apache.kafka.common.errors.TimeoutException: Expiring 28 record(s)
> for TEST-KTABLE-SUPPRESS-STATE-STORE-09-changelog-0:300100 ms
> has passed since batch creation
>
> My timeout related setting:
> request.timeout.ms = 30
> poll.ms = 100
> retry.backoff.ms = 100
> linger.ms = 100
>
> Once the node has above exception, it will turn to DEAD state and
> can't recover. Do you know what might cause this timeout issue? Does
> it mean it takes too long to send the messages to this changelog
> topic?
>
> Thanks
> Kathy
>
> On Mon, Oct 14, 2019 at 10:54 PM Sophie Blee-Goldman
>  wrote:
> >
> > Glad that helped!
> >
> > Honestly I can't say I recognize either that exception but I'm fairly
> > confident it's
> > not directly related to rocksdb or Streams. It sounds like a connection
> > somewhere
> > got screwed up, which would be more of a configuration issue I probably
> > can't help with.
> >
> > Of course it's always possible rocksdb is doing something weird that
> we're
> > unaware of.
> > If you revert the changes you just made and don't see these issues, I'd
> say
> > try continuing
> > to use your own rocksdb config setter and/or reset the app (including
> > clearing local state).
> >
> > Cheers,
> > Sophie
> >
> > On Mon, Oct 14, 2019 at 4:35 PM Xiyuan Hu  wrote:
> >
> > > Hi Sophie,
> > >
> > > Thanks for the information! After setting cache.max.bytes.buffering to
> > > zero, commenting out my customized rocksDB config and using default
> > > rocksDB config, I do see the repartition topic bytesout goes up. But I
> > > noticed that some nodes have IO exception as: An I/O error has
> > > occurred while writing a response message entity to the container
> > > output
> > > stream.(org.glassfish.jersey.server.internal.process.MappableException:
> > > org.apache.catalina.connector.ClientAbortException:
> > > java.io.IOException: Broken pipe). Is is also related to the rocksDB
> > > read and write? Anything I should do to get rid of this exception?
> > >
> > > Thanks a lot!
> > >
> > > On Mon, Oct 14, 2019 at 6:10 PM Sophie Blee-Goldman <
> sop...@confluent.io>
> > > wrote:
> > > >
> > > > Ah ok, 2.3.0 has a known performance issue in the caching layer which
> > > tends
> > > > to get worse the larger the cache size. That might explain what
> you're
> > > > seeing with
> > > > regards to the traffic correlation.
> > > >
> > > > It's fixed in 2.3.1 which should be released very soon, but until
> then
> > > you
> > > > might want
> > > > to try turning off the Streams cache (by setting
> > > cache.max.bytes.buffering
> > > > to zero,
> > > > although you can also use .withCachingDisabled to turn it off only
> for
> > > the
> > > > large/problem
> > > > store)
> > > >
> > > > Of course any memory you reclaim by turning off the Streams cache can
> > > just
> > > > go towards the
> > > > rocksdb cache instead, just note that the rocksdb cache comes from
> > > off-heap
> > > > memory
> > > > while the Streams cache would be taken from heap memory allocated to
> the
> > > > jvm.
> > > >
> > > > On Mon, Oct 14, 2019 at 2:48 PM Xiyuan Hu 
> wrote:
> > > >
> > > > > Hi Sophie,
> > > > >
> > > > > Thanks for the help!
> > > > > I'm using version 2.3.0.
> > > > >
> > > > > The repartition topic with huge lag is the created during the first
> > > > > reduce method, named "XX-KSTREAM-STATE-STORE-0030-repartition". All
> > > > > other internal topics have almost zero l

Re: Very low bytesout for internal repartition topic, is it related to RocksDB?

2019-10-14 Thread Sophie Blee-Goldman
Glad that helped!

Honestly I can't say I recognize either that exception but I'm fairly
confident it's
not directly related to rocksdb or Streams. It sounds like a connection
somewhere
got screwed up, which would be more of a configuration issue I probably
can't help with.

Of course it's always possible rocksdb is doing something weird that we're
unaware of.
If you revert the changes you just made and don't see these issues, I'd say
try continuing
to use your own rocksdb config setter and/or reset the app (including
clearing local state).

Cheers,
Sophie

On Mon, Oct 14, 2019 at 4:35 PM Xiyuan Hu  wrote:

> Hi Sophie,
>
> Thanks for the information! After setting cache.max.bytes.buffering to
> zero, commenting out my customized rocksDB config and using default
> rocksDB config, I do see the repartition topic bytesout goes up. But I
> noticed that some nodes have IO exception as: An I/O error has
> occurred while writing a response message entity to the container
> output
> stream.(org.glassfish.jersey.server.internal.process.MappableException:
> org.apache.catalina.connector.ClientAbortException:
> java.io.IOException: Broken pipe). Is is also related to the rocksDB
> read and write? Anything I should do to get rid of this exception?
>
> Thanks a lot!
>
> On Mon, Oct 14, 2019 at 6:10 PM Sophie Blee-Goldman 
> wrote:
> >
> > Ah ok, 2.3.0 has a known performance issue in the caching layer which
> tends
> > to get worse the larger the cache size. That might explain what you're
> > seeing with
> > regards to the traffic correlation.
> >
> > It's fixed in 2.3.1 which should be released very soon, but until then
> you
> > might want
> > to try turning off the Streams cache (by setting
> cache.max.bytes.buffering
> > to zero,
> > although you can also use .withCachingDisabled to turn it off only for
> the
> > large/problem
> > store)
> >
> > Of course any memory you reclaim by turning off the Streams cache can
> just
> > go towards the
> > rocksdb cache instead, just note that the rocksdb cache comes from
> off-heap
> > memory
> > while the Streams cache would be taken from heap memory allocated to the
> > jvm.
> >
> > On Mon, Oct 14, 2019 at 2:48 PM Xiyuan Hu  wrote:
> >
> > > Hi Sophie,
> > >
> > > Thanks for the help!
> > > I'm using version 2.3.0.
> > >
> > > The repartition topic with huge lag is the created during the first
> > > reduce method, named "XX-KSTREAM-STATE-STORE-0030-repartition". All
> > > other internal topics have almost zero lags. For my case, how could I
> > > find out if rocksDB causes the lags? One thing I noticed is, when the
> > > source traffic is about 30K/sec, I don't have any lags for the entire
> > > system but when the traffic goes up to 100K/sec, it has a huge lag. As
> > > you mentioned, if the memory usage is high, should I set any rocksDB
> > > memory related config to higher value? Thanks a lot!
> > > My topology is like below:
> > >
> > > final KStream source = builder.stream(inputTopic);
> > > KStream deserializedStream = source.mapValues( ... });
> > >
> > > KStream, Event> dedupedStream =
> > > deserializedStream.selectKey( ... )
> > > .groupByKey(Grouped.with(Serdes.String(), new
> JsonSerde<>(Event.class)))
> > >
> .windowedBy(TimeWindows.of(Duration.ofMinutes(60)).grace(Duration.ZERO))
> > > .reduce((value1, value2) -> value2)
> > > .suppress(untilWindowCloses(Suppressed.BufferConfig.unbounded()))
> > > .toStream();
> > >
> > > dedupedStream.selectKey( ... )
> > > .mapValues( ... )
> > > .filter(...)
> > > .groupByKey(Grouped.with(Serdes.String(), new MessagetSerde()))
> > > .reduce((value1, value2) -> {
> > > long count1 = value1.getCount();
> > > long count2 = value2.getCount();
> > > value2.setCount(count1 + count2);
> > > return value2;
> > > }
> > > )
> > > .toStream()
> > > .selectKey( ... )
> > > .to(outputTopic);
> > >
> > > On Mon, Oct 14, 2019 at 3:53 PM Sophie Blee-Goldman <
> sop...@confluent.io>
> > > wrote:
> > > >
> > > > Out of curiosity, which version are you using?
> > > >
> > > > There's nothing that really jumps out at me as problematic in your
> > > > RocksDBConfigSetter, but note that I think you may need to increase
> > > > the number of threads in the "LOW priority thread pool" in addition
> to
> > > > settin

Re: Very low bytesout for internal repartition topic, is it related to RocksDB?

2019-10-14 Thread Sophie Blee-Goldman
Ah ok, 2.3.0 has a known performance issue in the caching layer which tends
to get worse the larger the cache size. That might explain what you're
seeing with
regards to the traffic correlation.

It's fixed in 2.3.1 which should be released very soon, but until then you
might want
to try turning off the Streams cache (by setting cache.max.bytes.buffering
to zero,
although you can also use .withCachingDisabled to turn it off only for the
large/problem
store)

Of course any memory you reclaim by turning off the Streams cache can just
go towards the
rocksdb cache instead, just note that the rocksdb cache comes from off-heap
memory
while the Streams cache would be taken from heap memory allocated to the
jvm.

On Mon, Oct 14, 2019 at 2:48 PM Xiyuan Hu  wrote:

> Hi Sophie,
>
> Thanks for the help!
> I'm using version 2.3.0.
>
> The repartition topic with huge lag is the created during the first
> reduce method, named "XX-KSTREAM-STATE-STORE-0030-repartition". All
> other internal topics have almost zero lags. For my case, how could I
> find out if rocksDB causes the lags? One thing I noticed is, when the
> source traffic is about 30K/sec, I don't have any lags for the entire
> system but when the traffic goes up to 100K/sec, it has a huge lag. As
> you mentioned, if the memory usage is high, should I set any rocksDB
> memory related config to higher value? Thanks a lot!
> My topology is like below:
>
> final KStream source = builder.stream(inputTopic);
> KStream deserializedStream = source.mapValues( ... });
>
> KStream, Event> dedupedStream =
> deserializedStream.selectKey( ... )
> .groupByKey(Grouped.with(Serdes.String(), new JsonSerde<>(Event.class)))
> .windowedBy(TimeWindows.of(Duration.ofMinutes(60)).grace(Duration.ZERO))
> .reduce((value1, value2) -> value2)
> .suppress(untilWindowCloses(Suppressed.BufferConfig.unbounded()))
> .toStream();
>
> dedupedStream.selectKey( ... )
> .mapValues( ... )
> .filter(...)
> .groupByKey(Grouped.with(Serdes.String(), new MessagetSerde()))
> .reduce((value1, value2) -> {
> long count1 = value1.getCount();
> long count2 = value2.getCount();
> value2.setCount(count1 + count2);
> return value2;
> }
> )
> .toStream()
> .selectKey( ... )
> .to(outputTopic);
>
> On Mon, Oct 14, 2019 at 3:53 PM Sophie Blee-Goldman 
> wrote:
> >
> > Out of curiosity, which version are you using?
> >
> > There's nothing that really jumps out at me as problematic in your
> > RocksDBConfigSetter, but note that I think you may need to increase
> > the number of threads in the "LOW priority thread pool" in addition to
> > setting the maxBackgroundCompactions -- this can be done as
> >
> > options.setEnv(Env.getDefault().setBackgroundThreads(n,
> > Env.COMPACTION_POOL));
> >
> > Is your disk throughput possibly the bottleneck? Note that if the
> > repartition topic
> > is followed by a subtopology doing heavy processing this will likely show
> > up as
> > lag like you describe. Also, if you have a large number of stateful tasks
> > (large
> > number of stateful operations, and/or large number of partitions) each
> one
> > will
> > have its own separate rocksdb instance, and the memory usage could be
> quite
> > high (which can cause rocks to page in/out things like index blocks which
> > always
> > need to be read before a lookup) -- I'd recommend also setting
> >
> > tableConfig.setPinL0FilterAndIndexBlocksInCache(true);
> >
> >
> > On Sun, Oct 13, 2019 at 6:40 PM Xiyuan Hu  wrote:
> >
> > > Hi,
> > >
> > > I'm running a Kafka Streams app with windowing function. I noticed
> > > that internal topic -repartition has huge lag while the system CPU
> > > usage is low and app is stable(join rate is almost 0).
> > >
> > > The repartition topic is an internal topic and created by the
> > > application automatically. The bytes in per sec for this topic is
> > > about 65MB/sec while the bytes out for this topic is only 15MB/sec. I
> > > have tried a couple configs to customize RocksDB config, but none of
> > > it could increase the bytes out value.
> > >
> > > I changed the default RocksDB block case size to 125MB and block size
> > > to 125MB as well. Also set the max write buffer number to 3. But it
> > > didn't help.
> > >
> > > May I know what I missed here? What's the best way to find why
> > > internal repartition topic has huge lags?
> > >
> > > Thanks for all the helps!!
> > >
> > > My RocksDB config:
> > > public

Re: Long delay between incoming and outgoing messages using kafka streams

2019-10-14 Thread Sophie Blee-Goldman
Honestly I can't say whether 256 partitions is enough to trigger the
performance issues
in 2.3.0 but I'd definitely recommend upgrading as soon as you can, just in
case. On a
related note, how many instances, with how many threads, are you running?
256 partitions
with several subtopologies will result in a large number of tasks so make
sure you're parallelizing
as much as possible.

That said, I may have misread your topology the first time around -- are
you saying that you
load data from topic2, join it with some other data, then write back to the
same topic (topic2)?
Having cycles in your topology is not really supported, as stream
processing should generally
be a DAG -- when you introduce a cycle things often don't behave the way
you might expect
or want. Is there a particular reason you need to do this?

> the delay from sending a message on topic1 to the point where messages
received on topic2
are passing the join
Also, can you explain what you mean by this? That is, can you explain how
you are determining
when  messages are "passing the join" -- are you just reading from topic2?
Are you distinguishing
the "new" data, written by Streams, from the "original data" that is
written to this topic from elsewhere?

On Thu, Oct 10, 2019 at 3:05 AM Petter Arvidsson 
wrote:

> Hi Sophie,
>
> Thank you for your response.
>
> I tested the proposed setting for CACHE_MAX_BYTES_BUFFERING_CONFIG and it
> seem to not significantly change the behavior of the application. The
> latency remains very similar. The documentation states the following
> regarding CACHE_MAX_BYTES_BUFFERING_CONFIG and COMMIT_INTERVAL_MS_CONFIG:
>
> ---
> To enable caching but still have an upper bound on how long records will be
> cached, you can set the commit interval. In this example, it is set to 1000
> milliseconds:
>
> Properties props = new Properties();
> // Enable record cache of size 10 MB.
> props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 *
> 1024L);
> // Set commit interval to 1 second.
> props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
> ---
>
> Which made me believe that the COMMIT_INTERVAL_MS_CONFIG would already
> "override" the CACHE_MAX_BYTES_BUFFERING_CONFIG and provide an upper bound
> of the latency of 1s per processing step by flushing buffers every second.
> Is this the case or does these two configuration values interact in some
> other way?
>
> We are using 256 partitions for all our topics. Is this to be considered a
> very high partition count? Do you think we might be affected by the bug in
> 2.3.0?
>
> Thank you for the help!
>
> Best regards,
> Petter
>
> On Wed, Oct 9, 2019 at 7:33 PM Sophie Blee-Goldman 
> wrote:
>
> > Hi Petter,
> >
> > I'd recommend turning off caching by setting
> > p.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING, 0);
> >
> > 2.3.0 also has some known performance issues that will be fixed in 2.3.1,
> > but they
> > shouldn't be noticeable if you turn caching off and aren't
> reading/writing
> > to topics
> > with a very high partition count. These are fixed in 2.3.1 which should
> be
> > released
> > soon for you to upgrade, but the caching is likely the main reason for
> the
> > latency you see.
> >
> > I'd also note that Streams, and Kafka in general, is typically tuned for
> > high
> > throughput rather than low latency, so I wouldn't be too concerned about
> > a large latency unless that is a specific requirement.
> >
> > Cheers,
> > Sophie
> >
> > On Wed, Oct 9, 2019 at 6:05 AM Petter Arvidsson <
> > petter.arvids...@relayr.io>
> > wrote:
> >
> > > Hi,
> > >
> > > I have a fairly simple kafka streams application that read messages
> from
> > > two topics. The problem I am facing is that the delay between sending
> > > events to the streams application and it producing results is very high
> > (as
> > > in several minutes). My question is: how can I make this latency
> smaller?
> > >
> > > The streams is doing the following:
> > > ktable1 = topic1
> > >   -> (filter out messages using flatMap)
> > >   -> groupBy (with new key, adds internal rekeying topic)
> > >   -> aggregate (in memory store backed by internal compacted topic)
> > >
> > > ktabe2 = topic2
> > >   -> (rekey to same key as ktable1 over internal topic)
> > >   -> join (with ktable1)
> > >   -> aggregate (in memory store backed by internal compacted topic)
> > >
> > > ktable2.toStream.to(topic2)
> > >
> > > Ktable1 keep configu

Re: Very low bytesout for internal repartition topic, is it related to RocksDB?

2019-10-14 Thread Sophie Blee-Goldman
Out of curiosity, which version are you using?

There's nothing that really jumps out at me as problematic in your
RocksDBConfigSetter, but note that I think you may need to increase
the number of threads in the "LOW priority thread pool" in addition to
setting the maxBackgroundCompactions -- this can be done as

options.setEnv(Env.getDefault().setBackgroundThreads(n,
Env.COMPACTION_POOL));

Is your disk throughput possibly the bottleneck? Note that if the
repartition topic
is followed by a subtopology doing heavy processing this will likely show
up as
lag like you describe. Also, if you have a large number of stateful tasks
(large
number of stateful operations, and/or large number of partitions) each one
will
have its own separate rocksdb instance, and the memory usage could be quite
high (which can cause rocks to page in/out things like index blocks which
always
need to be read before a lookup) -- I'd recommend also setting

tableConfig.setPinL0FilterAndIndexBlocksInCache(true);


On Sun, Oct 13, 2019 at 6:40 PM Xiyuan Hu  wrote:

> Hi,
>
> I'm running a Kafka Streams app with windowing function. I noticed
> that internal topic -repartition has huge lag while the system CPU
> usage is low and app is stable(join rate is almost 0).
>
> The repartition topic is an internal topic and created by the
> application automatically. The bytes in per sec for this topic is
> about 65MB/sec while the bytes out for this topic is only 15MB/sec. I
> have tried a couple configs to customize RocksDB config, but none of
> it could increase the bytes out value.
>
> I changed the default RocksDB block case size to 125MB and block size
> to 125MB as well. Also set the max write buffer number to 3. But it
> didn't help.
>
> May I know what I missed here? What's the best way to find why
> internal repartition topic has huge lags?
>
> Thanks for all the helps!!
>
> My RocksDB config:
> public static class CustomRocksDBConfig implements RocksDBConfigSetter {
> private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(125 *
> 1024L * 1024L);
>
> @Override
> public void setConfig(final String storeName, final Options
> options, final Map configs) {
> int n = Runtime.getRuntime().availableProcessors();
> options.setMaxBackgroundCompactions(n);
> options.setWriteBufferSize(125 * 1024 * 1024);
> BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
> options.tableFormatConfig();
> tableConfig.setBlockCache(cache);
> tableConfig.setBlockCacheSize(125 * 1024 * 1024L);
> tableConfig.setBlockSize(125 * 1024L);
> tableConfig.setCacheIndexAndFilterBlocks(true);
> options.setTableFormatConfig(tableConfig);
> options.setMaxWriteBufferNumber(3);
> }
>
> public void close(final String storeName, final Options options) {
> // See #5 below.
> cache.close();
> }
>
> }
>
> Thanks
> Kathy
>


Re: Stale data in KStream->KTable join

2019-10-14 Thread Sophie Blee-Goldman
I'm not sure you really need to/ should be sending data back to the table
with .to(table) -- that was kind of the original issue, since it introduces
a cycle*
into your topology (which should be a DAG).

*Of course it's still technically a cycle if you implement the store
updates manually
with a transformer, but the PAPI is supposed to allow you to do anything
while
the DSL assumes things about your topology, and thus, may not work as
expected
if those assumptions are broken

On Mon, Oct 7, 2019 at 10:19 AM Trey Hutcheson 
wrote:

> Update - I tried Sophie's suggestion; I implemented a Transformer that
> performs puts on the table's backing store. I hid the complexities behind a
> kotlin extension method. So now the code looks like this (pesudocode):
>
> KStream.commit() { // transformer is implemented here }
>
> stream.join(table) { event, detailRecord ->
>   // append record
> }
> .commit()
> .to(table)
>
> So this approach achieves the desired result - the data is no longer stale.
> *However*, my logs are now littered with statements like this:
>   WARN  o.a.k.s.k.internals.KTableSource - Detected out-of-order KTable
> update for cdr at offset 89, partition 1.
>
> Basically, there's a corresponding warning emitted for every put into the
> backing store. I've tried swapping the order of the logical "commit" and
> "to" operations, and still receive the warnings.
>
> So, what is the proper approach here? Just keep it as-is and ignore the
> warnings?
>
>
> On Sat, Oct 5, 2019 at 10:23 AM Trey Hutcheson 
> wrote:
>
> > Thank you for your response Sophie.
> >
> > What confuses me is that I have already implemented this same pattern
> > (join to table, mutate entity, write back to table) in two other streams
> > applications without any issue whatsoever. After thinking about it, I
> think
> > the difference here is the input data. In the other applications, the
> > mutations were caused by rest api invocations and were relatively low
> > frequency. In this case, we have course grained buckets of data,
> semi-batch
> > data, that is decomposed and flatMapped into a bunch of discreet events
> > which are then effectively bursted to the events topic. In this
> particular
> > case, I don't really care about latency *or* throughput; this is
> > effectively a write-behind reporting application with no time
> sensitivity.
> >
> > Anyway, I've considered another option; I might just change the payload
> of
> > the KStream to KStream>. The individual
> > messages on that topic will be larger, but lower frequency. But that
> won't
> > solve the problem of replaying from earlier offsets - which effectively
> > will be another bursty scenario.
> >
> > If that doesn't work as I hope, then I'll drop down and implement a
> > transformer that injects to the data store itself. That just feels icky
> > though.
> >
> > On Fri, Oct 4, 2019 at 5:09 PM Sophie Blee-Goldman 
> > wrote:
> >
> >> I think the issue here is that you're basically creating a cycle in your
> >> streams topology,
> >> which is generally supposed to be a DAG. If I understand correctly,
> rather
> >> than writing
> >> the new data to the underlying store you're sending it to the topic from
> >> which the table
> >> is built. Is that right?
> >>
> >> The problem is that for this to work, Streams has to recognize that you
> >> sent an
> >> update to the table while processing the stream, and has to somehow know
> >> that
> >> you want that update to be processed before the next record from the
> >> stream
> >> is processed.
> >> But there aren't any kind of guarantees as to which will be processed
> >> first, for good reason --
> >> how should Streams know which order you want them to be processed? And
> the
> >> update to
> >> the table will have some nonzero latency as it has to go to the table's
> >> topic, then be read from
> >> the topic by Streams. Is it supposed to block all processing until it's
> >> received an update to the table?
> >>
> >> All that said, what you're trying to do is not impossible, though you
> >> might
> >> need to use the
> >> lower-level Processor API (or attach a transformer). Instead of trying
> to
> >> write to the table
> >> and then hope streams will process the update first, you can attach your
> >> own state store
> >> and write directly to it. Then, any subsequent reads will reflect the
> >> latest updates.
> >>
> >> Hope 

Re: Long delay between incoming and outgoing messages using kafka streams

2019-10-09 Thread Sophie Blee-Goldman
Hi Petter,

I'd recommend turning off caching by setting
p.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING, 0);

2.3.0 also has some known performance issues that will be fixed in 2.3.1,
but they
shouldn't be noticeable if you turn caching off and aren't reading/writing
to topics
with a very high partition count. These are fixed in 2.3.1 which should be
released
soon for you to upgrade, but the caching is likely the main reason for the
latency you see.

I'd also note that Streams, and Kafka in general, is typically tuned for
high
throughput rather than low latency, so I wouldn't be too concerned about
a large latency unless that is a specific requirement.

Cheers,
Sophie

On Wed, Oct 9, 2019 at 6:05 AM Petter Arvidsson 
wrote:

> Hi,
>
> I have a fairly simple kafka streams application that read messages from
> two topics. The problem I am facing is that the delay between sending
> events to the streams application and it producing results is very high (as
> in several minutes). My question is: how can I make this latency smaller?
>
> The streams is doing the following:
> ktable1 = topic1
>   -> (filter out messages using flatMap)
>   -> groupBy (with new key, adds internal rekeying topic)
>   -> aggregate (in memory store backed by internal compacted topic)
>
> ktabe2 = topic2
>   -> (rekey to same key as ktable1 over internal topic)
>   -> join (with ktable1)
>   -> aggregate (in memory store backed by internal compacted topic)
>
> ktable2.toStream.to(topic2)
>
> Ktable1 keep configuration that allows messages to pass through and be
> aggregated into ktable2. Ktable2 keeps aggregates based on messages on
> topic2. Ktable2.toStream is then used to put the aggregated messages back
> out on topic2. The "problem" (or misunderstanding as to how kafka stream is
> processing messages) is that the delay from sending a message on topic1 to
> the point where messages received on topic2 are passing the join is several
> minutes. With the settings I have (see below) on a not that heavily loaded
> system, I would assume the latency would be a couple of seconds (based on
> the COMMIT_INTERVAL_MS_CONFIG).
>
> I use the following settings (as well as settings for bootstrap servers,
> application id and so forth):
> p.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000)
> p.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
> p.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 2)
>
> The store used for the KTables is the one returned by
> "Stores.inMemoryKeyValueStore()".
>
> Kafka libraries use version "2.3.0" and the "kafka-streams-scala" scaladsl
> is used to build the streams. The broker is using version "1.1.0".
>
> Best regards,
> Petter
>


Re: Stream branching and states

2019-10-06 Thread Sophie Blee-Goldman
Hi Navneeth,

I think your plan to implement branching with the Processor API sounds
reasonable, but I'm
wondering why not just use the DSL branch operator and connect some
transformers? That
will let you mix the DSL and PAPI so you don't have to re-implement
anything that is already
in the DSL.

If you supplied a rocksdb store supplier, then yes the state will be held
in rocksdb and not in
memory. Note that this also the default, ie if you don't pass in any
supplier it will default to
using rocksdb. The logging option is, as you describe, to provide fault
tolerance and rebuild
the existing state from the changelog in case your app goes down. This will
also be used in
case you add another instance to scale out -- in that case the global state
will need to be rebuilt,
and any non-global state stores that are assigned to the new instance to
balance the cluster
will also need to be built. You almost definitely want to enable logging,
it will still contain only
the current value vs potentially containing nothing at all if you scale out
or instance restarts.
Global stores technically do not enable logging since it is unnecessary,
they just read from an
input topic so they can just re-read from that same topic rather than a
changelog which would
duplicate the same data.

The caching option differs slightly depending on whether you're using it in
the DSL or PAPI, but you
can read about it here:
https://kafka.apache.org/10/documentation/streams/developer-guide/memory-mgmt.html
Basically it adds an in-memory cache above the state stores to reduce disk
reads/writes. In the DSL,
it also reduces downstream traffic by buffering records with the same key.

On Fri, Oct 4, 2019 at 11:37 PM Navneeth Krishnan 
wrote:

> Hi All,
>
> I'm waiting for inputs to proceed with my POC and convert existing
> applications from flink to kafka streams. It would be really helpful if I
> can get some clarity on this.
>
> Thanks
>
> On Wed, Oct 2, 2019 at 7:05 PM Navneeth Krishnan  >
> wrote:
>
> > Hello All,
> >
> > Any inputs?
> >
> > Thanks
> >
> > On Tue, Oct 1, 2019 at 12:40 PM Navneeth Krishnan <
> > reachnavnee...@gmail.com> wrote:
> >
> >> Hi All,
> >>
> >> I found the below example on how branching can be achieved with kafka
> >> streams. I want to implement the same with processor API and the way I
> get
> >> it is to have the same input topic  processed with multiple process
> >> functions and output the data based on the filters. Is this the right
> >> understanding?
> >>
> >> final StreamsBuilder builder = new StreamsBuilder();
> >> KStream source_o365_user_activity =
> builder.stream("o365_user_activity");KStream[] branches =
> source_o365_user_activity.branch(
> >>   (key, value) -> (value.contains("Operation\":\"SharingSet") &&
> value.contains("ItemType\":\"File")),
> >>   (key, value) ->
> (value.contains("Operation\":\"AddedToSecureLink") &&
> value.contains("ItemType\":\"File")),
> >>   (key, value) -> true
> >>  );
> >>
> >> branches[0].to("o365_sharing_set_by_date");
> >> branches[1].to("o365_added_to_secure_link_by_date");
> >> branches[2].to("o365_user_activity_by_date");
> >>
> >>
> >> Also is the global state kept in memory or on rocksdb? I'm providing a
> >> state store supplier with rocksdb as the option.
> >>
> >> I believe the logging option is to rebuild the state from start when the
> >> application goes down. For my scenario I just need the current value, in
> >> that case should logging be enabled and for global state store it says
> >> logging cannot be enabled. Also what is the caching option? The
> >> documentation doesn't have enough details on these.
> >>
> >> StoreBuilder> globalState =
> Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("test"),
> Serdes.String(), Serdes.String())
> >> .withLoggingDisabled()
> >> .withCachingEnabled();
> >>
> >>
> >> One last question, during auto scaling if I bring up a new node I would
> >> assume some of the traffic will be reallocated to the new node. What
> will
> >> happen to the rocksdb state that was accumulated in the already running
> >> node?
> >>
> >> Thanks, appreciate all the help.
> >>
> >>
>


Re: Stale data in KStream->KTable join

2019-10-04 Thread Sophie Blee-Goldman
I think the issue here is that you're basically creating a cycle in your
streams topology,
which is generally supposed to be a DAG. If I understand correctly, rather
than writing
the new data to the underlying store you're sending it to the topic from
which the table
is built. Is that right?

The problem is that for this to work, Streams has to recognize that you
sent an
update to the table while processing the stream, and has to somehow know
that
you want that update to be processed before the next record from the stream
is processed.
But there aren't any kind of guarantees as to which will be processed
first, for good reason --
how should Streams know which order you want them to be processed? And the
update to
the table will have some nonzero latency as it has to go to the table's
topic, then be read from
the topic by Streams. Is it supposed to block all processing until it's
received an update to the table?

All that said, what you're trying to do is not impossible, though you might
need to use the
lower-level Processor API (or attach a transformer). Instead of trying to
write to the table
and then hope streams will process the update first, you can attach your
own state store
and write directly to it. Then, any subsequent reads will reflect the
latest updates.

Hope that makes sense!
Sophie

On Fri, Oct 4, 2019 at 2:33 PM Boyang Chen 
wrote:

> Hey Trey,
>
> as I was reading, several suggestions I have are:
>
> 1. Could you revert 0ms commit interval to default? It will not help with
> the situation as you will try to commit on every poll()
> 2. I couldn't know how you actually write your code, but  you could try
> something really simple as print statement within the join operation to see
> if your application is actually taking incoming traffic. If you have
> metrics exported, that would also be useful.
>
> Boyang
>
> On Fri, Oct 4, 2019 at 6:42 AM Trey Hutcheson 
> wrote:
>
> > This is my third kafka streams application and I'd thought I had gotten
> to
> > know the warts and how to use it correctly. But I'm beating my head
> against
> > something that I just cannot explain. Values written to a table, when
> later
> > read back in a join operation, are stale.
> >
> > Assume the following simplified domain:
> > ModificationEvent - describes some mutation in the system
> > DetailRecord - a detailed record of some occurrence, contains some
> > metadata, and all of the modification events that occurred as part of the
> > occurrence/incident.
> >
> > Very simple topology:
> > KStream
> > KTable
> > The UUID, in this case, is the id of the incident/occurrence being
> > reported. When there is a "terminal" event, the DetailRecord will be
> > shipped to an external system.
> >
> > So I have a very very simple transformation (pseudo kotlin):
> > eventsStream.join(detailTable) { event, detailRecord ->
> >   detailRecord.events.add(event)
> >   return detailRecord
> > }.to(detailTable)
> >
> > So the ModificationEvent's stream is joined to the DetailRecord's table,
> > and the ModificationEvent is appended to the end of the DetailRecord,
> which
> > is then written back to the table.
> >
> > However, on the next modification event for the same detail record, the
> > detail record is stale and it's list of events is empty. Whats going on
> > here?
> >
> > I've tried temporarily disabling record caching (didn't think that was
> the
> > issue), and even setting the offset commit interval to 0 ms (again didn't
> > think this was the issue). Neither had an effect, other than slowing the
> > stream.
> >
> > Definitely need some help here.
> > Trey
> >
>


Re: Kafka Streams and broker compatibility

2019-08-26 Thread Sophie Blee-Goldman
I'm pretty sure one of the Suppress bug fixes that went into 2.2.1 involved
adding headers. Updating the compatibility matrix must have just slipped
when that bugfix was merged -- thanks for bringing this up!

On Mon, Aug 26, 2019 at 5:37 PM Alisson Sales 
wrote:

> Hi Guozhang, thanks for your reply.
>
> I suspect the "problem" has to do with the fixes released on 2.2.1. I'm
> upgrading to this version mostly because we were facing problems with
> KTable suppress.
>
> I was experiencing this exact same problem:
>
> https://stackoverflow.com/questions/54145281/why-do-the-offsets-of-the-consumer-group-app-id-of-my-kafka-streams-applicatio/54227156
> This was the fix: https://issues.apache.org/jira/browse/KAFKA-7895.
>
> When trying to confirm the fix worked for my topology/app I encountered the
> issue: java.lang.IllegalArgumentException: Magic v1 does not support
> record.
>
> In summary the topology works fine on 0.10.2.1 with kafka-streams 2.2.0,
> but fails with the error above if I use 2.2.1.
>
> I haven't changed any part of the code, simply updated my gradle file
> updating the dependency.
>
> Thanks again
>
> On Tue, Aug 27, 2019 at 12:24 PM Guozhang Wang  wrote:
>
> > Hello Alisson,
> >
> > The root cause you've seen is the message header support, which is added
> in
> > brokers as in 0.11.0 (KIP-82) and in streams client as in 2.0 (KIP-244).
> If
> > your code does not add any more headers then it would only inherit the
> > headers from source topics when trying to write to intermediate / sink
> > topics. So I think that even if you were using 2.2.0 you'd still hit this
> > issue if you happen to have headers in some of your source topic
> messages.
> >
> > I've updated
> > https://cwiki.apache.org/confluence/display/KAFKA/Compatibility+Matrix
> per
> > the updates.
> >
> >
> > Guozhang
> >
> > On Thu, Aug 22, 2019 at 4:12 PM Alisson Sales 
> > wrote:
> >
> > > Hi all, I've just upgraded a project that was using kafka-streams 2.2.0
> > to
> > > 2.2.1 and found the following error at the end.
> > >
> > > I was using Kafka Broker 0.10.2.1 and after upgrading the Broker to
> 0.11
> > > the error doesn't happen anymore.
> > >
> > > My question here is: where is the best place we can find the required
> > > minimum broker version for the kafka-streams version one is using?
> > >
> > > This is not clear to me and the
> > > https://cwiki.apache.org/confluence/display/KAFKA/Compatibility+Matrix
> > > wiki
> > > page seems outdated.
> > >
> > > Thanks in advance
> > >
> > > Exception in thread
> > >
> "streams-batch-opens-f5d8fdb1-db8b-415b-b812-ba448fac3dfa-StreamThread-1"
> > > org.apache.kafka.streams.errors.ProcessorStateException: task [1_2]
> > Failed
> > > to flush state store KTABLE-SUPPRESS-STATE-STORE-09
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:251)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.AbstractTask.flushState(AbstractTask.java:204)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamTask.flushState(StreamTask.java:521)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:473)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:461)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.AssignedTasks.commit(AssignedTasks.java:286)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.TaskManager.commitAll(TaskManager.java:412)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:1056)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:910)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:804)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:773)
> > > Caused by: org.apache.kafka.streams.errors.StreamsException: task [1_2]
> > > Abort sending since an error caught with a previous record (key
> > > A:2019-03-10T07:00:00Z\x00\x00\x01l\xBB\x8FZ\xB0 value [B@c28e8db
> > > timestamp
> > > null) to topic
> > > streams-batch-opens-KTABLE-SUPPRESS-STATE-STORE-09-changelog
> due
> > to
> > > java.lang.IllegalArgumentException: Magic v1 does not support record
> > > headers
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.send(RecordCollectorImpl.java:244)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.logValue(InMemoryTimeOrderedKeyValueBuffer.java:284)
> > > at
> > >
> > >
> >
> org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.flush(InMemoryTimeOrderedKeyValueBuffer.java:266)
> > > at
> > >
> > >
> >
> 

Re: Kafka Streams - unbounded memory growth

2019-07-19 Thread Sophie Blee-Goldman
I believe if you name your stores, the directory name will reflect that. If
you see any directories in your state.dir corresponding to the store names,
that's rocksdb. As the name suggest, in memory stores are held (completely)
in memory

On Fri, Jul 19, 2019 at 3:21 AM Muhammed Ashik  wrote:

> Hi If I’m not wrong I remember seeing the streams code ., that default
> rocksdb state.dir is rocksdb itself. Any content would go under
> /tmp/Kafka-streams/rocksdb
>
> On Fri, 19 Jul 2019 at 1:55 AM, Sophie Blee-Goldman 
> wrote:
>
> > And all four stores (BucketData, CacheData, StationKeyValue,
> > StationCacheData) are definitely explicitly added as
> > Stores.inMemoryKeyValueStore("name")? Hm.
> >
> > Does du -h show any contents within the
> > ./buzzard.MonitoringSeWlanStatsAggregator/0_1 directories that match any
> of
> > these store names? Also if you rerun the app with a different state.dir,
> do
> > all the same directories get created?
> >
> > On Thu, Jul 18, 2019 at 12:15 AM Muhammed Ashik 
> wrote:
> >
> > > Hi, my Topology description..
> > >
> > > - Topologies:
> > >Sub-topology: 0
> > > Source: SeSource (topics: [buzzard.decoded.amon.stats.ap,
> > > buzzard.app.monitoring.internal.clientsestats])
> > >   --> SeProcess
> > > Processor: SeProcess (stores: [BucketData, CacheData])
> > >   --> AeSink
> > >   <-- SeSource
> > > Sink: AeSink (topic: buzzard.app.monitoring.internal.stats)
> > >   <-- SeProcess
> > >
> > >   Sub-topology: 1
> > > Source: StationSource (topics:
> [buzzard.decoded.amon.stats.station])
> > >   --> StationProcess
> > > Processor: StationProcess (stores: [StationKeyValue,
> > StationCacheData])
> > >   --> StationSink
> > >   <-- StationSource
> > > Sink: StationSink (topic:
> > > buzzard.app.monitoring.internal.clientsestats)
> > >   <-- StationProcess
> > >
> > > Regards
> > > Ashik
> > >
> > > On Thu, Jul 18, 2019 at 1:31 AM Sophie Blee-Goldman <
> sop...@confluent.io
> > >
> > > wrote:
> > >
> > > > Hm. These directories shouldn't be created if using only an in memory
> > > > store. Can you print your topology?
> > > >
> > > > On Wed, Jul 17, 2019 at 11:02 AM Muhammed Ashik 
> > > wrote:
> > > >
> > > > > Hi I just did `du -mh` on `\tmp\kafka-streams` below are the
> folders
> > > > listed
> > > > > with some .lock files inside.
> > > > > not sure if these are coming from rocksdb.. and looks like the
> sizes
> > of
> > > > > these files are less.
> > > > >
> > > > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_5
> > > > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_3
> > > > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_2
> > > > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/1_4
> > > > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_1
> > > > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_4
> > > > > 28K ./buzzard.MonitoringSeWlanStatsAggregator
> > > > > 32K .
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jul 17, 2019 at 11:18 PM Sophie Blee-Goldman <
> > > > sop...@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Hm. Just to be absolutely sure, could you try throwing an
> exception
> > > or
> > > > > > something in your RocksDBConfigSetter?
> > > > > >
> > > > > > On Wed, Jul 17, 2019 at 10:43 AM Muhammed Ashik <
> ashi...@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > I can confirm the /tmp/kafka-streams doesn't have any data
> > related
> > > to
> > > > > > > rocksdb.
> > > > > > >
> > > > > > > On Wed, Jul 17, 2019 at 11:11 PM Sophie Blee-Goldman <
> > > > > > sop...@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > You can describe your topology to see if there are any state
> > > stores
> > > > > in
> > > > > > it
> 

Re: Kafka Streams - unbounded memory growth

2019-07-18 Thread Sophie Blee-Goldman
And all four stores (BucketData, CacheData, StationKeyValue,
StationCacheData) are definitely explicitly added as
Stores.inMemoryKeyValueStore("name")? Hm.

Does du -h show any contents within the
./buzzard.MonitoringSeWlanStatsAggregator/0_1 directories that match any of
these store names? Also if you rerun the app with a different state.dir, do
all the same directories get created?

On Thu, Jul 18, 2019 at 12:15 AM Muhammed Ashik  wrote:

> Hi, my Topology description..
>
> - Topologies:
>Sub-topology: 0
> Source: SeSource (topics: [buzzard.decoded.amon.stats.ap,
> buzzard.app.monitoring.internal.clientsestats])
>   --> SeProcess
> Processor: SeProcess (stores: [BucketData, CacheData])
>   --> AeSink
>   <-- SeSource
> Sink: AeSink (topic: buzzard.app.monitoring.internal.stats)
>   <-- SeProcess
>
>   Sub-topology: 1
> Source: StationSource (topics: [buzzard.decoded.amon.stats.station])
>   --> StationProcess
> Processor: StationProcess (stores: [StationKeyValue, StationCacheData])
>   --> StationSink
>   <-- StationSource
> Sink: StationSink (topic:
> buzzard.app.monitoring.internal.clientsestats)
>   <-- StationProcess
>
> Regards
> Ashik
>
> On Thu, Jul 18, 2019 at 1:31 AM Sophie Blee-Goldman 
> wrote:
>
> > Hm. These directories shouldn't be created if using only an in memory
> > store. Can you print your topology?
> >
> > On Wed, Jul 17, 2019 at 11:02 AM Muhammed Ashik 
> wrote:
> >
> > > Hi I just did `du -mh` on `\tmp\kafka-streams` below are the folders
> > listed
> > > with some .lock files inside.
> > > not sure if these are coming from rocksdb.. and looks like the sizes of
> > > these files are less.
> > >
> > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_5
> > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_3
> > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_2
> > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/1_4
> > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_1
> > > 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_4
> > > 28K ./buzzard.MonitoringSeWlanStatsAggregator
> > > 32K .
> > >
> > >
> > >
> > > On Wed, Jul 17, 2019 at 11:18 PM Sophie Blee-Goldman <
> > sop...@confluent.io>
> > > wrote:
> > >
> > > > Hm. Just to be absolutely sure, could you try throwing an exception
> or
> > > > something in your RocksDBConfigSetter?
> > > >
> > > > On Wed, Jul 17, 2019 at 10:43 AM Muhammed Ashik 
> > > wrote:
> > > >
> > > > > I can confirm the /tmp/kafka-streams doesn't have any data related
> to
> > > > > rocksdb.
> > > > >
> > > > > On Wed, Jul 17, 2019 at 11:11 PM Sophie Blee-Goldman <
> > > > sop...@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > You can describe your topology to see if there are any state
> stores
> > > in
> > > > it
> > > > > > that you aren't aware of. Alternatively you could check out the
> > state
> > > > > > directory (/tmp/kafka-streams by default) and see if there is any
> > > data
> > > > in
> > > > > > there
> > > > > >
> > > > > > On Wed, Jul 17, 2019 at 10:36 AM Muhammed Ashik <
> ashi...@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks How can I verify If there is some data really going on
> > > rocksdb
> > > > > > > I tried printing the statistics with no success.
> > > > > > >
> > > > > > > class CustomRocksDBConfig extends RocksDBConfigSetter {
> > > > > > >   override def setConfig(storeName: String, options: Options,
> > > > configs:
> > > > > > > util.Map[String, AnyRef]): Unit = {
> > > > > > >
> > > > > > > val stats = new Statistics
> > > > > > > stats.setStatsLevel(StatsLevel.ALL)
> > > > > > > options.setStatistics(stats)
> > > > > > >   .setStatsDumpPeriodSec(600)
> > > > > > > options
> > > > > > >   .setInfoLogLevel(InfoLogLevel.INFO_LEVEL)
> > > > > > > options.setDbLogDir("/tmp/dump")
> > > > > > >
> > > > > > >   }
>

Re: Kafka Streams - unbounded memory growth

2019-07-17 Thread Sophie Blee-Goldman
Hm. These directories shouldn't be created if using only an in memory
store. Can you print your topology?

On Wed, Jul 17, 2019 at 11:02 AM Muhammed Ashik  wrote:

> Hi I just did `du -mh` on `\tmp\kafka-streams` below are the folders listed
> with some .lock files inside.
> not sure if these are coming from rocksdb.. and looks like the sizes of
> these files are less.
>
> 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_5
> 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_3
> 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_2
> 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/1_4
> 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_1
> 4.0K ./buzzard.MonitoringSeWlanStatsAggregator/0_4
> 28K ./buzzard.MonitoringSeWlanStatsAggregator
> 32K .
>
>
>
> On Wed, Jul 17, 2019 at 11:18 PM Sophie Blee-Goldman 
> wrote:
>
> > Hm. Just to be absolutely sure, could you try throwing an exception or
> > something in your RocksDBConfigSetter?
> >
> > On Wed, Jul 17, 2019 at 10:43 AM Muhammed Ashik 
> wrote:
> >
> > > I can confirm the /tmp/kafka-streams doesn't have any data related to
> > > rocksdb.
> > >
> > > On Wed, Jul 17, 2019 at 11:11 PM Sophie Blee-Goldman <
> > sop...@confluent.io>
> > > wrote:
> > >
> > > > You can describe your topology to see if there are any state stores
> in
> > it
> > > > that you aren't aware of. Alternatively you could check out the state
> > > > directory (/tmp/kafka-streams by default) and see if there is any
> data
> > in
> > > > there
> > > >
> > > > On Wed, Jul 17, 2019 at 10:36 AM Muhammed Ashik 
> > > wrote:
> > > >
> > > > > Thanks How can I verify If there is some data really going on
> rocksdb
> > > > > I tried printing the statistics with no success.
> > > > >
> > > > > class CustomRocksDBConfig extends RocksDBConfigSetter {
> > > > >   override def setConfig(storeName: String, options: Options,
> > configs:
> > > > > util.Map[String, AnyRef]): Unit = {
> > > > >
> > > > > val stats = new Statistics
> > > > > stats.setStatsLevel(StatsLevel.ALL)
> > > > > options.setStatistics(stats)
> > > > >   .setStatsDumpPeriodSec(600)
> > > > > options
> > > > >   .setInfoLogLevel(InfoLogLevel.INFO_LEVEL)
> > > > > options.setDbLogDir("/tmp/dump")
> > > > >
> > > > >   }
> > > > > }
> > > > >
> > > > >
> > > > > and included in the stream config ..
> > > > >
> > > > > settings.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG,
> > > > > classOf[CustomRocksDBConfig])
> > > > >
> > > > >
> > > > > Regards
> > > > > Ashik
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jul 17, 2019 at 10:52 PM Sophie Blee-Goldman <
> > > > sop...@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Sorry, didn't see the "off-heap" part of the email. Are you using
> > any
> > > > > > stateful DSL operators? The default stores are persistent, so you
> > may
> > > > > have
> > > > > > a RocksDB store in your topology without explicitly using one.
> > > > > >
> > > > > > On Wed, Jul 17, 2019 at 10:12 AM Sophie Blee-Goldman <
> > > > > sop...@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > If you are using inMemoryKeyValueStore, the records are stored
> by
> > > > > > > definition in memory. RocksDB is not used at all. This store
> will
> > > > > > continue
> > > > > > > to grow proportionally to your keyspace. If you do not have
> > > > sufficient
> > > > > > > memory to hold your entire dataset in memory, consider adding
> > > another
> > > > > > > instance or switching to the RocksDB store
> > > > > > >
> > > > > > > On Wed, Jul 17, 2019 at 6:22 AM Muhammed Ashik <
> > ashi...@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > >> Kafka Streams version - 2.0.0
> > > > > > >>
> > > > > > >> Hi, in our streaming instance we are observing a steady growth
> > in
> > > > the
> > > > > > >> off-heap memory (out of 2gb allocated memory 1.3 is reserved
> for
> > > > heap
> > > > > > >> memory and the ~700mb memory is utilised over a time of ~6hrs
> > and
> > > > the
> > > > > > >> process is OOM killed eventually).
> > > > > > >>
> > > > > > >> we are using only the inMemoryKeyValueStore and not doing any
> > > > > > persistence.
> > > > > > >> as suggested the iterators are closed at the places it is
> > > used(using
> > > > > it
> > > > > > in
> > > > > > >> only once).
> > > > > > >>
> > > > > > >> Some forums were relating such issues with rocksdb but we are
> > not
> > > > > > >> specifying rocksdb in the config explicitly though. I was not
> > sure
> > > > > > whether
> > > > > > >> it is used as a default in memory store by kafka streams.
> > > > > > >>
> > > > > > >> Regards
> > > > > > >> Ashik
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: Kafka Streams - unbounded memory growth

2019-07-17 Thread Sophie Blee-Goldman
Hm. Just to be absolutely sure, could you try throwing an exception or
something in your RocksDBConfigSetter?

On Wed, Jul 17, 2019 at 10:43 AM Muhammed Ashik  wrote:

> I can confirm the /tmp/kafka-streams doesn't have any data related to
> rocksdb.
>
> On Wed, Jul 17, 2019 at 11:11 PM Sophie Blee-Goldman 
> wrote:
>
> > You can describe your topology to see if there are any state stores in it
> > that you aren't aware of. Alternatively you could check out the state
> > directory (/tmp/kafka-streams by default) and see if there is any data in
> > there
> >
> > On Wed, Jul 17, 2019 at 10:36 AM Muhammed Ashik 
> wrote:
> >
> > > Thanks How can I verify If there is some data really going on rocksdb
> > > I tried printing the statistics with no success.
> > >
> > > class CustomRocksDBConfig extends RocksDBConfigSetter {
> > >   override def setConfig(storeName: String, options: Options, configs:
> > > util.Map[String, AnyRef]): Unit = {
> > >
> > > val stats = new Statistics
> > > stats.setStatsLevel(StatsLevel.ALL)
> > > options.setStatistics(stats)
> > >   .setStatsDumpPeriodSec(600)
> > > options
> > >   .setInfoLogLevel(InfoLogLevel.INFO_LEVEL)
> > > options.setDbLogDir("/tmp/dump")
> > >
> > >   }
> > > }
> > >
> > >
> > > and included in the stream config ..
> > >
> > > settings.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG,
> > > classOf[CustomRocksDBConfig])
> > >
> > >
> > > Regards
> > > Ashik
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > On Wed, Jul 17, 2019 at 10:52 PM Sophie Blee-Goldman <
> > sop...@confluent.io>
> > > wrote:
> > >
> > > > Sorry, didn't see the "off-heap" part of the email. Are you using any
> > > > stateful DSL operators? The default stores are persistent, so you may
> > > have
> > > > a RocksDB store in your topology without explicitly using one.
> > > >
> > > > On Wed, Jul 17, 2019 at 10:12 AM Sophie Blee-Goldman <
> > > sop...@confluent.io>
> > > > wrote:
> > > >
> > > > > If you are using inMemoryKeyValueStore, the records are stored by
> > > > > definition in memory. RocksDB is not used at all. This store will
> > > > continue
> > > > > to grow proportionally to your keyspace. If you do not have
> > sufficient
> > > > > memory to hold your entire dataset in memory, consider adding
> another
> > > > > instance or switching to the RocksDB store
> > > > >
> > > > > On Wed, Jul 17, 2019 at 6:22 AM Muhammed Ashik 
> > > > wrote:
> > > > >
> > > > >> Kafka Streams version - 2.0.0
> > > > >>
> > > > >> Hi, in our streaming instance we are observing a steady growth in
> > the
> > > > >> off-heap memory (out of 2gb allocated memory 1.3 is reserved for
> > heap
> > > > >> memory and the ~700mb memory is utilised over a time of ~6hrs and
> > the
> > > > >> process is OOM killed eventually).
> > > > >>
> > > > >> we are using only the inMemoryKeyValueStore and not doing any
> > > > persistence.
> > > > >> as suggested the iterators are closed at the places it is
> used(using
> > > it
> > > > in
> > > > >> only once).
> > > > >>
> > > > >> Some forums were relating such issues with rocksdb but we are not
> > > > >> specifying rocksdb in the config explicitly though. I was not sure
> > > > whether
> > > > >> it is used as a default in memory store by kafka streams.
> > > > >>
> > > > >> Regards
> > > > >> Ashik
> > > > >>
> > > > >
> > > >
> > >
> >
>


Re: Kafka Streams - unbounded memory growth

2019-07-17 Thread Sophie Blee-Goldman
You can describe your topology to see if there are any state stores in it
that you aren't aware of. Alternatively you could check out the state
directory (/tmp/kafka-streams by default) and see if there is any data in
there

On Wed, Jul 17, 2019 at 10:36 AM Muhammed Ashik  wrote:

> Thanks How can I verify If there is some data really going on rocksdb
> I tried printing the statistics with no success.
>
> class CustomRocksDBConfig extends RocksDBConfigSetter {
>   override def setConfig(storeName: String, options: Options, configs:
> util.Map[String, AnyRef]): Unit = {
>
> val stats = new Statistics
> stats.setStatsLevel(StatsLevel.ALL)
> options.setStatistics(stats)
>   .setStatsDumpPeriodSec(600)
> options
>   .setInfoLogLevel(InfoLogLevel.INFO_LEVEL)
> options.setDbLogDir("/tmp/dump")
>
>   }
> }
>
>
> and included in the stream config ..
>
> settings.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG,
> classOf[CustomRocksDBConfig])
>
>
> Regards
> Ashik
>
>
>
>
>
>
>
>
>
> On Wed, Jul 17, 2019 at 10:52 PM Sophie Blee-Goldman 
> wrote:
>
> > Sorry, didn't see the "off-heap" part of the email. Are you using any
> > stateful DSL operators? The default stores are persistent, so you may
> have
> > a RocksDB store in your topology without explicitly using one.
> >
> > On Wed, Jul 17, 2019 at 10:12 AM Sophie Blee-Goldman <
> sop...@confluent.io>
> > wrote:
> >
> > > If you are using inMemoryKeyValueStore, the records are stored by
> > > definition in memory. RocksDB is not used at all. This store will
> > continue
> > > to grow proportionally to your keyspace. If you do not have sufficient
> > > memory to hold your entire dataset in memory, consider adding another
> > > instance or switching to the RocksDB store
> > >
> > > On Wed, Jul 17, 2019 at 6:22 AM Muhammed Ashik 
> > wrote:
> > >
> > >> Kafka Streams version - 2.0.0
> > >>
> > >> Hi, in our streaming instance we are observing a steady growth in the
> > >> off-heap memory (out of 2gb allocated memory 1.3 is reserved for heap
> > >> memory and the ~700mb memory is utilised over a time of ~6hrs and the
> > >> process is OOM killed eventually).
> > >>
> > >> we are using only the inMemoryKeyValueStore and not doing any
> > persistence.
> > >> as suggested the iterators are closed at the places it is used(using
> it
> > in
> > >> only once).
> > >>
> > >> Some forums were relating such issues with rocksdb but we are not
> > >> specifying rocksdb in the config explicitly though. I was not sure
> > whether
> > >> it is used as a default in memory store by kafka streams.
> > >>
> > >> Regards
> > >> Ashik
> > >>
> > >
> >
>


Re: Kafka Streams - unbounded memory growth

2019-07-17 Thread Sophie Blee-Goldman
Sorry, didn't see the "off-heap" part of the email. Are you using any
stateful DSL operators? The default stores are persistent, so you may have
a RocksDB store in your topology without explicitly using one.

On Wed, Jul 17, 2019 at 10:12 AM Sophie Blee-Goldman 
wrote:

> If you are using inMemoryKeyValueStore, the records are stored by
> definition in memory. RocksDB is not used at all. This store will continue
> to grow proportionally to your keyspace. If you do not have sufficient
> memory to hold your entire dataset in memory, consider adding another
> instance or switching to the RocksDB store
>
> On Wed, Jul 17, 2019 at 6:22 AM Muhammed Ashik  wrote:
>
>> Kafka Streams version - 2.0.0
>>
>> Hi, in our streaming instance we are observing a steady growth in the
>> off-heap memory (out of 2gb allocated memory 1.3 is reserved for heap
>> memory and the ~700mb memory is utilised over a time of ~6hrs and the
>> process is OOM killed eventually).
>>
>> we are using only the inMemoryKeyValueStore and not doing any persistence.
>> as suggested the iterators are closed at the places it is used(using it in
>> only once).
>>
>> Some forums were relating such issues with rocksdb but we are not
>> specifying rocksdb in the config explicitly though. I was not sure whether
>> it is used as a default in memory store by kafka streams.
>>
>> Regards
>> Ashik
>>
>


Re: Kafka Streams - unbounded memory growth

2019-07-17 Thread Sophie Blee-Goldman
If you are using inMemoryKeyValueStore, the records are stored by
definition in memory. RocksDB is not used at all. This store will continue
to grow proportionally to your keyspace. If you do not have sufficient
memory to hold your entire dataset in memory, consider adding another
instance or switching to the RocksDB store

On Wed, Jul 17, 2019 at 6:22 AM Muhammed Ashik  wrote:

> Kafka Streams version - 2.0.0
>
> Hi, in our streaming instance we are observing a steady growth in the
> off-heap memory (out of 2gb allocated memory 1.3 is reserved for heap
> memory and the ~700mb memory is utilised over a time of ~6hrs and the
> process is OOM killed eventually).
>
> we are using only the inMemoryKeyValueStore and not doing any persistence.
> as suggested the iterators are closed at the places it is used(using it in
> only once).
>
> Some forums were relating such issues with rocksdb but we are not
> specifying rocksdb in the config explicitly though. I was not sure whether
> it is used as a default in memory store by kafka streams.
>
> Regards
> Ashik
>


Re: Kafka Streams - unbounded memory growth - stateful processing (rocksdb)

2019-07-16 Thread Sophie Blee-Goldman
Hi Ashok,

1) RocksDB uses memory in four ways, one of which (iterators) *should* be
negligible -- however if you have a very large number of them open at any
one time, they can consume a lot of memory (until they are closed). If you
are opening many iterators throughout the day, consider closing them more
frequently than once a day.

2) The other three users of memory are: index & filter blocks, block cache,
and memtable (write buffer). You can limit the memory usage of index/filter
blocks by setting tableConfig.setCacheIndexAndFilterBlocks(true) -- this
will cause these blocks to be stored in the block cache alongside data
blocks (and be evicted to disk when full). If you do this I would suggest
also setting tableConfig.setPinL0FilterAndIndexBlocksInCache(true). You can
then control the off-heap memory usage by setting the block cache size, and
write buffer size + write buffer number.

3) By disabling the RocksDBConfigSetter, you aren't disabling the RocksDB
cache (or other users of memory), you are reverting to the defaults set by
Streams (not sure if that's what you expect or not) -- Rocks will continue
to use off-heap memory

4) RocksDB objects are backed by C++ objects, so you need to actually close
some objects you construct to free the memory. Since you construct a
BloomFilter in your config setter but never close it, you are leaking
memory. Unfortunately we did not have a RocksDBConfigSetter#close method in
1.0.0, but there is one as of 2.3 -- you should either remove the
BloomFilter from your config setter or consider upgrading to 2.3 (or 2.2,
which already uses a BloomFilter -- you can just get/update the existing
BlockBasedTableConfig instead of creating a new one to utilize the
BloomFilter)

5) The settings from the config setter (or Streams defaults) are actually
per rocksdb instance, not per Streams instance. If you have a very large
number of stores, you may hit OOM even with the relatively conservative
defaults Streams uses. If you have a large number of stores in your
subtopology, or a large number of partitions all being read by the same
instance, the total off-heap memory will be quite large. (If you're able to
upgrade to 2.3, you can actually use the config setter to limit the total
memory across all stores rather than on a per-store basis)

If you don't have a large number of stores on an instance, don't open a
large number of iterators at a time, and still hit OOM over 100GB even with
the default rocks configs, there may be a memory leak. But I would first
try setting the configs suggested in 2) above, with a smaller write
buffer/block cache size (and no bloom filter)

Cheers,
Sophie

On Tue, Jul 16, 2019 at 12:21 PM Jayaraman, AshokKumar (CCI-Atlanta-CON) <
ashokkumar.jayara...@cox.com> wrote:

> Hi,
>
> In our streaming instance, the internal caching has been disabled and
> RocksDB caching has been enabled, with the override as shown below.
> Although the heap is restricted to 36GB, the memory utilization is going
> over 100GB in a week and eventually runs out of memory.  As part of the
> profiling, we have confirmed that the garbage collection process is within
> the within the limit 36GB (on-heap).  However, the additional memory
> utilization is not appearing within the profiling and is the one we
> suspect, growing unbounded (off-heap).
>
> We have also tried enabling the streams caching (5GB) and disabling the
> RocksDB config setter (commented as below).  However, we are still seeing
> the similar behaviour where the memory is growing unlimited overtime.   We
> process 20 million records each 20 minutes (a message size - 1KB) on an
> average.  Can you please review and advise what could cause this behavior?
> We have ensured that the iterators are closed (which happens once a day).
>
> //streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG,
> RocksDBOverride.class)
>
> Kafka Broker / Kafka Stream version: 1.0.0
> Rocks DB: 5.7.3
>
> Command:
> java -Xms12g -Xmx36g -XX:MetaspaceSize=576m -XX:+UseG1GC
> -XX:ParallelGCThreads=8 -XX:MaxGCPauseMillis=80
> -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
> -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -cp
> /scripts/device_metrics.jar:/libs/kafka/*
> -Dlog4j.configuration=file:/cfg/device_metrics_log4j.properties
> org.ssd.devicemetrics /cfg/device_metrics.properties
>
>
> Rocks DB Config setter
>  BlockBasedTableConfig tableConfig = new
> org.rocksdb.BlockBasedTableConfig();
>  BloomFilter bloomFilter = new BloomFilter();
>  tableConfig.setBlockCacheSize(512MB);
>  tableConfig.setBlockSize(64KB);
>  tableConfig.setCacheIndexAndFilterBlocks(false);
>  tableConfig.setFilter(bloomFilter);
>
>  options.setTableFormatConfig(tableConfig);
>  options.setWriteBufferSize(512MB);
>  options.setMaxWriteBufferNumber(5);
>  options.setCompressionType(CompressionType.LZ4_COMPRESSION);
>

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-11 Thread Sophie Blee-Goldman
I believe the "resource temporarily unavailable" actually is related to the
open files, most likely you are hitting the total file descriptor limit.
Sorry if you mentioned this and I missed it, but what was the
max.open.files in your RocksDBConfigSetter when you ran this? Actually,
could you just include your full RocksDBConfigSetter implementation?

Not sure why Rocks would be spawning so many child processes...it does use
background threads for compaction but this sounds like something totally
different. I also notice a read "This is a RocksDB options file" -- are you
trying to use an options file vs setting the options with the config setter
(or otherwise using Rocksdb outside of Streams?)

 Have you tried cleaning up the state between runs? Are you using
interactive queries?

Regarding the .checkpoint file not found thing -- this is an annoying but
pretty much harmless bug we only recently figured out. There's an open PR
for it but for now you can just ignore the warning. See KAFKA-5998
 for a long history (but
scroll to the bottom for the actual explanation)


On Thu, Jul 4, 2019 at 11:02 AM Thameem Ansari  wrote:

> i have few more details to share from today’s testing.
>
> Attached strace to the process and noticed that there are thousands of
> short lived child processes have been created by the stream application.
> Not sure whether rocksdb is playing any role here. Noticed more than 73000
> child processes were created and this is after i increased the default
> linux system limit from 32767 to 4million per PID. Appreciate if someone
> answer the following questions.
> 1. Why rocksdb creates these many processes? Each process is trying to
> read the contents of the file and getting EAGAIN (Resource temporarily
> unavailable)
> 2. Noticed very high number of .checkpoint files missing and hence “No
> such file or directory” message
> 3. The child processes were created using “clone” system call. Not sure
> whether there is any significance of this but wanna mention. Moreover these
> child processes were keep on increasing when the load is getting applied
> continuously. Not sure under what condition it will hit the ceiling.
> 4. Noticed very high VIRT memory usage (around 31G) but RES memory was
> only 1.9G. Isn’t every file open/close consume memory? But it is not going
> up even though the number of processes is increased.
>
> Thanks
> Thameem
>
> > On Jul 4, 2019, at 9:28 AM, Thameem Ansari  wrote:
> >
> > Sorry for reposting the previous message as the images didn’t come
> thru.. pasting as text.
> >
> > I have changed both system and user limits.
> >
> > To completely isolate the problem, I have tried the application in
> Centos 7 environment. Set the ulimit to 1million and system limits to
> 10million open files. Now 3 kafka nodes are running in separate servers
> each and streaming application is running in a dedicated VM. Now the
> application is not explicitly throwing “too many open files” error but
> automatically aborted with the message
> >
> > terminate called after throwing an instance of 'std::system_error'
> >   what():  Resource temporarily unavailable
> > ./bin/start.sh: line 42:  1100 Aborted
> >
> > Here is the last few lines from strace output which shows the aborted
> message.
> >
> > 25721 14:58:35
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/MANIFEST-07",
> O_RDONLY|O_CLOEXEC) = 12505
> > 25721 14:58:35
> open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", O_RDONLY)
> = 12506
> > 25721 14:58:35 read(12506, "512\n", 4096) = 4
> > 25721 14:58:35 close(12506) = 0
> > 25721 14:58:35 write(12502, "s.advise_random_on_open: 0\n2019/"...,
> 4096) = 4096
> > 25721 14:58:35 write(12502, "ions.comparator: leveldb.Bytewis"..., 4096)
> = 4096
> > 25721 14:58:35 read(12505,
> "V\371\270\370\34\0\1\1\32leveldb.BytewiseCompara"..., 32768) = 192
> > 25721 14:58:35 read(12505, "", 28672)   = 0
> > 25721 14:58:35 close(12505) = 0
> > 17701 14:58:35
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/06.sst",
> O_RDONLY|O_CLOEXEC) = 12505
> > 17702 14:58:35 +++ exited with 0 +++
> > 25721 14:58:35 write(2, "terminate called after throwing "..., 48) = 48
> > 25721 14:58:35 write(2, "std::system_error", 17) = 17
> > 25721 14:58:35 write(2, "'\n", 2)   = 2
> > 25721 14:58:35 write(2, "  what():  ", 11) = 11
> > 25721 14:58:35 write(2, "Resource temporarily unavailable", 32) = 32
> > 25721 14:58:35 write(2, "\n", 1)= 1
> > 

Re: org.apache.kafka.streams.processor.TimestampExtractor#extract method in version 2.3 always returns -1 as value

2019-07-08 Thread Sophie Blee-Goldman
Thanks for the notice Jonathan! We tracked down the problem and it should
be an easy fix: https://github.com/apache/kafka/pull/6719/files

On Fri, Jul 5, 2019 at 6:25 AM Jonathan Santilli 
wrote:

> Thanks a lot Bill for creating the issue, I have updated it with a little
> bit more of info.
>
> Cheers!
> --
> Jonathan
>
>
>
>
> On Fri, Jun 28, 2019 at 9:21 PM Bill Bejeck  wrote:
>
> > Jonathan, Matthias
> >
> > I've created a Jira for this issue
> > https://issues.apache.org/jira/browse/KAFKA-8615.
> >
> > Jonathan, I plan to work on this when I get back from vacation on 7/8.
> If
> > you would like to work in this yourself before that, feel free to do so
> and
> > assign the ticket to yourself.
> >
> > Thanks,
> > Bill
> >
> > On Thu, Jun 27, 2019 at 1:38 PM Matthias J. Sax 
> > wrote:
> >
> > > Sounds like a regression to me.
> > >
> > > We did change some code to track partition time differently. Can you
> > > open a Jira?
> > >
> > >
> > > -Matthias
> > >
> > > On 6/26/19 7:58 AM, Jonathan Santilli wrote:
> > > > Sure Bill, sure, is the same code I have reported the issue for the
> > > > suppress some months ago:
> > > >
> > >
> >
> https://stackoverflow.com/questions/54145281/why-do-the-offsets-of-the-consumer-group-app-id-of-my-kafka-streams-applicatio
> > > >
> > > > In fact, I have reported at that moment, that after restarting the
> app,
> > > the
> > > > suppress was sending again downstream the already processed records.
> > > > Now, with the version 2.2.1+ after restarting the app, the
> > > > aggregation/suppress (do not know exactly where) is missing some
> > records
> > > to
> > > > be aggregated, even though they are in the input topic.
> > > >
> > > > Kafka Version 2.3
> > > >
> > > > *public* *class* OwnTimeExtractor *implements* TimestampExtractor {
> > > >
> > > > @Override
> > > >
> > > > *public* *long* extract(*final* ConsumerRecord
> > > record,
> > > > *final* *long* previousTimestamp) {
> > > >
> > > >
> > > > *// *previousTimestamp is always == -1
> > > >
> > > > }
> > > > }
> > > >
> > > > final StreamsBuilder builder = new StreamsBuilder();
> > > > final KStream<..., ...> events = builder
> > > > .stream(inputTopicNames, Consumed.with(..., ...)
> > > > .withTimestampExtractor(new OwnTimeExtractor());
> > > >
> > > > events
> > > > .filter((k, v) -> ...)
> > > > .flatMapValues(v -> ...)
> > > > .flatMapValues(v -> ...)
> > > > .selectKey((k, v) -> v)
> > > > .groupByKey(Grouped.with(..., ...))
> > > > .windowedBy(
> > > > TimeWindows.of(Duration.ofSeconds(windowSizeInSecs))
> > > > .advanceBy(Duration.ofSeconds(windowSizeInSecs))
> > > > .grace(Duration.ofSeconds(windowSizeGraceInSecs)))
> > > > .reduce((agg, new) -> {
> > > > ...
> > > > return agg;
> > > > })
> > > >
> > >
> >
> .suppress(Suppressed.untilWindowCloses(Suppressed.BufferConfig.unbounded()))
> > > > .toStream()
> > > > .to(outPutTopicNameOfGroupedData, Produced.with(..., ...));
> > > >
> > > >
> > > >
> > > > On Wed, Jun 26, 2019 at 3:40 PM Bill Bejeck 
> wrote:
> > > >
> > > >> Thanks for the reply Jonathan.
> > > >>
> > > >> Are you in a position to share your code so I can try to reproduce
> on
> > my
> > > >> end?
> > > >>
> > > >> -Bill
> > > >>
> > > >>
> > > >> On Wed, Jun 26, 2019 at 10:23 AM Jonathan Santilli <
> > > >> jonathansanti...@gmail.com> wrote:
> > > >>
> > > >>> Hello Bill,
> > > >>>
> > > >>> am implementing the TimestampExtractor Interface, then using it to
> > > >> consume,
> > > >>> like:
> > > >>>
> > > >>> *final* KStream<..., ...> events = builder.stream(inputTopicList,
> > > >> Consumed.
> > > >>> *with*(keySerde, valueSerde).withTimestampExtractor(*new
> > > >> *OwnTimeExtractor(
> > > >>> ...)));
> > > >>>
> > > >>> Am not setting the default.timestamp.extractor config value.
> > > >>>
> > > >>> Cheers!
> > > >>> --
> > > >>> Jonathan
> > > >>>
> > > >>>
> > > >>> On Wed, Jun 26, 2019 at 3:16 PM Bill Bejeck 
> > wrote:
> > > >>>
> > >  Hi Jonathan,
> > > 
> > >  Thanks for reporting this.  Which timestamp extractor are you
> using
> > in
> > > >>> the
> > >  configs?
> > > 
> > >  Thanks,
> > >  Bill
> > > 
> > >  On Wed, Jun 26, 2019 at 9:14 AM Jonathan Santilli <
> > >  jonathansanti...@gmail.com> wrote:
> > > 
> > > > Hello, hope you all are doing well,
> > > >
> > > > am testing the new version 2.3 for Kafka Streams specifically. I
> > have
> > > > noticed that now, the implementation of the method extract from
> the
> > > > interface org.apache.kafka.streams.processor.TimestampExtractor
> > > >
> > > > *public* *long* extract(ConsumerRecord record,
> > *long*
> > > > previousTimestamp)
> > > >
> > > >
> > > > is always returning -1 as value.
> > > >
> > > >
> > > > Previous version 2.2.1 was returning the correct value for the
> > record
> > > > 

Re: Kafka 2.3.0 - inMemoryKeyValueStore changes (KAFKA-7918) - java.lang.NullPointerException

2019-07-08 Thread Sophie Blee-Goldman
Hi Nitay,

InMemoryKeyValueStore is in the internal package, not part of the public
API, so it is not meant to be used directly since it's implementation may
change at any time (as has happened here). It's intended that you use the
store builders/suppliers to get a new state store, but as you noticed this
means you do have to call init yourself. You can just use a
MockProcessorContext to do so (see the unit tests of these stores for an
example)

Best,
Sophie

On Mon, Jul 8, 2019 at 5:14 AM Nitay Kufert  wrote:

> Hey,
> Following https://issues.apache.org/jira/browse/KAFKA-7918 I had to change
> the current implementation of our unit tests.
>
> Before the change, I created a store using:
> new InMemoryKeyValueStore[String, BigDecimal](
> countersStoreName, Serdes.String, SpecificSerdes.bigDecimalSerde
>   )
>
> It seems that post the change, I need to do:
> Stores.keyValueStoreBuilder(
> Stores.inMemoryKeyValueStore(countersStoreName), Serdes.String,
> SpecificSerdes.bigDecimalSerde
>   ).build()
>
> But when I do, and I try to use the "put" command like so:
> countersStore.put("unique_key_1", BigDecimal(9.87))
>
> I get:
>
> An exception or error caused a run to abort.
> java.lang.NullPointerException
> at
>
> org.apache.kafka.streams.state.internals.MeteredKeyValueStore.put(MeteredKeyValueStore.java:160)
>
> When digging a little into the code, it seems that the "init" function is
> not called, which in turn keep putTime un-initialized.
>
> Am I missing something?
>
> --
>
> Nitay Kufert
> Backend Developer
> [image: ironSource] 
>
> email nita...@ironsrc.com
> mobile +972-54-5480021
> fax +972-77-5448273
> skype nitay.kufert.ssa
> 9 Ehad Ha'am st. Tel- Aviv
> ironsrc.com 
> [image: linkedin]  [image:
> twitter]  [image: facebook]
>  [image: googleplus]
> 
> This email (including any attachments) is for the sole use of the intended
> recipient and may contain confidential information which may be protected
> by legal privilege. If you are not the intended recipient, or the employee
> or agent responsible for delivering it to the intended recipient, you are
> hereby notified that any use, dissemination, distribution or copying of
> this communication and/or its content is strictly prohibited. If you are
> not the intended recipient, please immediately notify us by reply email or
> by telephone, delete this email and destroy any copies. Thank you.
>


Re: stream.filter() based on message header

2019-07-08 Thread Sophie Blee-Goldman
Hi Jorg,

The transform/transformValues/process operators are not inherently
stateful. They *can* be stateful if you choose to connect a statestore, but
otherwise they are stateless and in your case, most likely what you want to
use.

Cheers,
Sophie

On Thu, Jul 4, 2019 at 2:51 AM Jorg Heymans  wrote:

> Hi,
>
> I understand that it's currently not possible to access message headers
> from the filter() DSL operation, is there a semantically equivalent
> (stateless) operation that can achieve the same result ? I understand that
> transform(), transformValues() and process() could achieve the same result
> but they are stateful.
>
> Thanks,
> Jorg
>
>
>


Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-03 Thread Sophie Blee-Goldman
Also, when you say you changed the OS limit are you referring to the system
limit or the user limit or both? If you increased one but not the other you
may still be hitting the lower limit.

On Wed, Jul 3, 2019 at 1:53 AM Patrik Kleindl  wrote:

> Hi
> Try to set it really low like Sophie suggested.
> You can verify if the settings take effect by checking the files in the
> rocksdb directories, I think it should be somewhere in OPTIONS or LOG
> br, Patrik
>
> > Am 03.07.2019 um 09:37 schrieb Thameem Ansari :
> >
> > Tried setting the open files to 100 and 50 but the results are same. I
> checked the total open files while the streaming application was busy
> running just before getting the “too many open files” message it was around
> 41756 which is same as what we have got when we set to -1.
> >
> > VisualVM shows that there is no abnormality with the threads / memory or
> heap.
> >
> > Thanks
> > Thameem
> >
> >> On Jul 3, 2019, at 11:50 AM, Sophie Blee-Goldman 
> wrote:
> >>
> >> How sure are you that the open file count never goes beyond 50K? Are
> those
> >> numbers just from a snapshot after it crashed? It's possible rocks is
> >> creating a large number of files just for a short period of time (maybe
> >> while compacting) that causes the open file count to spike and go back
> down.
> >>
> >> For things to try, you should set the rocks config max.open.files to
> >> something less than infinity...if you're OS limit is 1 million and you
> have
> >> (rounding up) ~5k rocks instances, set this to 1 million / 5k = 200. If
> you
> >> set a lower limit and still hit this error, we can go from there
> >>
> >> On Tue, Jul 2, 2019 at 11:10 PM emailtokir...@gmail.com <
> >> emailtokir...@gmail.com> wrote:
> >>
> >>>
> >>>
> >>>> On 2019/07/03 05:46:45, Sophie Blee-Goldman 
> wrote:
> >>>> It sounds like rocksdb *is* honoring your configs -- the
> max.open.files
> >>>> config is an internal restriction that tells rocksdb how many open
> files
> >>> it
> >>>> is allowed to have, so if that's set to -1 (infinite) it won't ever
> try
> >>> to
> >>>> limit its open files and you may hit the OS limit.
> >>>>
> >>>> Think of it this way: if you have 100 rocksdb instances and a OS
> limit of
> >>>> 500, you should set max.open.files to 5  to avoid hitting this limit
> >>>> (assuming there are no other open files on the system, in reality
> you'd
> >>>> want some extra room there)
> >>>>
> >>>> On Tue, Jul 2, 2019 at 7:53 PM emailtokir...@gmail.com <
> >>>> emailtokir...@gmail.com> wrote:
> >>>>
> >>>>>
> >>>>>
> >>>>>> On 2019/06/28 23:29:16, John Roesler  wrote:
> >>>>>> Hey all,
> >>>>>>
> >>>>>> If you want to figure it out theoretically, if you print out the
> >>>>>> topology description, you'll have some number of state stores listed
> >>>>>> in there. The number of Rocks instances should just be
> >>>>>> (#global_state_stores +
> >>>>>> sum(#partitions_of_topic_per_local_state_store)) . The number of
> >>>>>> stream threads isn't relevant here.
> >>>>>>
> >>>>>> You can also figure it out empirically: the first level of
> >>>>>> subdirectories in the state dir are Tasks, and then within that, the
> >>>>>> next level is Stores. You should see the store directory names match
> >>>>>> up with the stores listed in the topology description. The number of
> >>>>>> Store directories is exactly the number of RocksDB instances you
> >>> have.
> >>>>>>
> >>>>>> There are also metrics corresponding to each of the state stores, so
> >>>>>> you can compute it from what you find in the metrics.
> >>>>>>
> >>>>>> Hope that helps,
> >>>>>> -john
> >>>>>>
> >>>>>> On Thu, Jun 27, 2019 at 6:46 AM Patrik Kleindl 
> >>>>> wrote:
> >>>>>>>
> >>>>>>> Hi Kiran
> >>>>>>> Without much research my guess would be "num_stream_threads *
> >>>>>>> (#global_state_stores +
> >>>&g

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-03 Thread Sophie Blee-Goldman
How sure are you that the open file count never goes beyond 50K? Are those
numbers just from a snapshot after it crashed? It's possible rocks is
creating a large number of files just for a short period of time (maybe
while compacting) that causes the open file count to spike and go back down.

For things to try, you should set the rocks config max.open.files to
something less than infinity...if you're OS limit is 1 million and you have
(rounding up) ~5k rocks instances, set this to 1 million / 5k = 200. If you
set a lower limit and still hit this error, we can go from there

On Tue, Jul 2, 2019 at 11:10 PM emailtokir...@gmail.com <
emailtokir...@gmail.com> wrote:

>
>
> On 2019/07/03 05:46:45, Sophie Blee-Goldman  wrote:
> > It sounds like rocksdb *is* honoring your configs -- the max.open.files
> > config is an internal restriction that tells rocksdb how many open files
> it
> > is allowed to have, so if that's set to -1 (infinite) it won't ever try
> to
> > limit its open files and you may hit the OS limit.
> >
> > Think of it this way: if you have 100 rocksdb instances and a OS limit of
> > 500, you should set max.open.files to 5  to avoid hitting this limit
> > (assuming there are no other open files on the system, in reality you'd
> > want some extra room there)
> >
> > On Tue, Jul 2, 2019 at 7:53 PM emailtokir...@gmail.com <
> > emailtokir...@gmail.com> wrote:
> >
> > >
> > >
> > > On 2019/06/28 23:29:16, John Roesler  wrote:
> > > > Hey all,
> > > >
> > > > If you want to figure it out theoretically, if you print out the
> > > > topology description, you'll have some number of state stores listed
> > > > in there. The number of Rocks instances should just be
> > > > (#global_state_stores +
> > > > sum(#partitions_of_topic_per_local_state_store)) . The number of
> > > > stream threads isn't relevant here.
> > > >
> > > > You can also figure it out empirically: the first level of
> > > > subdirectories in the state dir are Tasks, and then within that, the
> > > > next level is Stores. You should see the store directory names match
> > > > up with the stores listed in the topology description. The number of
> > > > Store directories is exactly the number of RocksDB instances you
> have.
> > > >
> > > > There are also metrics corresponding to each of the state stores, so
> > > > you can compute it from what you find in the metrics.
> > > >
> > > > Hope that helps,
> > > > -john
> > > >
> > > > On Thu, Jun 27, 2019 at 6:46 AM Patrik Kleindl 
> > > wrote:
> > > > >
> > > > > Hi Kiran
> > > > > Without much research my guess would be "num_stream_threads *
> > > > > (#global_state_stores +
> > > sum(#partitions_of_topic_per_local_state_store))"
> > > > > So 10 stores (regardless if explicitly defined or implicitely
> because
> > > of
> > > > > some stateful operation) with 10 partitions each should result in
> 100
> > > > > Rocksdb instances if you are running at the default of
> > > num_stream_threads=1.
> > > > >
> > > > > As I wrote before, start with 100.
> > > > > If the error persists, half the number, if not, double it ;-)
> Repeat as
> > > > > needed.
> > > > >
> > > > > If you reach the single-digit-range and the error still shows up,
> start
> > > > > searching for any iterators over a store you might not have closed.
> > > > >
> > > > > br, Patrik
> > > > >
> > > > > On Thu, 27 Jun 2019 at 13:11, emailtokir...@gmail.com <
> > > > > emailtokir...@gmail.com> wrote:
> > > > >
> > > > > >
> > > > > >
> > > > > > On 2019/06/27 09:02:39, Patrik Kleindl 
> wrote:
> > > > > > > Hello Kiran
> > > > > > >
> > > > > > > First, the value for maxOpenFiles is per RocksDB instance, and
> the
> > > number
> > > > > > > of those can get high if you have a lot of topic partitions
> etc.
> > > > > > > Check the directory (state dir) to see how many there are.
> > > > > > > Start with a low value (100) and see if that has some effect.
> > > > > > >
> > > > > > > Second, because I just found out, you should use

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-02 Thread Sophie Blee-Goldman
It sounds like rocksdb *is* honoring your configs -- the max.open.files
config is an internal restriction that tells rocksdb how many open files it
is allowed to have, so if that's set to -1 (infinite) it won't ever try to
limit its open files and you may hit the OS limit.

Think of it this way: if you have 100 rocksdb instances and a OS limit of
500, you should set max.open.files to 5  to avoid hitting this limit
(assuming there are no other open files on the system, in reality you'd
want some extra room there)

On Tue, Jul 2, 2019 at 7:53 PM emailtokir...@gmail.com <
emailtokir...@gmail.com> wrote:

>
>
> On 2019/06/28 23:29:16, John Roesler  wrote:
> > Hey all,
> >
> > If you want to figure it out theoretically, if you print out the
> > topology description, you'll have some number of state stores listed
> > in there. The number of Rocks instances should just be
> > (#global_state_stores +
> > sum(#partitions_of_topic_per_local_state_store)) . The number of
> > stream threads isn't relevant here.
> >
> > You can also figure it out empirically: the first level of
> > subdirectories in the state dir are Tasks, and then within that, the
> > next level is Stores. You should see the store directory names match
> > up with the stores listed in the topology description. The number of
> > Store directories is exactly the number of RocksDB instances you have.
> >
> > There are also metrics corresponding to each of the state stores, so
> > you can compute it from what you find in the metrics.
> >
> > Hope that helps,
> > -john
> >
> > On Thu, Jun 27, 2019 at 6:46 AM Patrik Kleindl 
> wrote:
> > >
> > > Hi Kiran
> > > Without much research my guess would be "num_stream_threads *
> > > (#global_state_stores +
> sum(#partitions_of_topic_per_local_state_store))"
> > > So 10 stores (regardless if explicitly defined or implicitely because
> of
> > > some stateful operation) with 10 partitions each should result in 100
> > > Rocksdb instances if you are running at the default of
> num_stream_threads=1.
> > >
> > > As I wrote before, start with 100.
> > > If the error persists, half the number, if not, double it ;-) Repeat as
> > > needed.
> > >
> > > If you reach the single-digit-range and the error still shows up, start
> > > searching for any iterators over a store you might not have closed.
> > >
> > > br, Patrik
> > >
> > > On Thu, 27 Jun 2019 at 13:11, emailtokir...@gmail.com <
> > > emailtokir...@gmail.com> wrote:
> > >
> > > >
> > > >
> > > > On 2019/06/27 09:02:39, Patrik Kleindl  wrote:
> > > > > Hello Kiran
> > > > >
> > > > > First, the value for maxOpenFiles is per RocksDB instance, and the
> number
> > > > > of those can get high if you have a lot of topic partitions etc.
> > > > > Check the directory (state dir) to see how many there are.
> > > > > Start with a low value (100) and see if that has some effect.
> > > > >
> > > > > Second, because I just found out, you should use
> > > > > BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
> > > > > options.tableFormatConfig();
> > > > > tableConfig.setBlockCacheSize(100*1024*1024L);
> > > > > tableConfig.setBlockSize(8*1024L);
> > > > > instead of creating a new object to prevent accidently messing up
> > > > > references.
> > > > >
> > > > > Hope that helps
> > > > > best regards
> > > > > Patrik
> > > > >
> > > > > On Thu, 27 Jun 2019 at 10:46, emailtokir...@gmail.com <
> > > > > emailtokir...@gmail.com> wrote:
> > > > >
> > > > > >
> > > > > >
> > > > > > On 2019/06/26 21:58:02, Patrik Kleindl 
> wrote:
> > > > > > > Hi Kiran
> > > > > > > You can use the RocksDBConfigSetter and pass
> > > > > > >
> > > > > > > options.setMaxOpenFiles(100);
> > > > > > >
> > > > > > > to all RocksDBs for the Streams application which limits how
> many are
> > > > > > > kept open at the same time.
> > > > > > >
> > > > > > > best regards
> > > > > > >
> > > > > > > Patrik
> > > > > > >
> > > > > > >
> > > > > > > On Wed, 26 Jun 2019 at 16:14, emailtokir...@gmail.com <
> > > > > > > emailtokir...@gmail.com> wrote:
> > > > > > >
> > > > > > > > Hi,
> > > > > > > >
> > > > > > > > We are using Kafka streams DSL APIs for doing some counter
> > > > aggregations
> > > > > > > > (running on OpenJDK 11.0.2). Our topology has some 400 sub
> > > > topologies
> > > > > > & we
> > > > > > > > are using 8 partitions in source topic. When we start
> pumping more
> > > > > > load, we
> > > > > > > > start getting RockDBException stating "too many open files".
> > > > > > > >
> > > > > > > > Here are the stack trace samples:
> > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> --
> > > > > > > > Caused by: org.rocksdb.RocksDBException: while open a file
> for
> > > > lock:
> > > > > > > > PPP.151200/LOCK: Too many open files
> > > > > > > > at org.rocksdb.RocksDB.open(Native Method)
> > > > > > > > at org.rocksdb.RocksDB.open(RocksDB.java:235)
> > 

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-02 Thread Sophie Blee-Goldman
This can also happen if you have any open iterators that you forget to
close (for example using IQ), although that's probably not what's going on
here since 3152 is certainly a lot of rocks instances for a single fs.

There's no default number of open files per instance, since rocks creates
new files on new levels as you add more data. To understand the impact you
should check out the description of *max_open_files* here

--
but you will probably want to increase your system limit in addition to
constraining the number of open files per instance

On Mon, Jul 1, 2019 at 9:58 AM emailtokir...@gmail.com <
emailtokir...@gmail.com> wrote:

>
>
> On 2019/06/28 23:29:16, John Roesler  wrote:
> > Hey all,
> >
> > If you want to figure it out theoretically, if you print out the
> > topology description, you'll have some number of state stores listed
> > in there. The number of Rocks instances should just be
> > (#global_state_stores +
> > sum(#partitions_of_topic_per_local_state_store)) . The number of
> > stream threads isn't relevant here.
> >
> > You can also figure it out empirically: the first level of
> > subdirectories in the state dir are Tasks, and then within that, the
> > next level is Stores. You should see the store directory names match
> > up with the stores listed in the topology description. The number of
> > Store directories is exactly the number of RocksDB instances you have.
> >
> > There are also metrics corresponding to each of the state stores, so
> > you can compute it from what you find in the metrics.
> >
> > Hope that helps,
> > -john
> >
> > On Thu, Jun 27, 2019 at 6:46 AM Patrik Kleindl 
> wrote:
> > >
> > > Hi Kiran
> > > Without much research my guess would be "num_stream_threads *
> > > (#global_state_stores +
> sum(#partitions_of_topic_per_local_state_store))"
> > > So 10 stores (regardless if explicitly defined or implicitely because
> of
> > > some stateful operation) with 10 partitions each should result in 100
> > > Rocksdb instances if you are running at the default of
> num_stream_threads=1.
> > >
> > > As I wrote before, start with 100.
> > > If the error persists, half the number, if not, double it ;-) Repeat as
> > > needed.
> > >
> > > If you reach the single-digit-range and the error still shows up, start
> > > searching for any iterators over a store you might not have closed.
> > >
> > > br, Patrik
> > >
> > > On Thu, 27 Jun 2019 at 13:11, emailtokir...@gmail.com <
> > > emailtokir...@gmail.com> wrote:
> > >
> > > >
> > > >
> > > > On 2019/06/27 09:02:39, Patrik Kleindl  wrote:
> > > > > Hello Kiran
> > > > >
> > > > > First, the value for maxOpenFiles is per RocksDB instance, and the
> number
> > > > > of those can get high if you have a lot of topic partitions etc.
> > > > > Check the directory (state dir) to see how many there are.
> > > > > Start with a low value (100) and see if that has some effect.
> > > > >
> > > > > Second, because I just found out, you should use
> > > > > BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
> > > > > options.tableFormatConfig();
> > > > > tableConfig.setBlockCacheSize(100*1024*1024L);
> > > > > tableConfig.setBlockSize(8*1024L);
> > > > > instead of creating a new object to prevent accidently messing up
> > > > > references.
> > > > >
> > > > > Hope that helps
> > > > > best regards
> > > > > Patrik
> > > > >
> > > > > On Thu, 27 Jun 2019 at 10:46, emailtokir...@gmail.com <
> > > > > emailtokir...@gmail.com> wrote:
> > > > >
> > > > > >
> > > > > >
> > > > > > On 2019/06/26 21:58:02, Patrik Kleindl 
> wrote:
> > > > > > > Hi Kiran
> > > > > > > You can use the RocksDBConfigSetter and pass
> > > > > > >
> > > > > > > options.setMaxOpenFiles(100);
> > > > > > >
> > > > > > > to all RocksDBs for the Streams application which limits how
> many are
> > > > > > > kept open at the same time.
> > > > > > >
> > > > > > > best regards
> > > > > > >
> > > > > > > Patrik
> > > > > > >
> > > > > > >
> > > > > > > On Wed, 26 Jun 2019 at 16:14, emailtokir...@gmail.com <
> > > > > > > emailtokir...@gmail.com> wrote:
> > > > > > >
> > > > > > > > Hi,
> > > > > > > >
> > > > > > > > We are using Kafka streams DSL APIs for doing some counter
> > > > aggregations
> > > > > > > > (running on OpenJDK 11.0.2). Our topology has some 400 sub
> > > > topologies
> > > > > > & we
> > > > > > > > are using 8 partitions in source topic. When we start
> pumping more
> > > > > > load, we
> > > > > > > > start getting RockDBException stating "too many open files".
> > > > > > > >
> > > > > > > > Here are the stack trace samples:
> > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> --
> > > > > > > > Caused by: org.rocksdb.RocksDBException: while open a file
> for
> > > > lock:
> > > > > > > > PPP.151200/LOCK: Too many open files
> > > > > > > > at