ation mechanism where we
calculate lengths first and then write out bytes. If we do that, we can avoid
having any restrictions on tagged fields vs. regular fields. I will take a
look at how complex this would be.
best,
Colin
On Sun, Aug 18, 2019, at 22:27, Satish Duggana wrote:
> Please read str
On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> Thanks for the KIP, Colin. This looks great!
>
> I really like the idea of separating the Controller and Broker JVMs.
>
> As you alluded to above, it might be nice to have a separate
> broker-registration API to avoid overlo
Hi all,
The KIP has been out for a while, so I'm thinking about calling a vote some
time this week.
best,
Colin
On Mon, Aug 19, 2019, at 15:52, Colin McCabe wrote:
> On Mon, Aug 19, 2019, at 12:52, David Arthur wrote:
> > Thanks for the KIP, Colin. This looks great!
> >
&
On Wed, Aug 21, 2019, at 06:38, Eno Thereska wrote:
> Hi Colin,
>
> Nice KIP! For such a big change it would be good to add a pointer or
> two to related work that provides some sort of soft proof that the
> approach taken makes sense. Also such work often builds on other work
&g
On Wed, Aug 21, 2019, at 04:22, Ron Dagostino wrote:
> Hi Colin. I like the concept of a "bridge release" for migrating off of
> Zookeeper, but I worry that it may become a bottleneck if people hesitate
> to replace Zookeeper -- they would be unable to adopt newer versions of
&
On Tue, Aug 20, 2019, at 07:19, Ron Dagostino wrote:
> Hi Colin. The diagram up at the top confused me -- specifically, the lines
> connecting the controller/active-controller to the brokers. I had assumed
> the arrows on those lines represented the direction of data flow, but that
>
currently. The tradeoff is that
Raft doesn't support redundancy with fewer than 3 replicas. But that is a
tradeoff that is appropriate to make for many applications.
best,
Colin
On Wed, Aug 21, 2019, at 12:19, Ryanne Dolan wrote:
> Colin, have you considered leveraging Apache Ratis (incubatin
to do in a compatible way.
cheers,
Colin
On Wed, Aug 21, 2019, at 12:05, Adam Bellemare wrote:
> +users mailing list
>
> David,
>
> I don't think I really understand your email. Are you saying that this can
> already be achieved only using the READ ACL?
>
> Thank
On Wed, Aug 21, 2019, at 19:48, Ron Dagostino wrote:
> Thanks, Colin. The changes you made to the KIP related to the bridge
> release help make it clearer. I still have some confusion about the phrase
> "The rolling upgrade from the bridge release will take several steps."
&
On Fri, Aug 23, 2019, at 06:24, Ryanne Dolan wrote:
> Colin, can you outline what specifically would be in scope for this KIP vs
> deferred to the follow-on KIPs you've mentioned? Maybe a Future Work
> section? Is the idea to get to the bridge release with this KIP, and then
&
+1 (binding).
cheers,
Colin
On Tue, Aug 20, 2019, at 10:55, Jason Gustafson wrote:
> Hi All,
>
> I'd like to start a vote on KIP-352, which is a follow-up to KIP-455 to
> fix
> a long-known shortcoming of URP reporting and to improve reassignment
> monitoring:
>
URPs based on
that. +1 for this. (I assume Jason will update the KIP...)
best,
Colin
>
> Taking that into account, +1 from me! (non-binding)
>
> On Fri, Aug 23, 2019 at 7:00 PM Colin McCabe wrote:
>
> > +1 (binding).
> >
> > cheers,
> > Colin
> >
Hi Ryanne,
Good point. I added a section titled "future work" with information about the
follow-on KIPs that we discussed here.
best,
Colin
On Fri, Aug 23, 2019, at 13:15, Ryanne Dolan wrote:
> Thanks Colin, sgtm. Please make this clear in the KIP -- otherwise it is
> hard t
On Mon, Aug 26, 2019, at 05:23, Magnus Edenhill wrote:
> Great KIP as always, Colin!
>
> Some comments:
>
> > If the flexible versions are not specified, it is assumed that all
> versions are flexible.
>
> This is ambiguous, if a protocol-generator is pointed t
Hi all,
After some discussion with Jun and Stan, we decided that we should bump the
version of the topics znode from 1 to 2. The bump is backwards compatible
(older brokers can read the v2 znode). I have updated the KIP.
best,
Colin
On Thu, Aug 8, 2019, at 11:09, Colin McCabe wrote:
>
ied to set up client-side hacks
based on the broker version, and were only stopped by the fact that we don't
expose this information. I agree with Jay that we should think very carefully
before exposing it. In any case, this seems out of scope...
best,
Colin
>
> /Magnus
>
>
Hi Pere,
Thanks for contributing. KIP-500 will not be implemented in 2.4.
KAFKA-8843 seems like a good improvement. Since it involves changing
command-line arguments, I think a KIP will be needed. However, it should be a
relatively short one if the change is straightforward.
best,
Colin
>
t needs a REST admin API. There will probably be a
lot of other stuff that we'll want to add to it.
best,
Colin
>
> Thanks,
> Jason
>
>
>
>
> On Fri, Aug 23, 2019 at 5:38 PM Arjun Satish wrote:
>
> > Jason,
> >
> > Thanks for your comm
As Gwen commented earlier, the client already has the record that it sent,
including all the headers.
>
> Future future = producer.send(myRecord, null);
> future.get();
> System.out.println("I sent myRecord with headers " + myRecord.headers());
>
best,
Colin
On Tu
accepted doesn't
mean it will be part of 2.4.0, though-- it also has to be implemented.) Hope
this helps.
cheers,
Colin
On Fri, Aug 30, 2019, at 07:50, Pere Urbón Bayes wrote:
> Hi,
> quick question, I saw in another mail that 2.4 release is planned for
> September. I think it w
+1 for making SslEngineBuilder configurable. This would give implementers a
lot more flexibility-- to use key distribution methods that were not files, for
example.
best,
Colin
On Fri, Aug 30, 2019, at 02:03, Rajini Sivaram wrote:
> Just to make sure we are on the same page - KIP-383
I think we could reuse KIP-383 for a KIP making SSLEngineBuilder pluggable.
However, we need to be able to change this code in the future. That means it
needs to be a very minimal interface. Ideally just the createSslEngine and
shouldBeRebuilt methods.
best,
Colin
On Fri, Aug 30, 2019, at
On Mon, Sep 2, 2019, at 07:51, Ron Dagostino wrote:
> Hi Colin. It is not unusual for customers to wait before upgrading —
> to avoid so-called “point-zero” releases — to avoid as many of the
> inevitable bugs that ride along with new functionality as possible.
> Removal of Zookee
take up that work for making it configurable.
> > >
> > > Thanks
> > > Maulin
> > >
> > >
> > >
> > >
> > >
> > > On Fri, Aug 30, 2019 at 10:34 AM Maulin Vasavada <
> > > maulin.vasav...@gmail.com>
> &
.html/cdc801ae886491b73ef7efecac7ef81b24382f8b6b025899ee343f7a@%3Cdev.kafka.apache.org%3E
best,
Colin
that implies background
thread(s) at work somewhere. We should specify when that these thread(s) are
created, and when they are stopped. Probably they should be created in start,
and stopped in close? We should update the JavaDoc.
best,
Colin
On Tue, Sep 3, 2019, at 05:50, Rajini Sivaram
ApiVersionsRequest exceeds the broker's.
best,
Colin
On Tue, Sep 3, 2019, at 01:26, David Jacot wrote:
> Hi all,
>
> I have updated the KIP to address the various comments. I have also added a
> section about the handling of the ApiVersionsRequest/Response in the broker.
>
>
gurable params the
> SslFactory should try to reconfigure the SSLEngine.
Each SslEngineBuilder should define its own set of reconfigurable
configurations. We don't know ahead of time what they will need. We want to
be flexible. People might want to fetch certificates from a central loca
On Wed, Sep 4, 2019, at 13:01, Jason Gustafson wrote:
> Hi Colin,
>
> Just a couple questions.
>
> 1. I think we discussed that we would do a lazy version bump of all
> protocols in order to get flexible version support. Can you add that to the
> KIP?
Good point. Added.
e sync for now we would need a
longer discussion about how to manage and monitor the authorization purgatory
to make this async, and it's not clear that we need it (yet?)
best,
Colin
On Fri, Sep 6, 2019, at 11:32, Rajini Sivaram wrote:
> I would suggest that we go with synchronous au
easier to
maintain than a separate set of metadata about blacklists.
best,
Colin
On Fri, Sep 6, 2019, at 11:58, George Li wrote:
> Hi,
>
> Just want to ping and bubble up the discussion of KIP-491.
>
> On a large scale of Kafka clusters with thousands of brokers in many
> c
pasting in order to avoid creating an API
that would constrain the project in the future. Hopefully, we can come up with
something good here that will be useful to everyone.
best,
Colin
On Fri, Sep 6, 2019, at 07:48, Pellerin, Clement wrote:
> This is the way I see it, which is in no way author
Sorry, I meant to write "removing an API typically requires a new major release
of Kafka". Deprecating an API can be done in a minor release.
regards,
Colin
On Fri, Sep 6, 2019, at 14:49, Colin McCabe wrote:
> Hi Maulin,
>
> Clement brought up a good point, which is
,
Colin
On Fri, Sep 6, 2019, at 13:36, Jason Gustafson wrote:
> +1 Thanks Colin. This is really going to help with compatibility.
>
> -Jason
>
> On Wed, Sep 4, 2019 at 1:34 PM Colin McCabe wrote:
>
> > On Wed, Sep 4, 2019, at 13:01, Jason Gustafson wrote:
> > > Hi
the broker to know exactly how long the
supplied request header is. (This assumes that the first 4 fields of the
request header will never change, which seems like a reasonable assumption...)
best,
Colin
On Tue, Sep 3, 2019, at 23:54, David Jacot wrote:
> Hi Colin,
>
> Than
On Sat, Sep 7, 2019, at 09:21, Harsha Chintalapani wrote:
> Hi Colin,
> Can you give us more details on why you don't want this to be
> part of the Kafka core. You are proposing KIP-500 which will take away
> zookeeper and writing this interim tools to change the zook
here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum
regards,
Colin
both cases, you have to
handle connections that set the version later than others, or don't set the
version at all (for compatibility). So the version/type has to be mutable and
added after the TCP connection itself is established.
best,
Colin
On Mon, Sep 9, 2019, at 06:11, David Jacot w
Hi Ash,
At first guess, you probably had a problem with your log cleaner thread, which
resulted in the offsets log not being cleaned. Check if that thread is running.
best,
Colin
On Wed, Sep 11, 2019, at 09:52, Ash G wrote:
> Bump, no reply,
>
> It seems this condition was misse
Hi Radai,
Thanks for the KIP. Sounds interesting. I assume that if an
InterruptedException were caught, that would be rethrown, rather than returning
false? It might be good to specify that. Can you give an example of how this
would be used?
best,
Colin
On Thu, Sep 12, 2019, at 15:26
lude UUID to avoid issues where
requests are made to stale partitions. However, adding a UUID to
MetadataRequest is listed as future work, out of scope for this KIP. How will
the client learn what the topic UUID is, if the metadata response doesn't
include that information? It seems l
Hi David,
Sounds good.
best,
Colin
On Fri, Sep 13, 2019, at 04:45, David Jacot wrote:
> Hi all,
>
> I would like to do another modification to the proposal. In the proposal,
> the OffsetDeleteResponse
> doesn't have a top level error field so I would like to add one. Many
Hi all,
With 3 binding +1 votes from Gwen Shapira, Ismael Juma, and Bill Bejeck and 5
non-binding +1 votes from Tom Bentley, Ron Dagostino, David Jacot, Dhruvil
Shah, Mickael Maison, the vote passes.
Thanks, everyone!
best,
Colin
On Fri, Sep 13, 2019, at 01:14, Mickael Maison wrote:
>
ese strings in metrics names. Probably we want something like:
[\.\-A-Za-z0-9]?[\.\-A-Za-z0-9 ]*[\.\-A-Za-z0-9]?
Notice this does _not* include underscores, since they get converted to dots in
JMX, causing ambiguity. It also doesn't allow the first or last character to
be a space.
best,
C
describe, --delete,
> --reset-offsets
Did you mean to add a new action here that was offsets-related?
best,
Colin
>
> What do you think?
>
> Best,
> David
>
>
> On Fri, Sep 13, 2019 at 6:42 PM Colin McCabe wrote:
>
> > Hi David,
> >
> > Sounds
think there was some concern that having a public API that returned API
version information would encourage people to write code that only worked
against specific broker versions. I remember Ismael and Jay expressed this
concern, though I can't find the email threads now...
best,
Colin
O
That's fair. We could use the existing error code in the response and pass
back something like INVALID_REQUEST.
I'm not sure if we want to add an error string field just for this (although
they're a good idea in general...)
best,
Colin
On Wed, Sep 18, 2019, at 12:31, Magnus
onfig(String topic);
> public KafkaFuture numPartitions(String topic);
> public KafkaFuture replicationFactor(String topic);
Or is it better to have the "public KafkaFuture topicConfig(String
topic)" method?
best,
Colin
On Tue, Sep 17, 2019, at 02:12, Rajini Sivaram wrote:
> Hi all,
On Thu, Sep 19, 2019, at 06:31, Rajini Sivaram wrote:
> Hi Colin,
>
> Thanks for reviewing the KIP!
>
> I have added default values for the RPC. Since other int fields seem to be
> using -1, I used -1 as the default for NumPartitions and ReplicationFactor.
>
Thanks.
>
On Wed, Sep 18, 2019, at 23:44, David Jacot wrote:
> Hi Colin,
>
> Thank you for your feedback! Please find my comments/answers below:
>
> *> Nitpick: in the intro paragraph, "Operators of Apache Kafka clusters
> have literally no information about the clients con
+1 for starting the vote. Thanks, Rajini!
C.
On Thu, Sep 19, 2019, at 10:47, Rajini Sivaram wrote:
> Thanks, Colin! That makes sense. I have updated the KIP to use separate
> methods. Can you do take a quick look to see if the KIP is ready for voting?
>
> Thank you,
>
>
Sounds good to me. It makes sense to add this functionality to the command
line.
best,
Colin
On Wed, Sep 18, 2019, at 11:26, David Jacot wrote:
> Indeed, I have forgotten to add the action. There will be a new action «
> —delete-offsets ». Sorry!
>
> *Proposed API*
> kafka-con
et it into shape for what we need.
best,
Colin
On Fri, Sep 20, 2019, at 09:18, Nikolay Izhikov wrote:
> Hello, all.
>
> Any feedback on this?
> Do we need support of ByteBuffer in RPC generated code?
>
> Which PR should be reviwed and merged?
>
> В Чт, 19/09/2019 в
o actually throttle only reassignment traffic, not
just any non-ISR traffic as it does now. Or add a different quota type?
best,
Colin
On Fri, Sep 20, 2019, at 04:38, Tom Bentley wrote:
> Hi,
>
> I was wondering, what is the current status of efforts to add an
> AdminClient API fo
onses based on the length of the response. We should add
this detail to the KIP.
+1 (binding) after that change.
cheers,
Colin
>
> > Agreed. This is a good use-case for INVALID_REQUEST. We should add a
> comment that this is now a valid error.
>
> I have documented the erro
+1 (binding). Thanks, Rajini.
best,
Colin
On Fri, Sep 20, 2019, at 00:43, Rajini Sivaram wrote:
> Hi all,
>
> I would like to start vote on KIP-525 to return configs in CreateTopics
> response. This is a minor KIP that returns additional data in the response
> without breakin
as
being a new AlterConfigs version (although this wasn't done, for reasons which
were good, in my opinion.)
Considering how likely this API is to be misused, I think we should avoid
adding this for now.
best,
Colin
On Fri, Sep 20, 2019, at 10:26, Mickael Maison wrote:
> Thank
On Fri, Sep 20, 2019, at 18:05, Jun Rao wrote:
> Hi, Colin,
>
> Thanks for the KIP. Overall, looks good to me too. A couple of minor
> comments.
>
> 100. Does the tag number need to be 31-bit? It seems that 15-bit could be
> enough.
I think 15 bit would probably be enough,
Looks good. +1 (binding)
best,
Colin
On Tue, Sep 24, 2019, at 09:42, Jason Gustafson wrote:
> Hi All,
>
> I'm starting a vote for KIP-524, which is a small change to the config
> tool:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-524%3A+Allow+users+to+choos
On Sat, Sep 28, 2019, at 17:49, Magnus Edenhill wrote:
> Den mån 23 sep. 2019 kl 14:42 skrev Colin McCabe :
>
> > On Fri, Sep 20, 2019, at 18:05, Jun Rao wrote:
> > > 101. We already use varInt in the message format. I assume that the
> > > protocol uses the same var
This would definitely be a good project for someone to tackle to
improve the reassignment experience.
best,
Colin
On Fri, Sep 27, 2019, at 02:13, Tom Bentley wrote:
> Hi Viktor and Colin,
>
> Thanks for the update. Viktor, if you publish your KIP after summit then we
> can at least
+1. Thanks, Viktor.
Colin
On Thu, Oct 10, 2019, at 03:30, Viktor Somogyi-Vass wrote:
> Hi All,
>
> During the code review it came up that we shouldn't count replication bytes
> together with reassignment bytes so they count to a different metrics. This
> is a change
Hi Ismael,
I think every replica is doing replication, by definition. But not every
replica is undergoing reassignment.
If the broker that died was in the set of new replicas being added, its death
will not add a new under-replicated partition. Otherwise, it will add a new
URP.
best,
Colin
Hi all,
I wrote a short KIP about adding a metric to track the number of open
connections with a given SSL cipher type. Take a look here:
https://cwiki.apache.org/confluence/x/txPjBw
best,
Colin
m the merge, when you do it the other way.
best,
Colin
On Thu, Oct 17, 2019, at 02:59, Nikolay Izhikov wrote:
> Hello.
>
> Is there something wrong with the PR?
> Do we need this ticket to be done? [2]
> If no, let's close both PR [1] and ticket.
>
> The design or impl
Hi Radai,
It seems reasonable to me.
best,
Colin
On Mon, Oct 21, 2019, at 09:52, radai wrote:
> yet another bump.
>
> can we please have a vote if there are no objections ?
>
> On Wed, Sep 25, 2019 at 1:28 PM radai wrote:
> >
> > bump.
> >
> > so if
hash type, username, etc.
since the caller already knows what they passed for those in the request. Not
repeating yourself is the most efficient encoding.
I guess the other question is, do we want an API to show what the password is
for an existing user? It seems undeniably useful. And i
bering. We actually
only have it for reconnecting.
To address this, I think we should add exponential backoff here, like we did
with reconnect backoff. We could have a similar retry.backoff.max.ms to set
the upper limit for backoff.
best,
Colin
On Wed, Oct 9, 2019, at 12:06, Jason Gusta
Hi all,
I wrote a KIP about creating a fetch.max.bytes configuration for the broker.
Please take a look here: https://cwiki.apache.org/confluence/x/4g73Bw
thanks,
Colin
5e6eb@%3Cdev.kafka.apache.org%3E
best,
Colin
ber of replicas that was created, in
case the client is interested in this information. Returning an error code
would certainly cause problems for a lot of users, who use all().get() to
verify that all the topics have been successfully created.
best,
Colin
On Mon, Oct 21, 2019, at 09:50, Mic
On Mon, Oct 21, 2019, at 15:52, M. Manna wrote:
> Hello Colin,
>
> The KIP looks concise. My comments are below.
>
> replica.fetch.max.bytes is relevant when there is replication involved, so
> I am trying to understand how fetch.max.bytes for a broker will play a role
> he
+1 (binding).
Thanks, Jason.
best,
Colin
On Mon, Oct 21, 2019, at 17:27, Jason Gustafson wrote:
> I'd like to start a vote for KIP-537:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-537%3A+Increase+default+zookeeper+session+timeout
> .
>
> +1 from me
>
> Thanks,
> Jason
>
27;ll need a
follow-on KIP to do them.
best,
Colin
On Tue, Oct 22, 2019, at 12:42, Brian Byrne wrote:
> Hello all,
>
> I wrote a KIP about expanding the ConfigCommand's functionality when not
> accessing ZooKeeper directly, i.e. when --bootstrap-servers is specified
> ins
I'll add my +1 (binding) as well
best,
Colin
On Tue, Oct 22, 2019, at 15:47, M. Manna wrote:
> +1 (non-bonding) it’s a helpful info.
>
> Thanks for the KIP.
>
> Regards,
>
> On Tue, 22 Oct 2019 at 23:45, Jason Gustafson wrote:
>
> > +1 Thanks Colin.
>
+ dev@kafka.apache.org
On Tue, Oct 22, 2019, at 15:48, Colin McCabe wrote:
> +1. I ran the broker, producer, consumer, etc.
>
> best,
> Colin
>
> On Tue, Oct 22, 2019, at 13:32, Guozhang Wang wrote:
> > +1. I've ran the quick start and unit tests.
> >
> &
Hi Ismael,
That's a good idea! I added it to the KIP.
best,
Colin
On Wed, Oct 23, 2019, at 12:59, Ismael Juma wrote:
> Hi Colin,
>
> Thanks for the KIP. Would it make sense to have a tag with the TLS version
> as well?
>
> Ismael
>
> On Mon, Oct 21, 2019 a
Hi all,
With 5 binding +1 votes from Gwen Shapira, Jason Gustafson, Colin McCabe,
Ismael Juma, Rajini Sivaram, and 2 non-binding +1 votes from Tom Bentley and M.
Manna, the vote passes. Thanks, all.
best,
Colin
On Thu, Oct 24, 2019, at 08:32, Rajini Sivaram wrote:
> +1 (binding)
>
&g
On Tue, Oct 22, 2019, at 01:10, Stanislav Kozlovski wrote:
> Thanks for the KIP, Colin! This is a good idea, I think it makes total
> sense.
>
> We also have the max-size tunables on the producer-side (max.request.size).
> Do we have a broker equivalent denoting the maximum si
Hi Ismael,
55 MB seems reasonable. I'll change it to that.
best,
Colin
On Tue, Oct 22, 2019, at 07:22, Ismael Juma wrote:
> Hi Colin,
>
> Thanks for the KIP. This is definitely needed. Have we considered having a
> lower max limit by default? 75 MB is pretty high. Did we c
rg%3E
cheers,
Colin
Hi Vikto,
That's an interesting idea. However, I think it's better to consider it
outside the context of this KIP. I think one-letter abbreviations would be
controversial, and aren't really related to fixing the ZK dependency here.
+1 (binding). Thanks, Brian.
best,
Colin
Sorry, that should read "Hi Viktor",-C.
On Thu, Oct 31, 2019, at 14:08, Colin McCabe wrote:
> Hi Vikto,
>
> That's an interesting idea. However, I think it's better to consider
> it outside the context of this KIP. I think one-letter abbreviations
>
I will add my +1, binding
best,
Colin
On Fri, Nov 1, 2019, at 01:57, Stanislav Kozlovski wrote:
> +1 (non-binding).
> Thanks!
> Stanislav
>
> On Fri, Oct 25, 2019 at 2:29 PM David Arthur wrote:
>
> > +1 binding, this will be a nice improvement. Thanks, Colin!
>
Hi all,
With binding +1 votes from Ismael Juma, David Arthur, and Colin McCabe, and
non-binding +1 votes from Tom Bentley and Stanislav Kozlovski, the vote passes.
thanks, all.
Colin
On Fri, Nov 1, 2019, at 09:41, Colin McCabe wrote:
> I will add my +1, binding
>
> best,
> Colin
Hi Brian,
Thanks for the KIP. +1 (binding).
best,
Colin
On Thu, Nov 7, 2019, at 08:41, Brian Byrne wrote:
> Hello all,
>
> Ping. Any further votes or opinions?
>
> Thanks,
> Brian
>
> On Tue, Oct 29, 2019 at 9:39 AM Brian Byrne wrote:
>
> > Hello all,
&
ould spread the load across time
rather than creating a spike every 5 minutes in this case.
best,
Colin
On Fri, Nov 8, 2019, at 08:59, Ismael Juma wrote:
> I think this KIP affects when we block which is actually user visible
> behavior. Right?
>
> Ismael
>
> On Fri, Nov 8,
Hi Mickael,
Thanks for the revisions. It looks good overall.
best,
Colin
On Mon, Nov 11, 2019, at 03:45, Mickael Maison wrote:
> Hi all,
>
> If there's no more feedback, I'll open a vote in the next few days.
>
> Thanks
>
> On Wed, Nov 6, 2019 at 11
On Tue, Jun 9, 2020, at 05:06, David Jacot wrote:
> Hi Colin,
>
> Thank you for your feedback.
>
> Jun has summarized the situation pretty well. Thanks Jun! I would like to
> complement it with the following points:
>
> 1. Indeed, when the quota is exceeded, the brok
+1. Thanks, David!
best,
Colin
On Thu, Jun 11, 2020, at 23:51, David Jacot wrote:
> Colin, Jun,
>
> Do the proposed error code and the updated KIP look good to you guys? I’d
> like to wrap up and close the vote.
>
> Thanks,
> David
>
> Le mer. 10 juin 2020 à
, by using RoundRobinPartitioner. (again, if I'm
understanding the question correctly).
best,
Colin
On Tue, Jun 9, 2020, at 00:48, Vinicius Scheidegger wrote:
> Anyone?
>
> On Fri, Jun 5, 2020 at 2:42 PM Vinicius Scheidegger <
> vinicius.scheideg...@gmail.com> wrote:
>
proposal takes up 40% of the doc which is pretty
confusing for someone reading through. Let's also not forget that someone can
just read the old version by using the "page history" function on the wiki. So
there's no need to keep that all here.
best,
Colin
On Mon, Jun 8
length encoding and it worked well.
Before we do all this, though, one simpler improvement would be making all the
"error" fields into tagged fields. Most of them remain at 0 most of the time,
so this could very well provide a big savings without any big encoding changes.
best,
Colin
s designed to do. In mathematical terms, the
sum of several uniform random variables is itself uniformly random.
(There is a bug in RRP right now, KAFKA-9965, but it's not related to what
we're talking about now and we have a fix ready.)
cheers,
Colin
On Sun, Jun 14, 2020, at 14:26,
iguration internal = true right
now? Does it fail? If not, that seems like a potential problem.
Are people going to be able to create or delete topics named __consumer_offsets
or __transaction_state using this mechanism? If so, how does the security
model work for that?
best,
Colin
On Fri, Ma
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
different strategy. One
example would be using an external atomic counter to determine what partition
the producers should send the messages to. Another would be using a single
consumer with fanout. I think this is outside the scope of Kafka, at least if
I understand the problem here (?)
best,
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 m
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
eful if it's trivially bypassable by sending an unprivileged metadata
request that any client could send.
I suppose we could have a compatibility setting where if your broker has a
create topics policy set, it uses the old direct-to-zk path. And then we could
start defaulting that t
601 - 700 of 2167 matches
Mail list logo