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
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
> > > 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
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
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
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,
>
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
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:
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
://cwiki.apache.org/confluence/display/KAFKA/KIP-487%3A+Automatic+Topic+Creation+on+Producer
Thank you,
Justine Olshan
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
>
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,
> > >
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
:
> 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
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.
>
&
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
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.
> >
>
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
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,
; 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
. 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
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
>
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
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
> &
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
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
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
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.
> > >
>
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,
> > >
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
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
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
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
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
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
/7efa8cd169cadc7dc9cf86a7c0dbbab1836ddb5024d310fcebacf80c@%3Cdev.kafka.apache.org%3E
Please take a look and vote if you have a chance.
Thanks,
Justine
On Tue, Sep 22, 2020 at 8:52 AM Justine Olshan wrote:
> Hi all,
>
> I'd like to call a vote on KIP-516: Topic Identifiers. Here is the KIP:
>
> https://c
topReplicaResponse below?
> > > > > > > StopReplica Response (Version: 4) => error_code [topics]
> > > > > > > error_code => INT16
> > > > > > > topics => topic topic_id* [partitions]
> > > > > > >
> >
;>
> > >> 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
> > >> 1. It seems that the schema of the StopReplicaRequest is slightly
> > > >> outdated.
> > > >> We
> > > >> did some changes as part of KIP-570. V3 is already organized by
> > topics.
> > > >>
> > > >> 2. I just
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
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
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
On Mon, Sep 28, 2020 at 11:38 AM Justine Olshan
wrote:
> Hello all,
>
> I just wanted to follow up on this discussion. Did we come to an
> understanding about the directory structure?
>
> I think the biggest question here is what is acceptable to leave out due
> to scope v
sue seems unfixed.
>
> 40. Since UUID is public facing, could you include its definition?
>
> 41. StopReplicaResponse still includes the topic field.
>
> 42. "It is unnecessary to include the name of the topic in the following
> Request/Response calls" It would b
update. The KIP looks good to me now. Just a minor comment
> below.
>
> 30. Perhaps "option[UUID]" can be put in the doc.
>
> Jun
>
> On Thu, Oct 1, 2020 at 3:28 PM Justine Olshan
> wrote:
>
> > Hi Jun,
> > Thanks for the response!
> >
>
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
.
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
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
> > > Basically my point is that we have to create additional
> infrastructure
> > > here
> > > > to support the name/id mapping, so I wanted to understand if we just
> > > > consider this a sort of tech debt. It would be useful to cover how we
> &g
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,
Hi George,
I've been looking at the discussion on improving the sticky partitioner,
and one of the potential issues we discussed is how we could get
information to the partitioner to tell it not to choose certain partitions.
Currently, the partitioner can only use availablePartitionsForTopic. I
Hi Evelyn,
Thanks for taking a look at improving the sticky partitioner! These edge
cases seem like they would cause quite a bit a trouble.
I think the idea to check for max.in.flight.requests.per.connection is a
good one, but one concern I have is how this information will be available
to the
The user has been blocked. https://issues.apache.org/jira/browse/INFRA-21268
On Tue, Jan 5, 2021 at 2:52 PM Brandon Brown
wrote:
> Is there any way to block Tim van der Kooi from making issues? I’m getting
> about 10 new email issues created a minute.
>
> Brandon Brown
>
>
+Topic+Identifiers#KIP516:TopicIdentifiers-AdminClientSupport
Thanks,
Justine
On Tue, Mar 30, 2021 at 2:38 PM Justine Olshan wrote:
> Hi all,
> Another quick update. After some offline discussion with KIP-500 folks,
> I'm making a small tweak to one of the configs in KIP-516.
Hi Alyssa,
Are you asking to be added to JIRA? If so, can you provide your jira
username?
Thanks,
Justine
On Fri, Apr 30, 2021 at 9:48 AM Alyssa Huang
wrote:
> Hello,
>
> I'm interested in contributing to Kafka! Can I be added to the project?
>
> Best,
> Alyssa
>
now if there are any questions,
Justine
On Thu, Feb 18, 2021 at 10:16 AM Justine Olshan
wrote:
> Hi all,
> I realized that the DISCUSS thread got very long, so I'll be posting
> updates to this thread from now on.
> Just a quick update to the KIP. As a part of
> https://issues.apache.org/
/display/KAFKA/KIP-516%3A+Topic+Identifiers#KIP516:TopicIdentifiers-LeaderAndIsrRequestv5
Please let me know if you have any thoughts or concerns with this change.
Thanks,
Justine
On Mon, Oct 19, 2020 at 8:50 AM Justine Olshan wrote:
> Thanks everyone for the votes. KIP-516 has been accep
Hi Mason,
Thanks for the KIP. I had a few questions.
Are you saying that we will be keeping the original (untagged) offline
partitions count metric? I was a little confused by the wording in the KIP>
I'm also curious about potential performance impacts. Have you looked into
this?
Thanks,
Hi Artem,
Thanks for working on improving the Sticky Partitioner!
I had a few questions about this portion:
*The batching will continue until either an in-flight batch completes or we
hit the N bytes and move to the next partition. This way it takes just 5
records to get to batching mode, not 5
Hi all,
I've filed a bug for an extra map allocation that is used in the fetch
path. https://issues.apache.org/jira/browse/KAFKA-13512
I think it qualifies as a blocker since this path is used pretty frequently
and it looks to be a regression.
I also have a PR open to fix the issue. With this
Hi Konstantine,
I've discovered a bug with topic IDs that can be encountered when upgrading
from IBP versions below 2.8.
Since 2.8, when handling leader and isr requests, the request topic IDs are
compared to the log IDs and partitions with inconsistent IDs are skipped.
With a change introduced
e 3.6 branch as well.
>
> 1. https://github.com/apache/kafka/pull/14324.
>
> Thanks,
> Satish.
>
> On Tue, 5 Sept 2023 at 05:06, Justine Olshan
> wrote:
> >
> > Sorry I meant to add the jira as well.
> > https://issues.apache.org/jira/browse/KAFKA-15424
Hey Satish,
We just discovered a gap in KIP-890 part 1. We currently don't verify on
txn offset commits, so it is still possible to have hanging transactions on
the consumer offsets partitions.
I've opened a jira to wire the verification in that request.
t touch the ISR then the
> ISR behavior will be the same as the current. I am open to the proposal
> that directly starting unclean recovery if the last leader fails. Let's see
> if other folks hope to have more if Unclean Recover delivers first.
>
> On Tue, Aug 29, 2023 at 4:53 P
.
>
> Thanks,
> Satish.
>
> On Tue, 12 Sept 2023 at 04:44, Justine Olshan
> wrote:
> >
> > Hey Satish,
> >
> > We just discovered a gap in KIP-890 part 1. We currently don't verify on
> > txn offset commits, so it is still possible to have hanging tra
bly best to have the conversation in the JIRA ticket vs the release
> > thread. Generally, we want to only include low risk bug fixes that are
> > fully compatible in patch releases.
> >
> > Ismael
> >
> > On Tue, Sep 12, 2023 at 7:16 AM Justine Olshan
> >
gt; wrote:
> > >
> > > > Justine,
> > > >
> > > > Probably best to have the conversation in the JIRA ticket vs the
> release
> > > > thread. Generally, we want to only include low risk bug fixes that
> are
> > > > fully com
Hey Calvin,
Thanks for the KIP. This will close some of the gaps in leader election! I
has a few questions:
*>* *High Watermark can only advance if the ISR size is larger or equal
to min.insync.replicas*.
For my understanding, will we block replication? Or just the high watermark
advancement?
On Thu, 14 Sept 2023 at 07:29, Luke Chen wrote:
> >
> > Hi Satish,
> >
> > Since this PR:
> > https://github.com/apache/kafka/pull/14366 only changes the doc, I've
> > backported to 3.6 branch. FYI.
> >
> > Thanks.
> > Luke
> >
> >
nput of people familiar with
> > the
> > > > admin
> > > > > > client to weigh in on the Jira ticket about whether we should
> > continue
> > > > to
> > > > > > preserve the current behavior (if the consensus is that we
> should
Sorry I meant to add the jira as well.
https://issues.apache.org/jira/browse/KAFKA-15424
Justine
On Mon, Sep 4, 2023 at 4:34 PM Justine Olshan wrote:
> Hey Satish,
>
> I was working on adding dynamic configuration for
> transaction verification. The PR is approved and ready to merg
If you would like to read any historical conversation you can do so from
the archive here: https://lists.apache.org/list.html?dev@kafka.apache.org
As Josep said, in order to reply, you can use your own client without
logging in.
Hope this helps!
Justine
On Wed, Nov 1, 2023 at 10:01 AM Josep
This makes sense to me. Thanks for following up, Stan.
On Thu, Nov 2, 2023 at 7:02 AM Stanislav Kozlovski
wrote:
> Hi all,
>
> Given the discussion here and the lack of any pushback, I have changed the
> dates of the release:
> - KIP Freeze - *November 22 *(moved 4 days later)
> - Feature
Hey Colin,
For context on this specific issue, we have opened a JIRA to consider
thread safety in the future. Another option is documentation or to make
thread local.
Don't want to detract too much from this conversation, but did want to say
there is a JIRA to discuss the buffer specific problem.
Hey folks,
Thanks for replying. If we could file a JIRA to track this work, that would
be great.
Thanks,
Justine
On Tue, Oct 24, 2023 at 11:55 AM Colin McCabe wrote:
> Hi Neil,
>
> Yes, I think we should probably report the UncleanLeaderElectionsPerSec
> metric in KRaft. We don't have it
Hi Satish,
I've done the following:
- Verified signature
- Built from Java 17/Scala 2.13 and Java 8/Scala 2.11
- Run unit + integration tests
- Ran a shorter Trogdor transactional-produce-bench on a single broker
cluster (KRaft and ZK) to verify transactional workloads worked reasonably
Minor
Hey all -- I noticed we still have the system tests as something that will
be updated. Did we get a run for this RC?
On Mon, Oct 2, 2023 at 1:24 PM Bill Bejeck wrote:
> Hi Satish,
>
> Thanks for running the release.
> I performed the following steps:
>
>- Validated all the checksums,
up with the documentation Divij mentioned outside this thread.
Thanks,
Justine
On Mon, Oct 2, 2023 at 3:05 PM Greg Harris
wrote:
> Hey Satish,
>
> I verified KIP-898 functionality and the KAFKA-15473 patch.
> +1 (non-binding)
>
> Thanks!
>
> On Mon, Oct 2, 2023 a
gt; > > In short, no failed tests. The flaky tests will pass in the 2nd run.
> > >
> >
> https://drive.google.com/drive/folders/1qwIKg-B4CBrswUeo5fBRv65KWpDsGUiS?usp=sharing
> > >
> > > Thank you.
> > > Luke
> > >
> > > On Tue,
Hey Artem,
Thanks for the KIP. I had a question about epoch bumping.
Previously when we send an InitProducerId request on Producer startup, we
bump the epoch and abort the transaction. Is it correct to assume that we
will still bump the epoch, but just not abort the transaction?
If we still bump
Hey Girish,
You may need to confirm the unsubscription with a second email.
When I was switching subscription emails, I sent one to the unsubscribe
email and then I got a reply.
In the reply it asked me to send to a unique email address to confirm. Look
for one from dev-h...@kafka.apache.org.
Sorry -- not MV but software version.
On Wed, Oct 4, 2023 at 9:51 AM Justine Olshan wrote:
> Catching up with this discussion.
>
> I was just curious -- have we had other instances where downgrading MV is
> not supported? I think Kafka typically tries to support downgrades, and
Catching up with this discussion.
I was just curious -- have we had other instances where downgrading MV is
not supported? I think Kafka typically tries to support downgrades, and I
couldn't think of other examples.
Thanks,
Justine
On Wed, Oct 4, 2023 at 9:40 AM Calvin Liu
wrote:
> Hi Jun,
>
Thanks Calvin.
I think this will be very helpful going forward to minimize data loss.
+1 from me (binding)
Justine
On Wed, Sep 20, 2023 at 3:42 PM Calvin Liu
wrote:
> Hi all,
> I'd like to call for a vote on KIP-966 which includes a series of
> enhancements to the current ISR model.
>
>-
Hey Neil,
I was taking a look at this code, and noticed that some unclean leader
election params were not implemented.
https://github.com/apache/kafka/blob/4612fe42af0df0a4c1affaf66c55d01eb6267ce3/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java#L499
I know you
Hey Neil,
I've given you permissions for wiki access. You should be able to create a
KIP now. Let me know if you have any other issues.
Justine
On Sun, Aug 20, 2023 at 5:46 AM Neil Buesing wrote:
> Wiki ID: neil.buesing
>
> JIRA ID: nbuesing
>
> when I click on the signup link through in the
Hey Hailey,
Can you share your wiki ID so I can grant you access? If you don't yet have
one you may need to create an account.
Justine
On Fri, Aug 18, 2023 at 3:58 PM Hailey Ni wrote:
> Hi,
>
> Can I get edit access to Apache Kafka's Wiki please?
>
> Thanks,
> Hailey
>
Hmmm. That's a bit strange if you are subscribed to dev@kafka.apache.org,
you should be getting responses.
Let me know if this one also doesn't work.
Justine
On Fri, Aug 25, 2023 at 6:04 AM Neil Buesing wrote:
> Justine,
>
> Thanks for taking care of the wiki access; weird in that response
ing stuck. Please
> update the release plan wiki and merge all the required changes to 3.6
> branch.
>
> Thanks,
> Satish.
>
> On Thu, 24 Aug 2023 at 22:19, Justine Olshan
> wrote:
> >
> > Hey Satish,
> > Does it make sense to include KIP-890 part 1? It
Hey Satish,
Does it make sense to include KIP-890 part 1? It prevents hanging
transactions for older clients. (An optimization and stronger EOS
guarantees will be included in part 2)
Thanks,
Justine
On Mon, Aug 21, 2023 at 3:29 AM Satish Duggana
wrote:
> Hi,
> 3.6 branch is created. Please
Hey Raghu,
I've added your ID to give you permissions to the wiki.
I'm not sure if committers can change your jira ID. You may want to try to
create a new account or file a ticket with apache for that.
Let me know if there are any issues.
Justine
On Mon, Aug 28, 2023 at 11:54 AM Raghu Baddam
Hey Hailey,
You should have permissions now!
Justine
On Mon, Aug 21, 2023 at 2:11 PM Hailey Ni wrote:
> Hi,
>
> This is Hailey. Wiki ID: hni. May I request edit permission to the Kafka
> Wiki please?
>
> Thanks,
> Hailey
>
Hi David,
Thanks for sharing this KIP! Really exciting to hear how we are changing
the protocol! The motivation section really made me realize how useful this
change will be.
I've done a first pass of the KIP, and may have more questions, but thought
I'd start with a few I thought of already.
Hi Artem,
Thanks for the update to include motivation. Makes sense to me.
Justine
On Wed, Jun 29, 2022 at 6:51 PM Luke Chen wrote:
> Hi Artem,
>
> Thanks for the update.
> LGTM.
>
> Luke
>
> On Thu, Jun 30, 2022 at 6:51 AM Artem Livshits
> wrote:
>
> > Thank you for your feedback. I've updated
ich makes sense. To help
> > avoid this situation, do we want to consider a default value that is the
> > same as the delivery timeout?
> >
> > Thanks again for the KIP.
> >
> > Bill
> >
> > On Thu, Jul 21, 2022 at 4:54 PM Justine Olshan
> > wrote:
> >
>
1 - 100 of 437 matches
Mail list logo