Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-16 Thread Ben Stopford
+1 thanks Anna On Mon, Apr 16, 2018 at 5:37 PM Jason Gustafson wrote: > +1 Thanks for the KIP! > > On Sun, Apr 15, 2018 at 4:04 PM, Damian Guy wrote: > > > Thanks Anna +1 > > On Sun, 15 Apr 2018 at 15:40, Guozhang Wang wrote: > > > > > Anna, thanks for the KIP! +1 from me. > > > > > > Just one

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-11-10 Thread Ben Stopford
Sounds like a good middle ground to me. What do you think Steven? On Mon, Nov 6, 2017 at 8:18 PM Colin McCabe wrote: > It would definitely be nice to use the jdk8 CompletableFuture. I think > that's a bit of a separate discussion, though, since it has such heavy > compatibility implications. >

Re: [DISCUSS] KIP-58 - Make Log Compaction Point Configurable

2016-05-18 Thread Ben Stopford
Generally, this seems like a sensible proposal to me. Regarding (1): time and message count seem sensible. I can’t think of a specific use case for bytes but it seems like there could be one. Regarding (2): The setting log.cleaner.min.cleanable.ratio currently seems to have two uses. It con

Re: [DISCUSS] KIP-58 - Make Log Compaction Point Configurable

2016-05-19 Thread Ben Stopford
s the size of the compacted region). >>>> >>>> 5. Whoever wrote the docs should be shot :) >>>> >>>> so, in conclusion: >>>> In my mind, min.cleanable.dirty.ratio is terrible, it is misleading, >>>> difficult to understand, and I

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Ben Stopford
+1 (non-binding) > On 25 May 2016, at 14:07, Ismael Juma wrote: > > +1 (binding) > > I also think `log.cleaner.compaction.delay.ms` is clearer. As an aside, I > did notice that the topic level config for `log.segment.delete.delay.ms` > (mentioned by Ewen) is `file.delete.delay.ms`, which seems

Re: [DISCUSS] KAFKA-2063 Add possibility to bound fetch response size (was Re: [DISCUSS] Optimise memory used by replication process by using adaptive fetch message size)

2016-07-29 Thread Ben Stopford
Thanks for the kicking this one off Andrey. Generally it looks great! I left a comment on the Jira regarding whether we should remove the existing limitBytes, along with a potential alternative to doing randomisation. B > On 29 Jul 2016, at 09:17, Andrey L. Neporada wrote: > > Hi all! > > I

Re: [DISCUSS] KAFKA-2063 Add possibility to bound fetch response size (was Re: [DISCUSS] Optimise memory used by replication process by using adaptive fetch message size)

2016-07-29 Thread Ben Stopford
Andrey - I’m not sure we quite have consensus on the Randomisation vs Round Robin issue but it’s probably worth you just raising a kip and put one of the options as a rejected alternative. B > On 29 Jul 2016, at 11:59, Ben Stopford wrote: > > Thanks for the kicking this one o

[DISCUSS] KIP-73: Replication Quotas

2016-08-08 Thread Ben Stopford
We’ve created KIP-73: Replication Quotas The idea is to allow an admin to throttle moving replicas. Full details are here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-73+Replication+Quotas Please take a loo

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-09 Thread Ben Stopford
ion rate for those partitions returns to within the quota threshold. > > I have more questions on the proposal, but was more interested in the above > to see if it could simplify things a bit. > > Also, can you open up access to the google-doc that you link to? > > Thanks, >

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-10 Thread Ben Stopford
Regarding the fetch.partition.max.bytes, there was some discussion on the Jira around removing this setting completely. It’s probably not the easiest thing for user’s to set, so there is certainly an argument for removing it. This would have the side effect that a catching up broker would fill r

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-10 Thread Ben Stopford
h some refactoring) and will be internal to the broker. >>>>> >>>>> So I guess the question is if this alternative is simpler enough and >>>>> equally functional to not go with dedicated throttled replica >> fetchers. >>>>>

[DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-11 Thread Ben Stopford
+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation> All comments and suggestions greatly appreciated. Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/>

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Ben Stopford
suggest ditch LeaderGeneration. I’ll update the KIP. B Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/> > On 11 Dec 2016, at 22:30, Neha Narkhede wrote: > > Good to see this KIP being proposed. Back when I added the epoch to the > replication prot

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Ben Stopford
Thanks Onur. That’s correct, we no longer nee that extra mapping. I’ll update the KIP. B Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/> > On 11 Dec 2016, at 23:54, Onur Karaman wrote: > > Pretty happy to see a KIP tackling this problem! On

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Ben Stopford
the Jira comments: https://issues.apache.org/jira/browse/KAFKA-1120 <https://issues.apache.org/jira/browse/KAFKA-1120>. Strictly speaking this is a separate issue though and I’ve updated the KIP accordingly. B Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-20 Thread Ben Stopford
nd up with 2 > notions > > of representing a leader's generation. When we added the epoch, we wanted > > to add it to the log but we didn't. Now that we are adding the generation > > id to the log, I think we should revisit calling it the epoch at all. > Have > &g

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-21 Thread Ben Stopford
anks, Apurva On Sun, Dec 11, 2016 at 4:30 AM, Ben Stopford wrote: > Hi All > > Please find the below KIP which describes a proposed solution to a couple > of issues that have been observed with the replication protocol. > > In short, the proposal replaces the

[VOTE] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2017-01-03 Thread Ben Stopford
uence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation Thanks B On Wed, Dec 21, 2016 at 1:55 PM Ben Stopford wrote: > Thanks Apurva - yes that's one of those funny english phrases which are > often read one way, but are really

[DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Ben Stopford
+unclean.leader.election.enabled+from+True+to+False <https://cwiki.apache.org/confluence/display/KAFKA/[WIP]+KIP-106+-+Change+Default+unclean.leader.election.enabled+from+True+to+False> Thanks B Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/>

Re: [VOTE] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2017-01-04 Thread Ben Stopford
Closing this thread. On Tue, Jan 3, 2017 at 6:00 PM Ben Stopford wrote: > Hi All > > Happy New Year! > > Hopefully everyone has had an opportunity to review this KIP now, if they > wished to, so I'd like to call a vote. > > As a reminder the KIP proposes a change t

[VOTE] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2017-01-04 Thread Ben Stopford
Hi All Happy New Year! Hopefully everyone has had an opportunity to review this KIP now, if they wished to, so I'd like to call a vote. As a reminder the KIP proposes a change to the replication protocol to remove the potential for replicas to diverge. https://cwiki.apache.org/confluence/displa

[VOTE] Vote for KIP-101 - Leader Epochs

2017-01-04 Thread Ben Stopford
Hi All We’re having some problems with this thread being subsumed by the [Discuss] thread. Hopefully this one will appear distinct. If you see more than one, please use this one. KIP-101 should now be ready for a vote. As a reminder the KIP proposes a change to the replication protocol to rem

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-05 Thread Ben Stopford
/[partitionId]/state in ZK? > > 4. Since there are a few other KIPs involving message format too, it would > be useful to consider if we could combine the message format changes in the > same release. > > Thanks, > > Jun > > > On Wed, Jan 4, 2017 at 9:24 AM, Ben Stopfo

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-06 Thread Ben Stopford
t; 1) In OffsetForLeaderEpochResponse, start_offset probably should be > >> end_offset since it's the end offset of that epoch. > >> 3) That's fine. We can fix KAFKA-1120 separately. > >> > >> Jun > >> > >> > >> On Thu, Jan 5, 20

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-11 Thread Ben Stopford
So I believe we can mark this as Accepted. I've updated the KIP page. Thanks for the input everyone. On Fri, Jan 6, 2017 at 9:31 AM Ben Stopford wrote: > Thanks Joel. I'll fix up the pics to make them consistent on nomenclature. > > > B > > On Fri, Jan 6, 2017

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-11 Thread Ben Stopford
Sorry - my mistake. Looks like I still need one more binding vote. Is there a committer out there that could add their vote? B On Wed, Jan 11, 2017 at 6:44 PM Ben Stopford wrote: > So I believe we can mark this as Accepted. I've updated the KIP page. > Thanks for the input everyone.

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-11 Thread Ben Stopford
OK - my mistake was mistaken! There is consensus. This KIP has been accepted. On Wed, Jan 11, 2017 at 6:48 PM Ben Stopford wrote: > Sorry - my mistake. Looks like I still need one more binding vote. Is > there a committer out there that could add their vote? > > B > > On Wed,

[VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Ben Stopford
Looks like there was a good consensus on the discuss thread for KIP-106 so lets move to a vote. Please chime in if you would like to change the default for unclean.leader.election.enabled from true to false. https://cwiki.apache.org/confluence/display/KAFKA/%5BWIP%5D+KIP-106+-+Change+Default+uncl

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Ben Stopford
Congrats Grant!! On Wed, 11 Jan 2017 at 20:01, Ismael Juma wrote: > Congratulations Grant, well deserved. :) > > Ismael > > On 11 Jan 2017 7:51 pm, "Gwen Shapira" wrote: > > > The PMC for Apache Kafka has invited Grant Henke to join as a > > committer and we are pleased to announce that he has a

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Ben Stopford
Thanks all. We can consider this accepted. B On Wed, 11 Jan 2017 at 19:49, Apurva Mehta wrote: > +1 (non-binding) > > On Wed, Jan 11, 2017 at 11:45 AM, Gwen Shapira wrote: > > > +1 > > > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > > > Looks

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-01-25 Thread Ben Stopford
Is there more discussion to be had on this KIP, or should it be taken to a vote? On Mon, Jan 16, 2017 at 6:37 AM Dongjin Lee wrote: > I updated KIP-110 with JMH-measured benchmark results. Please have a review > when you are free. (The overall result is not different yet.) > > Regards, > Dongjin

Re: [VOTE] KIP-115: Enforce offsets.topic.replication.factor

2017-01-26 Thread Ben Stopford
+1 Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/> > On 26 Jan 2017, at 07:45, James Cheng wrote: > > +1 > >> On Jan 25, 2017, at 9:48 PM, Ewen Cheslack-Postava wrote: >> >> +1 >> >> On Wed, Jan 25, 2017 at 9:23 PM

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-08 Thread Ben Stopford
Yes - using a tool like this to skip a set of consumer groups over a corrupt/bad message is definitely appealing. B On Wed, Feb 8, 2017 at 9:37 PM Gwen Shapira wrote: > I like the --reset-to-earliest and --reset-to-latest. In general, > since the JSON route is the most challenging for users, we

Jira Permissions

2015-08-07 Thread Ben Stopford
Hey - can I permission to take ownership of Jiras, and edit the wiki also? is there a process for this? Cheers B

Re: is SSL support feature ready to use in kafka-truck branch

2015-08-21 Thread Ben Stopford
Hi Qi Trunk seems fairly stable. There are guidelines here which includes how to generate keys https://cwiki.apache.org/confluence/display/KAFKA/Deploying+SSL+for+Kafka Your server config needs these properties (also

Re: Kafka New Consumer Performance Test ThroughPut Degradation

2015-08-28 Thread Ben Stopford
So I’ve been looking at the SSL performance (which potentially also has an issue unrelated to this). I’d noticed strange behaviour with poll() on the new consumer but I wasn’t sure whether this was a bug or a feature. On closer inspection it seems to arise from ConsumerPerformance relying on a c

Re: ProducerPerformance.scala compressing Array of Zeros

2015-09-01 Thread Ben Stopford
You’re absolutely right. This should be fixed. I’ve made a note of this in https://issues.apache.org/jira/browse/KAFKA-2499 . If you’d like to submit a pull request for this that would be awesome :) Otherwise I’ll try and fit it into the other

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-11 Thread Ben Stopford
still going to > ask for the that partition in the next fetch request. Would it be fair to > add some logic there so that the follower backs off ( for some configurable > time) from including those partitions in the next fetch request? > > Thanks, > > Mayuresh > &

Re: [DISCUSS] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-12 Thread Ben Stopford
Andrey To make progress, I suggest you keep the partition-level limit in, at least for now, and keep it on the FetchRequest too. B > On 10 Aug 2016, at 18:55, Andrey L. Neporada wrote: > > Hi, Jun! > > Thanks for feedback! > >> On 10 Aug 2016, at 17:42, Jun Rao wrote: >> >> Hi, Andrey,

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-12 Thread Ben Stopford
e ( If the inclusion of a partition, listed in the leader's >>>>> throttled-replicas list, causes the LeaderQuotaRate to be exceeded, >>> that >>>>> partition is omitted from the response (aka returns 0 bytes).). At >> this >>>>> point the follower

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-17 Thread Ben Stopford
> This might have been answered before. > > > > > > I was wondering when the leader quota is reached and it sends > empty > > > > > > response ( If the inclusion of a partition, listed in the > leader's > > > > > > throttled-replic

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Ben Stopford
hope that seems reasonable. Jun may wish to add to this. B > On 18 Aug 2016, at 06:56, Joel Koshy wrote: > > On Wed, Aug 17, 2016 at 9:13 PM, Ben Stopford wrote: > >> >> Let's us know if you have any further thoughts on KIP-73, else we'll kick >> o

[VOTE] KIP-73 - Replication Quotas

2016-08-19 Thread Ben Stopford
I’d like to initiate the voting process for KIP-73: https://cwiki.apache.org/confluence/display/KAFKA/KIP-73+Replication+Quotas Ben

Re: [VOTE] KIP-73 - Replication Quotas

2016-08-23 Thread Ben Stopford
both your points. Will amend. B Ben Stopford Confluent, http://www.confluent.io <http://www.confluent.io/> > On 23 Aug 2016, at 14:41, Ismael Juma wrote: > > Thanks for the KIP, +1 (binding) with a couple of minor suggestions: > > //Sample configuration f

Re: [VOTE] KIP-73 - Replication Quotas

2016-08-23 Thread Ben Stopford
ff-thread) > > On Fri, Aug 19, 2016 at 1:21 AM, Ben Stopford wrote: > >> I’d like to initiate the voting process for KIP-73: >> https://cwiki.apache.org/confluence/display/KAFKA/KIP- >> 73+Replication+Quotas <https://cwiki.apache.org/ >> confluence/display/KAFKA/KIP-73+Replication+Quotas> >> >> Ben

[DISCUSS] KIP-73: Replication Quotas

2016-09-25 Thread Ben Stopford
Hi All We’ve made an adjustment to KIP-73: Replication Quotas which I’d like to open up to the community for approval. Previously the admin passed a list of replicas to be throttled: quota.replication.throttled.replicas = [partId]:[replica],[partId]:[replica],[partId]:[replica] etc T

New Website Layout

2020-06-26 Thread Ben Stopford
Hey folks We've made some updates to the website's look and feel. There is a staged version in the link below. https://ec2-13-57-18-236.us-west-1.compute.amazonaws.com/ username: kafka password: streaming Comments welcomed. Ben

Re: [VOTE] KIP-595: A Raft Protocol for the Metadata Quorum

2020-08-03 Thread Ben Stopford
; notified of such changes. > > For me personally, this is super exciting since we have been thinking about > this work ever since I started working on Kafka! I am +1 of course. > > Best, > Jason > -- Ben Stopford Lead Technologist, Office of the CTO <https://www.confluent.io>

Re: New Website Layout

2020-08-04 Thread Ben Stopford
cs pages to also use most of the page. For instance, config > and > > metrics tables could look more readable using more space. > > Would it be possible to make the left menu expand/collapse, similar to > > current Confluence wiki menu? > > > > Thanks, > &g

Re: New Website Layout

2020-08-05 Thread Ben Stopford
The new website layout has gone live as you may have seen. There are a couple of rendering issues in the streams developer guide that we're getting addressed. If anyone spots anything else could they please reply to this thread. Thanks Ben On Fri, 26 Jun 2020 at 11:48, Ben Stopford

Re: New Website Layout

2020-08-05 Thread Ben Stopford
onsole. I opened a PR to remove the > console.log() call: https://github.com/apache/kafka-site/pull/278 > > On Wed, Aug 5, 2020 at 9:45 AM Ben Stopford wrote: > > > > The new website layout has gone live as you may have seen. There are a > > couple of rendering issues in the strea

Re: New Website Layout

2020-08-10 Thread Ben Stopford
> meaning you can't click on the link to copy and paste the URL. Could the > > old behaviour be restored? > > > > Thanks, > > > > Tom > > > > On Wed, Aug 5, 2020 at 12:43 PM Luke Chen wrote: > > > > > When entering stream

Re: New Website Layout

2020-08-12 Thread Ben Stopford
uot; section is rendering as > > > > $1 class="anchor-heading">$8$9$10 > > <https://kafka.apache.org/documentation.html#$4> > > > > with the . Similar thing in > > https://kafka.apache.org/documentation.html#design_quotas. The source > HTML >

Re: [VOTE] KIP-657: Add Customized Kafka Streams Logo

2020-08-20 Thread Ben Stopford
gt; > > > > > > > > On Tuesday, 18 August, 2020, 11:44:20 pm IST, Guozhang > Wang < > > > > > > wangg...@gmail.com> wrote: > > > > > > > > > > > > > > I'm leaning towards design B primarily because it reminds me > of > > > the > > > > > > Firefox > > > > > > > logo which I like a lot. But I also share Adam's concern that > it > > > > should > > > > > > > better not obscure the Kafka logo --- so if we can tweak a bit > to > > > fix > > > > > it > > > > > > my > > > > > > > vote goes to B, otherwise A :) > > > > > > > > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > On Tue, Aug 18, 2020 at 9:48 AM Bruno Cadonna < > > br...@confluent.io> > > > > > > wrote: > > > > > > > > > > > > > >> Thanks for the KIP! > > > > > > >> > > > > > > >> I am +1 (non-binding) for A. > > > > > > >> > > > > > > >> I would also like to hear opinions whether the logo should be > > > > > colorized > > > > > > >> or just black and white. > > > > > > >> > > > > > > >> Best, > > > > > > >> Bruno > > > > > > >> > > > > > > >> > > > > > > >> On 15.08.20 16:05, Adam Bellemare wrote: > > > > > > >>> I prefer Design B, but given that I missed the discussion > > > thread, I > > > > > > think > > > > > > >>> it would be better without the Otter obscuring any part of > the > > > > Kafka > > > > > > >> logo. > > > > > > >>> > > > > > > >>> On Thu, Aug 13, 2020 at 6:31 PM Boyang Chen < > > > > > > reluctanthero...@gmail.com> > > > > > > >>> wrote: > > > > > > >>> > > > > > > >>>> Hello everyone, > > > > > > >>>> > > > > > > >>>> I would like to start a vote thread for KIP-657: > > > > > > >>>> > > > > > > >>>> > > > > > > >> > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-657%3A+Add+Customized+Kafka+Streams+Logo > > > > > > >>>> > > > > > > >>>> This KIP is aiming to add a new logo for the Kafka Streams > > > > library. > > > > > > And > > > > > > >> we > > > > > > >>>> prepared two candidates with a cute otter. You could look up > > the > > > > KIP > > > > > > to > > > > > > >>>> find those logos. > > > > > > >>>> > > > > > > >>>> > > > > > > >>>> Please post your vote against these two customized logos. > For > > > > > > example, I > > > > > > >>>> would vote for *design-A (binding)*. > > > > > > >>>> > > > > > > >>>> This vote thread shall be open for one week to collect > enough > > > > votes > > > > > to > > > > > > >> call > > > > > > >>>> for a winner. Still, feel free to post any question you may > > have > > > > > > >> regarding > > > > > > >>>> this KIP, thanks! > > > > > > >>>> > > > > > > >>> > > > > > > >> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > Michael G. Noll > > > > Principal Technologist, Office of the CTO > > > > <https://www.confluent.io> > > > > > -- > > <https://www.confluent.io> > > Antony Stubbs > > Principal Consulting Engineer / Solutions Architect > > > Follow us: Blog > < > https://confluent.io/blog?utm_source=footer&utm_medium=email&utm_campaign=ch.email-signature_type.community_content.blog > > > • Slack <https://slackpass.io/confluentcommunity> • Twitter > <https://twitter.com/ConfluentInc> • YouTube < > https://youtube.com/confluent> > -- Ben Stopford Lead Technologist, Office of the CTO <https://www.confluent.io>

Re: virtual KIP meeting for KIP-405

2020-08-24 Thread Ben Stopford
t; and > > > > > outstanding issues, starting from this coming Tuesday at 9am PT. If > > you > > > > are > > > > > interested in attending, please let Harsha or me know. > > > > > > > > > > The recording of the meeting will be posted in > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals > > > > > . > > > > > > > > > > Thanks, > > > > > > > > > > Jun > > > > > > > > > > -- Ben Stopford

Re: [DISCUSS] KIP idea: Support of multipart messages

2020-09-08 Thread Ben Stopford
If all that makes sense, I'll create a full fledged KIP document and expand > the idea. > > Thanks, > A. > -- Ben Stopford

Re: [DISCUSS] KIP-720: Deprecate MirrorMaker v1

2021-03-17 Thread Ben Stopford
ote: > > > Hey y'all, I'd like to start the discussion on KIP-720, which proposes to > > deprecate the original MirrorMaker in the upcoming 3.0 major release. > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-720%3A+Deprecate+MirrorMaker+v1 > > > > Thanks! > > Ryanne > > > -- Ben Stopford

Re: producer-consumer issues during deployments

2015-11-26 Thread Ben Stopford
Hi Prabhjot I may have slightly misunderstood your question so apologies if that’s the case. The general approach to releases is to use a rolling upgrade where you take one machine offline at a time, restart it, wait for it to come online (you can monitor this via JMX) then move onto the next.

[jira] [Commented] (KAFKA-3726) Enable cold storage option

2016-05-18 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15289608#comment-15289608 ] Ben Stopford commented on KAFKA-3726: - The standard approach to this sort of pro

[jira] [Commented] (KAFKA-3736) Add http metrics reporter

2016-05-20 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3736?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15293234#comment-15293234 ] Ben Stopford commented on KAFKA-3736: - Nice little idea! I think it might requi

[jira] [Commented] (KAFKA-3736) Add http metrics reporter

2016-05-20 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3736?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15293328#comment-15293328 ] Ben Stopford commented on KAFKA-3736: - I think it's the "Any change th

[jira] [Commented] (KAFKA-3726) Enable cold storage option

2016-05-23 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296178#comment-15296178 ] Ben Stopford commented on KAFKA-3726: - Ah, OK. Yes I quite like this

[jira] [Assigned] (KAFKA-1464) Add a throttling option to the Kafka replication tool

2016-07-01 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-1464: --- Assignee: Ben Stopford (was: Ismael Juma) > Add a throttling option to the Ka

[jira] [Commented] (KAFKA-2063) Bound fetch response size

2016-07-29 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2063?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15399855#comment-15399855 ] Ben Stopford commented on KAFKA-2063: - Thanks for the kicking this one off Andre

[jira] [Commented] (KAFKA-2063) Bound fetch response size

2016-07-29 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2063?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15399960#comment-15399960 ] Ben Stopford commented on KAFKA-2063: - [~jkreps] OK, let's see what GZ say

[jira] [Commented] (KAFKA-4178) Replication Throttling: Consolidate Rate Classes

2016-11-23 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15691178#comment-15691178 ] Ben Stopford commented on KAFKA-4178: - Yes, that should be fine. > Repl

[jira] [Commented] (KAFKA-4522) Using Disruptor instead of Array Blocking queue in Kafka Producer

2016-12-14 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15748169#comment-15748169 ] Ben Stopford commented on KAFKA-4522: - It'd be good to have some empirical

[jira] [Commented] (KAFKA-1120) Controller could miss a broker state change

2016-12-15 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15750982#comment-15750982 ] Ben Stopford commented on KAFKA-1120: - [~junrao] added this comment on a mail th

[jira] [Created] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-01 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-4825: --- Summary: Likely Data Loss in ReassignPartitionsTest System Test Key: KAFKA-4825 URL: https://issues.apache.org/jira/browse/KAFKA-4825 Project: Kafka Issue

[jira] [Commented] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-01 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15890661#comment-15890661 ] Ben Stopford commented on KAFKA-4825: - This could be a result of KIP-101 (h

[jira] [Comment Edited] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-02 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15890661#comment-15890661 ] Ben Stopford edited comment on KAFKA-4825 at 3/2/17 2:5

[jira] [Commented] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-09 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15902877#comment-15902877 ] Ben Stopford commented on KAFKA-4825: - Thanks Jun. Makes sense. > Likely Da

[jira] [Resolved] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-09 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford resolved KAFKA-4825. - Resolution: Later Resolving this as it will be fixed by KIP-101 in 3.3 > Likely Data Loss

[jira] [Commented] (KAFKA-5036) Followups from KIP-101

2017-04-10 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15962827#comment-15962827 ] Ben Stopford commented on KAFKA-5036: - Points 2, 5 are addressed in PR: h

[jira] [Updated] (KAFKA-5036) Followups from KIP-101

2017-04-10 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-5036: Description: 1. It would be safer to hold onto the leader lock in Partition while serving an

[jira] [Comment Edited] (KAFKA-5036) Followups from KIP-101

2017-04-10 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15962827#comment-15962827 ] Ben Stopford edited comment on KAFKA-5036 at 4/10/17 6:4

[jira] [Work started] (KAFKA-4596) KIP-73 rebalance throttling breaks on plans for specific partitions

2017-01-18 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4596 started by Ben Stopford. --- > KIP-73 rebalance throttling breaks on plans for specific partiti

[jira] [Updated] (KAFKA-4596) KIP-73 rebalance throttling breaks on plans for specific partitions

2017-01-18 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-4596: Status: Patch Available (was: In Progress) > KIP-73 rebalance throttling breaks on plans

[jira] [Assigned] (KAFKA-2363) ProducerSendTest.testCloseWithZeroTimeoutFromCallerThread Transient Failure

2017-01-23 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2363: --- Assignee: (was: Ben Stopford

[jira] [Created] (KAFKA-3438) Rack Aware Replica Reassignment should warn of overloaded brokers

2016-03-21 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-3438: --- Summary: Rack Aware Replica Reassignment should warn of overloaded brokers Key: KAFKA-3438 URL: https://issues.apache.org/jira/browse/KAFKA-3438 Project: Kafka

[jira] [Created] (KAFKA-2964) Split Security Rolling Upgrade Test By Client and Broker Protocols

2015-12-08 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-2964: --- Summary: Split Security Rolling Upgrade Test By Client and Broker Protocols Key: KAFKA-2964 URL: https://issues.apache.org/jira/browse/KAFKA-2964 Project: Kafka

[jira] [Updated] (KAFKA-2771) Add Rolling Upgrade to Secured Cluster to System Tests

2015-12-11 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2771?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-2771: Resolution: Fixed Status: Resolved (was: Patch Available) > Add Rolling Upgrade to Secu

[jira] [Updated] (KAFKA-2637) Cipher suite setting should be configurable for SSL

2015-12-11 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-2637: Resolution: Fixed Status: Resolved (was: Patch Available) > Cipher suite setting should

[jira] [Created] (KAFKA-3223) Add System (ducktape) Test that asserts strict partition ordering despite node failure

2016-02-09 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-3223: --- Summary: Add System (ducktape) Test that asserts strict partition ordering despite node failure Key: KAFKA-3223 URL: https://issues.apache.org/jira/browse/KAFKA-3223

[jira] [Updated] (KAFKA-3223) Add System (ducktape) Test that asserts strict partition ordering despite node failure

2016-02-09 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3223?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-3223: Description: Kafka provides strong ordering when max.inflight.requests.per.connection = 1. There

[jira] [Commented] (KAFKA-3244) TopicCommand doesn't allow to add partitions with specific assignment

2016-02-17 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15151110#comment-15151110 ] Ben Stopford commented on KAFKA-3244: - Hey - [~granthenke] - I'm wor

[jira] [Assigned] (KAFKA-3270) Add Basic Integration Tests Around ReassignPartitionsCommand

2016-02-23 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-3270: --- Assignee: Ben Stopford > Add Basic Integration Tests Around ReassignPartitionsComm

[jira] [Created] (KAFKA-3270) Add Basic Integration Tests Around ReassignPartitionsCommand

2016-02-23 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-3270: --- Summary: Add Basic Integration Tests Around ReassignPartitionsCommand Key: KAFKA-3270 URL: https://issues.apache.org/jira/browse/KAFKA-3270 Project: Kafka

[jira] [Commented] (KAFKA-2364) Improve documentation for contributing to docs

2015-08-06 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2364?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14660041#comment-14660041 ] Ben Stopford commented on KAFKA-2364: - Hi Aseem Have you signed up to the mai

[jira] [Updated] (KAFKA-1955) Explore disk-based buffering in new Kafka Producer

2015-08-07 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1955?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-1955: Attachment: KAFKA-1955-RABASED-TO-8th-AUG-2015.patch > Explore disk-based buffering in new Ka

[jira] [Commented] (KAFKA-2398) Transient test failure for SocketServerTest - Socket closed.

2015-08-11 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14687399#comment-14687399 ] Ben Stopford commented on KAFKA-2398: - [~ijuma] [~becket_qin] so can we close

[jira] [Assigned] (KAFKA-2363) ProducerSendTest.testCloseWithZeroTimeoutFromCallerThread Transient Failure

2015-08-11 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2363: --- Assignee: Ben Stopford > ProducerSendTest.testCloseWithZeroTimeoutFromCallerThread Transi

[jira] [Assigned] (KAFKA-2015) Enable ConsoleConsumer to use new consumer

2015-08-12 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2015: --- Assignee: Ben Stopford (was: Guozhang Wang) > Enable ConsoleConsumer to use new consu

[jira] [Assigned] (KAFKA-2431) Test SSL/TLS impact on performance

2015-08-17 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2431: --- Assignee: Ben Stopford > Test SSL/TLS impact on performa

[jira] [Updated] (KAFKA-2015) Enable ConsoleConsumer to use new consumer

2015-08-20 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-2015: Status: Patch Available (was: In Progress) > Enable ConsoleConsumer to use new consu

[jira] [Assigned] (KAFKA-2453) enable new consumer in EndToEndLatency

2015-08-20 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2453: --- Assignee: Ben Stopford > enable new consumer in EndToEndLate

[jira] [Commented] (KAFKA-2453) enable new consumer in EndToEndLatency

2015-08-20 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2453?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14705257#comment-14705257 ] Ben Stopford commented on KAFKA-2453: - Yep - I'm on it > enable new

[jira] [Commented] (KAFKA-2015) Enable ConsoleConsumer to use new consumer

2015-08-20 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14705310#comment-14705310 ] Ben Stopford commented on KAFKA-2015: - I think we should move this to 0.8.3? It w

[jira] [Commented] (KAFKA-2431) Test SSL/TLS impact on performance

2015-08-24 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14709852#comment-14709852 ] Ben Stopford commented on KAFKA-2431: - Progress is recorded in this doc h

[jira] [Commented] (KAFKA-2467) ConsoleConsumer regressions

2015-08-25 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14710860#comment-14710860 ] Ben Stopford commented on KAFKA-2467: - Apologies for this regression. 2015

  1   2   3   >