detail that doesn't need to be in the KIP.
best,
Colin
> I assumed broker leases are implemented as a hard state. In that case, to
> check for broker id conflict, we need to check the broker ids at two places
> 1. Pending broker registrations (which are yet to be committed) 2. Alr
gt;
> > > > wrote:
> > > >
> > > > > Hi Ron/Colin,
> > > > >
> > > > > Without any restrictions, if delegation tokens can be used to create
> > > new
> > > > > users or change the password of the user you a
Dagostino wrote:
> Hi Colin. I've removed that requirement from the KIP and updated the PR
> accordingly.
>
> Ron
>
> On Fri, Aug 28, 2020 at 2:27 PM Colin McCabe wrote:
>
> > Hi Ron,
> >
> > Thanks for the update. I agree with all of these changes,
[
https://issues.apache.org/jira/browse/KAFKA-10384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-10384.
--
Fix Version/s: 2.7.0
Resolution: Fixed
> Separate converters from generated messa
single threaded as well. I think it makes sense for the
controller, since otherwise locking becomes very messy. I'm not sure I
understand your question about duplicate broker ID detection, though. There's
a section in the KIP about this -- is there a detail we should add there?
lication of the broker." Can you give a
little more context?
best,
Colin
>
> Thanks,
> Unmesh
>
> On Sat, Aug 29, 2020 at 4:50 AM Colin McCabe wrote:
>
> > Hi all,
> >
> > I'm thinking of calling a vote on KIP-631 on Monday. Let me know if
> > t
Thanks. I will be adding more to the prototype and will be happy to help
> and collaborate.
>
> Thanks,
> Unmesh
>
> On Tue, Aug 11, 2020 at 12:28 AM Colin McCabe wrote:
>
> > Hi Jose,
> >
> > That'a s good point that I hadn't considere
Hi Ron,
Thanks for the update. I agree with all of these changes, except I think we
should discuss this one further:
On Wed, Aug 26, 2020, at 14:59, Ron Dagostino wrote:
>
> 2. We added a restriction to not allow users who authenticated using
> delegation tokens to create or update user SCRAM c
On Tue, Aug 11, 2020, at 11:30, Ismael Juma wrote:
> Thanks for the KIP, +1 (binding). A couple of comments:
>
> 1. We have "quorum.voters=1@kafka-1:9092, 2@kafka-2:9092,
> 3@kafka-3:9092". Could
> this be a bit confusing given that the authority part of a url is defined
> as "authority = [userinf
Colin McCabe created KAFKA-10384:
Summary: Separate converters from generated messages
Key: KAFKA-10384
URL: https://issues.apache.org/jira/browse/KAFKA-10384
Project: Kafka
Issue Type: Bug
Hi Jose,
That'a s good point that I hadn't considered. It's probably worth having a
separate leader change message, as you mentioned.
Hi Unmesh,
Thanks, I'll take a look.
best,
Colin
On Fri, Aug 7, 2020, at 11:56, Jose Garcia Sancio wrote:
> Hi Unmesh,
>
> Very cool prototype!
>
> Hi Coli
On Mon, Aug 3, 2020, at 20:55, Jason Gustafson wrote:
> Hi Colin,
>
> Thanks for the responses.
>
> > I have a few lingering questions. I still don't like the fact that the
> > leader epoch / fetch epoch is 31 bits. What happens when this rolls over?
> > Can we just make this 63 bits now so tha
know if you have any questions/concerns,
> otherwise we can assume this change is acceptable.
>
> Ron
>
> On Tue, Jul 21, 2020 at 1:57 PM Colin McCabe wrote:
>
> > Hi all,
> >
> > With binding +1s from Rajini Sivaram, David Arthur, and Boyang Chen,
On Mon, Aug 3, 2020, at 15:51, Jose Garcia Sancio wrote:
> Thanks for the KIP Colin,
>
> Here is a partial review:
>
> > 1. Even when a broker and a controller are co-located in the same JVM, they
> > must
> > have different node IDs
>
> Why? What problem are you trying to solve?
>
Hi Jose,
Hi Jason,
The KIP looks great. Thanks for all the work you've put into this.
I have a few lingering questions. I still don't like the fact that the leader
epoch / fetch epoch is 31 bits. What happens when this rolls over? Can we
just make this 63 bits now so that we never have to worry abou
clients don't get their metadata
responses from the controller quorum anyway.
best,
Colin
> Thanks,
> Unmesh
>
>
>
> On Thu, Jul 30, 2020 at 6:04 AM Colin McCabe wrote:
>
> > On Thu, Jul 23, 2020, at 23:02, Boyang Chen wrote:
> > > Hey Col
; failure?
> >
> > 4. In the rejected alternatives, we mentioned we don't want to combine
> > heartbeats and fetch and listed out the reason was due to extra complexity.
> > However, we should also mention some cons caused by this model, for example
> > we are doing
This is something that we've gone back and forth about, but
overall I think it's good to at least implement the simple thing first.
best,
Colin
>
> Boyang
>
> On Wed, Jul 15, 2020 at 5:30 PM Colin McCabe wrote:
>
> > On Mon, Jul 13, 2020, at 11:08, Boyang C
address our other discussed points as well in the KIP, let me know
> >> if you have further questions.
> >>
> >> Thanks,
> >> Boyang
> >>
> >> On Tue, Jun 23, 2020 at 10:41 AM Ismael Juma wrote:
> >>
> >>> Option A is basi
points as well in the KIP, let me know
> > >> if you have further questions.
> > >>
> > >> Thanks,
> > >> Boyang
> > >>
> > >> On Tue, Jun 23, 2020 at 10:41 AM Ismael Juma wrote:
> > >>
t that these requests are "upserts" (update if there,
> otherwise insert). It was referred to as an "Addition" before, which I
> felt was not technically correct. If you diff the most recent two versions
> of the KIP it diffs pretty cleanly and makes the changes prett
[
https://issues.apache.org/jira/browse/KAFKA-10174?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-10174.
--
Fix Version/s: 2.7
Reviewer: Colin McCabe
Resolution: Fixed
> Pre
ic class UserScramCredentialAddition, and
> public class UserScramCredentialDeletion
>
Yeah, there is a bit of a mismatch between "credentials" and "users." Really,
these APIs are about credentials, not users. So I agree -- let's rename it.
best,
Colin
>
&
Hi Ying,
Thanks for the response.
It sounds like you agree that storing the metadata in the remote storage would
be a better design overall. Given that that's true, is there any reason to
include the worse implementation based on RocksDB?
Choosing a long-term metadata store is not something t
re of a nit, but should `DeleteBrokerRecord` be
> > `ShutdownBrokerRecord`? The broker is just getting removed from ISRs, but
> > it would still be present in the replica set (I assume).
> >
> > Thanks,
> > Jason
> >
> > On Sun, Jul 12, 2020 at 12:24 AM Coli
micromanaging snapshots
> > > on all followers. Alternatives include fuzzy snapshots which can be done
> > > concurrently. If this has been rejected, can you add some detail about
> > why?
> > > 4. More of a nit, but should `DeleteBrokerRecord` be
> > > `Shutdow
Hi all,
Thanks, everyone, for reviewing.
Since we made a few changes to the RPCs in the last few days, I'm going to
extend the vote until Monday and close it out then if it looks good.
best,
Colin
On Wed, Jul 15, 2020, at 14:47, Colin McCabe wrote:
> On Tue, Jul 14, 2020, at 16
ds?
>
Good call. I added this.
best,
Colin
>
> I haven't looked at the AlterScramUsers stuff yet; I'll look at that in
> detail tomorrow.
>
> Ron
>
>
> On Tue, Jul 14, 2020 at 4:11 PM Colin McCabe wrote:
>
> > On Tue, Jul 14, 2020, at 07:57, Ron Da
I missing/misunderstanding something? Thoughts?
> >
> > Also, separately, should the responses include a ThrottleTimeMs field? I
> > believe so but would like to confirm.
> >
> > Ron
> >
> > On Mon, Jul 13, 2020 at 3:44 PM David Arthur wrote:
> >
>
lterScramUsersRequest",
> Other than that, +1 (binding) from me.
>
>
> On Mon, Jul 13, 2020 at 8:38 AM Colin McCabe wrote:
>
> > Hi David,
> >
> > The API is for clients. Brokers will still listen to ZooKeeper to load
> > the SCRAM information.
>
ssword info through the RPC, how will brokers
> load this info? (I'm presuming that they need it to configure
> authentication)
>
> -David
>
> On Mon, Jul 13, 2020 at 10:57 AM Colin McCabe wrote:
>
> > On Fri, Jul 10, 2020, at 10:55, Boyang Chen wrote:
> > >
nt when we send the SCRAM requests correct?
>
Yes, the client generates the salt before sending the request.
best,
Colin
> Best,
> Boyang
>
> On Fri, Jul 10, 2020 at 8:10 AM Rajini Sivaram
> wrote:
>
> > +1 (binding)
> >
> > Thanks for the KIP, Coli
jira/browse/ZOOKEEPER-965>
>
> Thanks,
> Unmesh
>
> On Fri, Jul 10, 2020 at 11:32 AM Colin McCabe wrote:
>
> > Hi Unmesh,
> >
> > Yes, once the last stable offset advanced, we would consider the topic
> > creation to be done, and then we could return s
> PartitionRecord together always, and in no situation we can have
> > TopicRecord without PartitionRecord. Not sure if there are other situations
> > where multi-operation is needed.
> > <https://issues.apache.org/jira/browse/ZOOKEEPER-965>
> >
> > Thanks,
&
Hi all,
Thanks for the KIP.
I took a look and one thing that stood out to me is that the more metadata we
have, the more storage we will need on local disk for the rocksDB database.
This seems like it contradicts some of the goals of the project. Ideally the
space we need on local disk shoul
which is what most
tools care about anyway.
best,
Colin
On Thu, Jul 9, 2020, at 23:04, Colin McCabe wrote:
> Yeah. The issue with subclassing is that it's a source compatibility
> break, although not (I think) a binary compatibility break. I don't
> think it's really
lasses.
> >3. After a time, do the following:
> > 1. Change the return type of DescribeLogDirsResult#[all, values] from
> > old ones to the new ones, with a notice. Since we already deprecated
> > the
> > old ones, most users would already be
Previous HighWaterMark
>CreateTopic ||
>||
>||
>
>
>
>
>
>
>
>
>
>
>
> On Fri, Jul 10, 2020 at 1:30 AM Colin McCabe wrote:
>
> > On Thu, Jul 9, 2020, at 04:37, Unmesh Joshi wrote:
> > > I see that, when a new topic is created, two metadata re
committed to the Raft log.*" Should the three references above be
> > to
> > > the active controller rather than just the controller?
> > >
> > > "*Therefore, the controller must not make this future state "visible" to
> > > the re
[
https://issues.apache.org/jira/browse/KAFKA-10256?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-10256.
--
Resolution: Won't Fix
> Create a server gradle module for Java code needed only by
quot;
> > Why? Is it necessary to state this? It seems like it might be an
> > implementation detail rather than a necessary constraint/requirement that
> > we declare publicly and would have to abide by.
> >
> > "*It will reject brokers whose metadata is too stale*
Hi all,
I'd like to call a vote for KIP-554: Add a broker-side SCRAM configuration API.
The KIP is here: https://cwiki.apache.org/confluence/x/ihERCQ
The previous discussion thread is here:
https://lists.apache.org/thread.html/r69bdc65bdf58f5576944a551ff249d759073ecbf5daa441cff680ab0%40%3Cdev.k
Colin McCabe created KAFKA-10256:
Summary: Create a server gradle module for Java code needed only
by servers
Key: KAFKA-10256
URL: https://issues.apache.org/jira/browse/KAFKA-10256
Project: Kafka
;
There's a bit more context in KIP-500. Stale metadata is a general problem
since it can confuse clients-- they don't see a topic which actually does
exist, for example, or do see one that was deleted.
>
> "*it may lose subsequent conflicts if its broker epoch is stale*&
Hi Dongjin,
Hmm. I'm not sure I follow. How does deprecating
DescribeLogDirsResponse.LogDirInfo help here? The issue is not so much the
class, but the fact that it's exposed as a public API. So it seems appropriate
to deprecate the methods that return it, but not the class itself, since we'
Hi all,
I posted a KIP about how the quorum-based controller envisioned in KIP-500 will
work. Please take a look here: https://cwiki.apache.org/confluence/x/4RV4CQ
best,
Colin
Hi Tom,
Thanks for this. I think the tough part is probably the few messages that are
still using manual serialization, which can't be easily converted to using
this. So we will probably have to upgrade them to using automatic generation,
or accept a little inconsistency for a while until the
Thanks, Tom.
I tried to think of a better way to do this, but I think you're right that we
probably just need different methods.
+1 (binding).
best,
Colin
On Mon, Jul 6, 2020, at 01:14, Tom Bentley wrote:
> Hi,
>
> I'd like to start a vote on KIP-621 which is about deprecating methods in
> De
Hi Arvin,
Thanks for the KIP.
Unfortunately, I don't think this makes sense since it would increase the
amount of data we send back in the metadata response, which is pretty bad for
scalability. In general we probably want to avoid the case where we don't have
the appropriate number of in-syn
Hi Rajini,
OK. Let's remove the encrypted credentials from ListScramUsersResponse and the
associated API. I have updated the KIP-- take a look when you get a chance.
best,
Colin
On Fri, May 15, 2020, at 06:54, Rajini Sivaram wrote:
> Hi Colin,
>
> We have used different approaches for kafka
Ducktape runs on Python 2. You can't use it with Python 3, as you are trying
to do here.
If anyone's interested in porting it to Python 3 it would be a good change.
Otherwise, using docker as suggested here seems to be the best way to go.
best,
Colin
On Mon, Jun 29, 2020, at 02:14, Gokul Rama
+1 (binding).
Thanks, Sam.
best,
Colin
On Thu, Jun 25, 2020, at 18:05, Gwen Shapira wrote:
> +1 (binding)
>
> Thank you, Sam. It is great to see Trogdor getting the care it deserves.
>
> On Mon, Jun 22, 2020, 1:46 PM Sam Pal wrote:
>
> > Hi all,
> >
> > I would like to start a vote for KIP-
> > > On Fri, Jun 19, 2020 at 3:18 PM Ismael Juma wrote:
> > >
> > > > Hi Colin,
> > > >
> > > > The KIP states in the Compatibility section (not Future work):
> > > >
> > > > "To support the proxy of requests, we need to build a channel for
> > > > brokers to talk directly to the controller. This
Congratulations, Boyang!
cheers,
Colin
On Mon, Jun 22, 2020, at 16:26, Guozhang Wang wrote:
> The PMC for Apache Kafka has invited Boyang Chen as a committer and we are
> pleased to announce that he has accepted!
>
> Boyang has been active in the Kafka community more than two years ago.
> Since
On Tue, Jun 16, 2020, at 07:42, Tom Bentley wrote:
> Hi Colin,
>
> Thanks for taking a look at it. Replies inline...
>
> On Mon, Jun 15, 2020 at 6:22 PM Colin McCabe wrote:
>
> > Hi Tom,
> >
> > It's an interesting idea. Obviously protocol buf
; > On Thu, Jun 18, 2020 at 12:00 AM Jose Garcia Sancio <
> > > jsan...@confluent.io>
> > > > wrote:
> > > >
> > > > > +1.
> > > > >
> > > > > Thanks for the KIP and looking forward to the improvement
> > > imple
Thanks, Boyang! +1 (binding)
best,
Colin
On Mon, Jun 15, 2020, at 12:59, Boyang Chen wrote:
> Thanks for more feedback Colin! I have addressed them in the KIP.
>
> Boyang
>
> On Mon, Jun 15, 2020 at 11:29 AM Colin McCabe wrote:
>
> > On Fri, Jun 12, 2020, at
Hi Sam,
Thanks for the KIP.
Can you add some text clarifying whether a done task continues to be counted in
the created-task-count?
Looks good aside from that.
best,
Colin
On Wed, Jun 17, 2020, at 12:31, Sam Pal wrote:
> Hi all,
>
> I’d like to start a discussion about adding metrics to Tr
gt; of the key) to decide the partition.
> We noticed enqueuing and timeouts while several consumers were idle - which
> made us take a better look on how the load is balanced.
>
> I believe the only way to perform equal load balance without having to know
> other producers would be to do
On Fri, Jun 12, 2020, at 15:30, Boyang Chen wrote:
> Thanks Colin for the suggestions!
>
> On Fri, Jun 12, 2020 at 2:40 PM Colin McCabe wrote:
>
> > Hi Boyang,
> >
> > Thanks for the KIP! I think it's getting close.
> >
> > > For older reques
Hi Cheng,
The link from the main KIP page is an "edit link" meaning that it drops you
into the editor for the wiki page. I think the link you meant to use is a
"view link" that will just take you to view the page.
In general I'm not sure what I'm supposed to take away from the large UML
diagr
there is currently no way to do partition balancing on the
> broker - the producer sends messages directly to partition leaders so
> partition currently needs to be defined on the producer.
>
> I understand that in order to perform round robin across partitions of a
> topic when working w
Hi Tom,
It's an interesting idea. Obviously protocol buffers does this for all numeric
fields.
I have to admit I have some mixed feelings, since this is another thing that
makes encoding more complex. And it's not a clear win in all cases, although
it is in some.
I assume that the performan
Hi Boyang,
Thanks for the KIP! I think it's getting close.
> For older requests that need redirection, forwarding
> broker will just use its own authorizer to verify the principals. When the
> request looks good, it will just forward the request with its own
> credentials, no second valid
HI Vinicius,
This question seems like a better fit for the user mailing list rather than the
developer mailing list.
Anyway, if I understand correctly, you are asking if the producer can choose to
assign partitions in a round-robin fashion rather than based on the key. The
answer is, you can,
gt; >> We added a new quota name in the KIP. You chose not to bump up the version
> >> of DESCRIBE_CLIENT_QUOTAS and ALTER_CLIENT_QUOTAS, which seems ok since
> >> the
> >> quota name is represented as a string. However, the new quota name can be
> >>
should
do. So I think that the error code should actually be a little bit more
specific, or at least tell the end user what to do with it (that's why I
suggested "busy").
>
> Voilà. I hope that I have addressed all your questions/points and I am
> sorry for the lengthy emai
han QUOTA_VIOLATED, how about naming the error code BUSY? Then the
error text could indicate the quota that we violated. This would be more
generally useful as an error code and also avoid being confusingly similar to
POLICY_VIOLATION.
best,
Colin
>
> I will let David chime in more on that.
&g
Hi David,
Thanks for the KIP.
I thought about this for a while and I actually think this approach is not
quite right. The problem that I see here is that using an explicitly set quota
here requires careful tuning by the cluster operator. Even worse, this tuning
might be invalidated by change
Colin McCabe created KAFKA-10104:
Summary: Remove deprecated --zookeeper flags as specified in
KIP-604
Key: KAFKA-10104
URL: https://issues.apache.org/jira/browse/KAFKA-10104
Project: Kafka
Hi Cheng,
Thanks for working on this. Looks good.
How about "socket.connection.setup.timeout.ms" and
"socket.connection.setup.timeout.max.ms" (not connections with an S)?
+1 (binding)
best,
Colin
On Wed, Jun 3, 2020, at 06:24, Rajini Sivaram wrote:
> Hi Cheng,
>
> Thanks for the updates, l
[
https://issues.apache.org/jira/browse/KAFKA-9945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-9945.
-
Fix Version/s: 2.6.0
Reviewer: Colin McCabe
Resolution: Fixed
> TopicComm
Thanks, David. +1 (binding).
cheers,
Colin
On Wed, May 27, 2020, at 18:21, David Arthur wrote:
> Colin, thanks for the feedback. Good points. I've updated the KIP with your
> suggestions.
>
> -David
>
> On Wed, May 27, 2020 at 4:05 PM Colin McCabe wrote:
>
> &
Hi David,
Thanks for the KIP!
The KIP refers to "the KIP-500 bridge release (version 2.6.0 as of the time of
this proposal)". This is out of date-- the bridge release will be one of the
3.x releases. We should either update this to 3.0, or perhaps just take out
the reference to a specific v
Hi Gokul Ramanan Subramanian,
Thanks for the KIP.
Can you please modify the KIP to remove the reference to the deprecated
--zookeeper flag? This is not how kafka-configs.sh is supposed to be used in
new versions of Kafka. You get a warning message if you do use this deprecated
flag. As desc
[
https://issues.apache.org/jira/browse/KAFKA-9942?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-9942.
-
Fix Version/s: 2.5.1
2.6.0
Resolution: Fixed
> ConfigCommand fails
[
https://issues.apache.org/jira/browse/KAFKA-9980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-9980.
-
Fix Version/s: 2.5.1
Resolution: Fixed
> Fix bug where alterClientQuotas could not
0 at 5:41 PM Colin McCabe wrote:
>
> > On Tue, May 19, 2020, at 09:31, Jason Gustafson wrote:
> > > Hi Colin,
> > >
> > > Looks good. I just had one question. It sounds like your intent is to
> > > change kafka-configs.sh so that the --zookeeper flag i
t; >
> > > > +1 (non-binding).
> > > >
> > > > Thanks for the KIP.
> > > >
> > > > On Fri, May 15, 2020 at 12:41 AM Guozhang Wang
> > wrote:
> > > >
> > > > > +1.
> > > > >
>
we're trying to connect to a single fixed node, like the
controller node)
best,
Colin
>
> Regards,
>
> Rajini
>
>
> On Mon, May 18, 2020 at 8:48 PM Colin McCabe wrote:
>
> > Hi Rajini,
> >
> > I think the idea behind the 10 second default
NIO doesn't dictate how long we should wait before terminating the attempt to
connect. Hope I didn't miss anything.
best,
Colin
>
> Best, - Cheng Tan
>
>
>
> > On May 18, 2020, at 1:32 PM, Colin McCabe wrote:
> >
> > Hi Cheng,
> >
> &
[
https://issues.apache.org/jira/browse/KAFKA-9292?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-9292.
-
Fix Version/s: 2.6.0
Resolution: Fixed
> KIP-551: Expose disk read and write metr
Hi Cheng,
socket.connection.setup.timeout.ms seems more consistent with our existing
configuration names than socket.connections.setup.timeout.ms (with an s). What
do you think?
> If no connected or connecting node exists, provide the disconnected node which
> respects the reconnect backoff wi
Hi Rajini,
I think the idea behind the 10 second default is that if you have three Kafka
nodes A, B, C (or whatever), and you can't talk to A within 10 seconds, you'll
try again with B or C, and still have plenty of time left over. Whereas
currently, if your connection hangs while trying to co
Hmm. It would be good to figure out if we are going to remove this
compatibility hack in the next major release of Kafka? In other words, in
Kafka 3.0, will we enable TLS 1.3 by default even if the cipher suite is
specified?
best,
Colin
On Mon, May 18, 2020, at 09:26, Ismael Juma wrote:
> S
Hi Zhiguo,
I think it's really important to distinguish between "a class being public" and
"a class being part of Kafka's public API." These really are two completely
different concepts, unfortunately.
The best explanation of interface annotations is probably here:
https://github.com/apache/k
KafkaMetrics isn't a public API currently. Are we willing to make it one? I
think this is a very big change, if so.
This affects a huge number of classes. MetricConfig, MetricReporter,
MetricName, Sensor, KafkaMetric, and probably more I'm forgetting would need to
become public APIs that we
Ej/49+2yk0803y67WSXMYkgh77k=,iterations=4096]
>
>
> Please let me know what you think.
>
> Best, - Cheng Tan
>
> > On Apr 30, 2020, at 11:16 PM, Colin McCabe wrote:
> >
> >
>
>
-SHA-512=[salt=djR5dXdtZGNqamVpeml6NGhiZmMwY3hrbg==,stored_key=sb5jkqStV9RwPVTGxG1ZJHxF89bqjsD1jT4SFDK4An2goSnWpbNdY0nkq0fNV8xFcZqb7MVMJ1tyEgif5OXKDQ==,
> > server_key=3EfuHB4LPOcjDH0O5AysSSPiLskQfM5K9+mOzGmkixasmWEGJWZv7svtgkP+acO2Q9ms9WQQ9EndAJCvKHmjjg==,iterations=4096],SCRAM-SHA-256=[salt=10i
[
https://issues.apache.org/jira/browse/KAFKA-9688?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-9688.
-
Fix Version/s: 2.5.0
Reviewer: Colin McCabe
Resolution: Fixed
> kafka-topic
Hi all,
I'd like to start a vote on KIP-604: Remove ZooKeeper Flags from the
Administrative Tools.
As a reminder, this KIP is for the next major release of Kafka, the 3.0
release. So it won't go into the upcoming 2.6 release. It's a pretty small
change that just removes the --zookeeper flag
[
https://issues.apache.org/jira/browse/KAFKA-8820?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe resolved KAFKA-8820.
-
Fix Version/s: 2.5.0
Resolution: Fixed
> kafka-reassign-partitions.sh should support
;s going to have catastrophic bugs for
> > > years to come? It seems like such an act of hubris to me, as well as a
> > > massive waste of engineering effort. What is there to be gained?
> > >
> > >> On Fri, May 8, 2020 at 4:11 PM Matthias J. Sax
> > w
a new
> > > > distributed-consensus system that's going to have catastrophic bugs for
> > > > years to come? It seems like such an act of hubris to me, as well as a
> > > > massive waste of engineering effort. What is there to be gained?
> > > >
and then remove
> in major releases, and since 2.0 we've accumulated quite a few deprecated
> APIs, and I can compile a list of KIPs that contain those if people are
> interested.
>
>
> Guozhang
>
>
> On Thu, May 7, 2020 at 3:53 PM Colin McCabe wrote:
>
> > On
dependency here.
best,
Colin
On Thu, May 7, 2020, at 05:35, Jakub Scholz wrote:
> Hi Colin,
>
> Could you clarify how this fits with KIP-506 which seems to deal with the
> same?
>
> Thanks & Regards
> Jakub
>
> On Fri, May 1, 2020 at 8:18 AM Colin McCabe wrote:
&g
nged the API so that ScramMechanism is the enum,
ScramMechanismInfo is the enum + the number of iterations, and ScramCredential
is the enum, iterations, salt, and password data.
I also changed the salt and password to be bytes fields instead of strings to
reflect the fact that they are binary
er 3.0. I
think that's OK and in keeping with how we've handled deprecation and removal
in the past. It's important for users to have a smooth upgrade path.
best,
Colin
>
> Ryanne
>
> -
>
> On Wed, May 6, 2020 at 10:52 PM Colin McCabe wrote:
>
> &g
, dropping the zookeeper flags is a step forward for security and
encapsulation which also advances KIP-500. Another example is that removing
the kafka-preferred-replica-election.sh command removes a duplicate command
that has been deprecated for a while.
best,
Colin
>
> On Wed, May
own separate KIP, and should also apply to the other stuff you can
do with kafka-configs.sh (SSL passwords, etc.)
best,
Colin
>
> Kind regards,
>
> Tom
>
> Cheers,
>
> Tom
>
> On Tue, May 5, 2020 at 12:52 AM Guozhang Wang wrote:
>
> > Cool, that makes s
701 - 800 of 1666 matches
Mail list logo