Hey all,

Thanks much for all the comments. I have a rough idea of how to shrink
partitions of a topic. I need some time to think through and write down the
details of the procedure. Then I will reply to your comments.

Thanks,
Dong



On Thu, Feb 22, 2018 at 7:18 PM, Matthias J. Sax <matth...@confluent.io>
wrote:

> One more thought:
>
> What about older Producer/Consumers? They don't understand the new
> protocol. How can we guarantee backward compatibility?
>
> Or would this "only" imply, that there is no ordering guarantee for
> older clients?
>
>
> -Matthias
>
>
> On 2/22/18 6:24 PM, Matthias J. Sax wrote:
> > Dong,
> >
> > thanks a lot for the KIP!
> >
> > Can you elaborate how this would work for compacted topics? If it does
> > not work for compacted topics, I think Streams API cannot allow to scale
> > input topics.
> >
> > This question seems to be particularly interesting for deleting
> > partitions: assume that a key is never (or for a very long time)
> > updated, a partition cannot be deleted.
> >
> >
> > -Matthias
> >
> >
> > On 2/22/18 5:19 PM, Jay Kreps wrote:
> >> Hey Dong,
> >>
> >> Two questions:
> >> 1. How will this work with Streams and Connect?
> >> 2. How does this compare to a solution where we physically split
> partitions
> >> using a linear hashing approach (the partition number is equivalent to
> the
> >> hash bucket in a hash table)? https://en.wikipedia.org/wiki/
> Linear_hashing
> >>
> >> -Jay
> >>
> >> On Sat, Feb 10, 2018 at 3:35 PM, Dong Lin <lindon...@gmail.com> wrote:
> >>
> >>> Hi all,
> >>>
> >>> I have created KIP-253: Support in-order message delivery with
> partition
> >>> expansion. See
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 253%3A+Support+in-order+message+delivery+with+partition+expansion
> >>> .
> >>>
> >>> This KIP provides a way to allow messages of the same key from the same
> >>> producer to be consumed in the same order they are produced even if we
> >>> expand partition of the topic.
> >>>
> >>> Thanks,
> >>> Dong
> >>>
> >>
> >
>
>

Reply via email to