;>
> > >> 2. I just want to make sure that I understand the reconciliation
> > >> logic correctly. When
> > >> an "INCREMENTAL" LeaderAndIsr Request is received, the broker will
> also
> > >> reconcile
> > >> when the l
t; 25. LeaderAndIsrResponse v5, StopReplicaResponse v4: Could we use this
> opportunity to organize the response in 2 levels, first by topic, then by
> partition, as most other requests/responses?
>
> 26. FetchRequest v13 : Should forgotten_topics_data use topicId too?
>
> 27. &q
given an id. Is there any reason not to
> implement that in this KIP?
>
> Many thanks,
>
> Tom
>
> On Mon, Sep 21, 2020 at 9:54 PM Justine Olshan
> wrote:
>
> > Hi all,
> >
> > After thinking about it, I've decided to remove the topic name from the
> > Fet
Hi all,
I'd like to call a vote on KIP-516: Topic Identifiers. Here is the KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers
The discussion thread is here:
if there is anything else we should discuss before
voting.
Thank you,
Justine
On Fri, Sep 18, 2020 at 9:46 AM Justine Olshan wrote:
> Hi Jun,
>
> I see what you are saying. For now we can remove the extra information.
> I'll leave the option to add more fields to the file in the future. The KI
t; Thanks for the reply.
>
> 13. If the log directory is the source of truth, it means that the
> redundant info in the metadata file will be ignored. Then the question is
> why do we need to put the redundant info in the metadata file now?
>
> Thanks,
>
> Jun
>
> On Thu,
ween the log dir and the metadata file. For example, if
> the log dir is topicA-1 and the metadata file in it has topicB and
> partition 0 (say due to a bug or manual modification), which one do we use
> as the source of truth?
>
> Jun
>
> On Thu, Sep 17, 2020 at 3:4
hose fields in
> the metadata file when we actually change the directory structure.
>
> Thanks,
>
> Jun
>
> On Thu, Sep 17, 2020 at 2:01 PM Justine Olshan
> wrote:
>
> > Hello all,
> >
> > I've thought some more about removing the topic name field fr
.
I'm hoping to start the voting process soon, so let me know if there is
anything else we should discuss.
Thank you,
Justine
On Tue, Sep 15, 2020 at 3:57 PM Justine Olshan wrote:
> Hello again,
> To follow up on some of the other comments:
>
> 10/11) We can remove the top
Justine Olshan created KAFKA-10496:
--
Summary: Create an in-memory DNS server for ClientUtilsTest and
ClusterConnectionStatesTest
Key: KAFKA-10496
URL: https://issues.apache.org/jira/browse/KAFKA-10496
can't just send a StopReplica delete the
topic immediately like we'd want to for this KIP.
This makes me wonder if we want tagged fields on all the requests after
all. Let me know your thoughts!
Justine
On Tue, Sep 15, 2020 at 1:03 PM Justine Olshan wrote:
> Hi all,
> Jun brought up
s first before doing
> other actions. So, is StopReplicaRequest V2 needed?
>
> Jun
>
> On Fri, Sep 11, 2020 at 10:31 AM John Roesler wrote:
>
> > Thanks, Justine!
> >
> > Your response seems compelling to me.
> >
> > -John
> >
> > On Fri
em to mention changing
> > > DeleteTopicsRequest to identify the topic using an id. Maybe that's out
> > of
> > > scope, but DeleteTopicsRequest is not listed among the Future Work APIs
> > > either.
> > >
> > > Kind regards,
> > >
> &g
Hello all, it's been almost a year! I've made some changes to this KIP and hope
to continue the discussion.
One of the main changes I've added is now the metadata response will include
the topic ID (as Colin suggested). Clients can obtain the topicID of a given
topic through a
[
https://issues.apache.org/jira/browse/KAFKA-8601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Justine Olshan resolved KAFKA-8601.
---
Resolution: Fixed
> Producer Improvement: Sticky Partitio
. My
> >
> > point is
> >
> > that
> >
> > this KIP is not introducing any new functionality with regards to
> >
> > what
> >
> > rogue clients can do. It's using the existing protocol that is
> >
> > already
> >
> > e
; what
> > > > > > > rogue clients can do. It's using the existing protocol that is
> > > already
> > > > > > > exposed via the AdminClient. So, I don't think we need to
> address
> > > it in
> > > > > > > this KIP
s or configure custom constraints before these can be
> overridden by clients but not vice versa. There should be an option on
> brokers whether those constraints can be overridden by producers or
> not.
>
> Thanks,
> Satish.
>
> On Tue, Aug 6, 2019 at 11:39 PM Justine O
will present potential issue of rogue client creating ton of
> > topic-partitions and potentially bringing down the service for everyone
> >
> > or
> >
> > degrade the service itself.
> > By reading the KIP its not clear to me that there is a clear way to block
> &
n
>
>
> > On Wed, Jul 31, 2019 at 9:10 AM Mickael Maison >
> > wrote:
> >
> > > Hi Justine,
> > >
> > > We can rely on KIP-464 which allows to omit the partition count or
> > > replication factor when creating a topic. In that case
Mickael Maison
wrote:
> Hi Justine,
>
> We can rely on KIP-464 which allows to omit the partition count or
> replication factor when creating a topic. In that case, the broker
> defaults are used.
>
> On Wed, Jul 31, 2019 at 4:55 PM Justine Olshan
> wrote:
> >
>
this is what we're aiming to
> deprecate. When requesting metadata we can set the
> "allow_auto_topic_creation" field to false to avoid the broker auto
> creation. Then if the topic is not existing, send a
> CreateTopicRequest.
>
> What do you think?
>
> On Mon, Jul 29,
se the AdminClient (CreateTopics request) to
> create topics? and not rely on the broker auto create.
> I'm guessing the answer is yes but can you make it explicit.
>
> Thank you
>
> On Wed, Jul 24, 2019 at 6:23 PM Justine Olshan
> wrote:
> >
> > Hi,
> > J
the
partition, and the partition method will be left alone.
Please take a look when you get a chance and let me know what you think.
Thank you,
Justine
On Fri, Jul 26, 2019 at 9:31 AM Justine Olshan wrote:
> Hi Jun,
> I agree that it is confusing. I think there might be a way to not
>
e both
> partition() and computePartition(). It's not clear to them which one they
> should be using and which one takes precedence.
>
> Jun
>
> On Fri, Jul 19, 2019 at 9:39 AM Justine Olshan
> wrote:
>
> > Thanks everyone for reviewing and voting!
> >
> > I'm marking thi
the creation of
topics, so the loss of creation functionality will not be a big problem.
I'm happy to discuss any other compatibility problems or components of
this KIP.
Thank you,
Justine
On Wed, Jul 17, 2019 at 9:11 AM Justine Olshan wrote:
> Hello all,
>
> I was looking at this
gt; > wrote:
> > >
> > > +1 binding, looks like a nice improvement. Thanks!
> > >
> > > -David
> > >
> > > On Wed, Jul 17, 2019 at 6:17 PM Justine Olshan
> > wrote:
> > >
> > > > Hello all,
> > >
Stanislav Kozlovski
> wrote:
>
> > +1 (non-binding)
> >
> > Thanks!
> >
> > On Fri, Jul 12, 2019 at 6:02 PM Gwen Shapira wrote:
> >
> > > +1 (binding)
> > >
> > > Thank you for the KIP. This was long awaited.
> > >
>
the code to have the
producer's configurations take precedence is possible, but I was wondering
what everyone thought.
Thank you,
Justine
On Fri, Jul 12, 2019 at 2:49 PM Justine Olshan wrote:
> Just a quick update--
>
> It seems that enabling both the broker and producer configs w
this. Perhaps we would want to add more
to the documentation of the the producer configs to clarify.
Thank you,
Justine
On Fri, Jul 12, 2019 at 9:28 AM Justine Olshan wrote:
> Hi Colin,
>
> Thanks for looking at the KIP. I can definitely add to the title to make
> it more clear.
>
&
:
> On Fri, Jul 12, 2019, at 09:02, Justine Olshan wrote:
> > Hello all,
> >
> > Jun, thanks for taking a look at my KIP! We were also concerned about
> > batches containing a single record so we kept this in mind for the
> > implementation. The decision to switch the stick
y setting
> the -side setting to false-- but this is not the case, of course.
>
> best,
> Colin
>
>
> On Thu, Jul 11, 2019, at 16:22, Justine Olshan wrote:
> > Hi Dhruvil,
> >
> > Thanks for reading the KIP!
> > That was the general idea
o
> > implemented right away in the first PR, of course.) It would be an
> option
> > for people who wanted to configure this behavior.
> >
> > best,
> > Colin
> >
> >
> > On Wed, Jul 10, 2019, at 08:48, Justine Olshan wrote:
> > > Hi M,
> > >
e broker, for example?
>
> Thanks,
> Dhruvil
>
> On Thu, Jul 11, 2019 at 10:28 AM Justine Olshan
> wrote:
>
> > Hello all,
> >
> > I'd like to start a discussion thread for KIP-487.
> > This KIP plans to deprecate the current system of auto-creating topics
>
://cwiki.apache.org/confluence/display/KAFKA/KIP-487%3A+Automatic+Topic+Creation+on+Producer
Thank you,
Justine Olshan
Justine Olshan created KAFKA-8657:
-
Summary: Automatic Topic Creation on Producer
Key: KAFKA-8657
URL: https://issues.apache.org/jira/browse/KAFKA-8657
Project: Kafka
Issue Type: Improvement
the batch is sent.
I don't think you can have both round-robin and sticky partition behavior.
Thank you,
Justine Olshan
On Wed, Jul 10, 2019 at 1:54 AM M. Manna wrote:
> Thanks for the comments Colin.
>
> My only concern is that this KIP is addressing a good feature and having
> t
Hello all,
I'd like to start the vote for KIP-480 : Sticky Partitioner.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-480%3A+Sticky+Partitioner
Thank you,
Justine Olshan
Hello all,
If there are no more comments or concerns, I would like to start the vote
on this tomorrow afternoon.
However, if there are still topics to discuss, feel free to bring them up
now.
Thank you,
Justine
On Tue, Jul 2, 2019 at 4:25 PM Justine Olshan wrote:
> Hello again,
>
>
Hello again,
Another update to the interface has been made to the KIP.
Please let me know if you have any feedback!
Thank you,
Justine
On Fri, Jun 28, 2019 at 2:52 PM Justine Olshan wrote:
> Hi all,
> I made some changes to the KIP.
> The idea is to clean up the code, make beha
f
> > it gets called when an explicit partition id has been provided.
>
> Agreed.
>
> best,
> Colin
>
> >
> > Ismael
> >
> > On Mon, Jun 24, 2019, 2:04 PM Justine Olshan
> wrote:
> >
> > > Hello,
> > > This is the discuss
Moving the previous comment to the PR discussion. :)
On Thu, Jun 27, 2019 at 10:51 AM Justine Olshan
wrote:
> I was going through fixing some of the overloaded methods and I realized I
> overloaded the RecordAccumulator constructor. I added a parameter to
> include the partitioner so I
a partition where no replica was down
> if it's available. Such partitions are safer in general. There could be
> some downsides too, so worth thinking about the trade-offs.
>
> Ismael
>
> On Thu, Jun 27, 2019, 10:24 AM Justine Olshan
> wrote:
>
> > Ismael,
>
hat
> specifies it and its parameters. In particular, it would good to specify if
> it gets called when an explicit partition id has been provided.
>
> Ismael
>
> On Mon, Jun 24, 2019, 2:04 PM Justine Olshan wrote:
>
> > Hello,
> > This is the discussion thread for KIP-480:
ch` where we return null in the case of no change
> needed. Have we considered using Java's Optional type to avoid null values?
>
> Best,
> Stanislav
>
> On Tue, Jun 25, 2019 at 11:29 PM Colin McCabe wrote:
>
> > No worries. Thanks for fixing it!
> > C.
> >
>
Also apologies on the late link to the jira, but apparently https links do
not work and it kept defaulting to an image on my desktop even when it
looked like I put the correct link in. Weird...
On Tue, Jun 25, 2019 at 1:41 PM Justine Olshan wrote:
> I came up with a good solut
havior.
>
> Colin
>
>
> On Tue, Jun 25, 2019, at 12:04, Justine Olshan wrote:
> > I also just noticed that if we want to use this method on the keyed
> record
> > case, I will need to move the method outside of the sticky (no key, no
> set
> > partition) ch
> > > test classes in the KIP since they're not visible to users or external
> > > developers.
> > >
> > > It seems like maybe the performance tests should get their own section.
> > > Right now, the way the layout is makes it look like they are part of
t; It seems like maybe the performance tests should get their own section.
> > > Right now, the way the layout is makes it look like they are part of
> the
> > > "Compatibility, Deprecation, and Migration Plan"
> > >
> > > best,
> > > Colin
&g
Justine Olshan created KAFKA-8601:
-
Summary: Producer Improvement: Sticky Partitioner
Key: KAFKA-8601
URL: https://issues.apache.org/jira/browse/KAFKA-8601
Project: Kafka
Issue Type
Hello,
This is the discussion thread for KIP-480: Sticky Partitioner.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-480%3A+Sticky+Partitioner
Thank you,
Justine Olshan
Hi, I was wondering if I could have permission to create a KIP. My wiki
username is jolshan.
Thank you,
Justine Olshan
401 - 452 of 452 matches
Mail list logo