Hi everyone,
I'm going to resurrect this KIP, because I would like the community to
benefit from our solution.
In the end, we internally solved this problem using Option B: automatically
moving state directories to the correct location whenever they're no longer
aligned with the Topology. We
In the KIP, for Option A I suggested a new path of:
/state/dir/stores//
I made the mistake of thinking that the rocksdb/ segment goes *after* the
store name in the current scheme, e.g.
/state/dir//[/rocksdb]
This is a mistake. I'd always intended for a combination of the store name
and
Thanks for the clarification John!
Nick, sorry that I was not super clear in my latest email. I meant exactly
what John said.
Just to clarify, I do think that this KIP is relatively orthogonal to the
named topology work; as long as we still keep the topo name encoded it
should be fine since two
Hi Nick,
When Guozgang and I were chatting, we realized that it’s not completely
sufficient just to move the state store directories, because their names are
not unique. In particular, more than one partition of the store may be assigned
to the same instance. Right now, this is handled because
Hi Guozhang,
Sorry I haven't had the time to respond to your earlier email, but I just
wanted to clarify something with respect to your most recent email.
My original plan in option A is to remove the entire Task ID from the State
Store path, which would insulate it from any changes to the Task
Just to follow-up on this thread, I had another chat with John regarding
option a) and I think the key thought is that, today the task-id is in the
form of [sub-topologyID]-[partitionID] --- and in the future with
named-topology it could be extended to three digits as
Hello Nick,
I think I'm on the same page of the scope of your KIP, and what I was
trying to get is that, there are some other efforts going on in parallel
that tries to identify if two topologies, or some part of them, are
isomorphic in structure, and hence their corresponding persistent states
Thanks, Nick,
It sounds like we're on the same page. I didn't think (A)
would be fundamentally "hard", just that it might be a pain
in practice. Reading your response, if you're up for it, it
sounds like a prototype of (A) would be the tie-breaker
between the two approaches.
To be honest, I've
Hi everyone,
Guozhang, the scope of my KIP is specifically about deploying structural
changes to existing applications, i.e. "upgrades". Sharing state between
different applications was not in the scope of my original proposal.
John's email has it exactly right, and I think this points to my KIP
Hello all,
Thanks for the KIP, Nick!
Based on this conversation, I think I might have misread the
KIP, but it looks like Nick is just proposing a small fix to
the existing compatability mechanism.
Although we tell people to avoid changing topologies on the
fly in general, we also tell them
Hi folks,
I think the NamedTopology work would help with the convenience of the
solution for this KIP, but I feel it is not by itself the solution here. If
I'm not mistaken, the scope of this KIP is trying to tackle that, *assuming
the developer already knows* a new topology or part of the
Hi Guozhang, Sophie,
Thanks for both taking the time to review my proposal.
I did actually see the NamedTopology classes, and noted that they were
internal. I didn't realise they are part of an intended solution to this
problem, that's very interesting. I'm going to try to find some time to
take
Hey Nick,
thanks for the KIP, this is definitely a much-needed feature. I've actually
been working on
a somewhat similar feature for a while now and have a good chunk of the
implementation
completed -- but so far it's only exposed via internal APIs and hasn't been
brought to a KIP
yet, as it's a
Hello Nick,
Thanks for bringing this up and for the proposed options. I read though
your writeup and here are some of my thoughts:
1) When changing the topology of Kafka Streams, the developer need to first
decide if the whole topology's persisted state (including both the state
store as well as
Hi everyone,
I'd like to start a discussion on Kafka Streams KIP-816 (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-816%3A+Topology+changes+without+local+state+reset
)
This KIP outlines 3 possible solutions to the problem, and I plan to
whittle this down to a definitive solution based
15 matches
Mail list logo