Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-09-25 Thread Boyang Chen
roblem only. Admin clients still need to learn the proper ApiVersion from the broker, which means we need to bump IBP to limit the version range. Boyang > Thanks, > > Jun > > On Thu, Sep 24, 2020 at 6:22 PM Boyang Chen > wrote: > > > Hey Jason and Jun, > > > > tha

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-09-24 Thread Boyang Chen
or > > > > conversion between versions, but that would restrict the flexibility > > that > > > > the versioning is providing. It would also be a large effort to avoid > > > > introducing regressions through conversion. Sadly this broadens the >

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-09-24 Thread Boyang Chen
in any circumstance, since the flexible field support is required to be open-ended on the high side. Let me know if you have any questions. Best, Boyang On Thu, Aug 6, 2020 at 6:11 PM Boyang Chen wrote: > Hey there, > > we are going to introduce a minor change to bump the version of

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-21 Thread Boyang Chen
Thanks for the KIP Walker. In the KIP we mentioned "In order to communicate the shutdown request from one client to the others we propose to update the SubcriptionInfoData to include a short field which will encode an error code.", is there a dedicated error code that we should define here, or it

Re: [DISCUSS] Apache Kafka 2.7.0 release

2020-09-21 Thread Boyang Chen
Hey Bill, unfortunately KIP-590 will not be in 2.7 release, could you move it to postponed KIPs? Best, Boyang On Thu, Sep 10, 2020 at 2:41 PM Bill Bejeck wrote: > Hi Gary, > > It's been added. > > Regards, > Bill > > On Thu, Sep 10, 2020 at 4:14 PM Gary Russell wrote: > > > Can someone add a

[jira] [Created] (KAFKA-10508) Consider moving ForwardRequestHandler to a separate class

2020-09-21 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10508: --- Summary: Consider moving ForwardRequestHandler to a separate class Key: KAFKA-10508 URL: https://issues.apache.org/jira/browse/KAFKA-10508 Project: Kafka

Re: [VOTE] KIP-664: Provide tooling to detect and abort hanging transactions

2020-09-18 Thread Boyang Chen
Thanks Jason, +1 (binding) from me Boyang On Fri, Sep 11, 2020 at 10:12 AM Robert Barrett wrote: > +1 (non-binding) > > Thanks Jason! > > On Tue, Sep 8, 2020 at 5:28 PM Guozhang Wang wrote: > > > +1. Thanks! > > > > Guozhang > > > > On Tue, Sep 8, 2020 at 3:04 PM Ron Dagostino wrote: > > > >

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-09-17 Thread Boyang Chen
Thanks for the updates Jason. I'm pretty satisfied with the overall motivation and proposed solution, just a couple of more comments. 1. Why do we need to use type string for `StatesFilter` instead of a short value, as we could translate it and save space? 2. I'm wondering whether the

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-27 Thread Boyang Chen
Thanks Jason for the tooling proposal. A couple of comments: 1. For the analysis section, is there any consistency guarantee for `ListTransactions` and `DescribeTransactions`? Let's say the coordinator receives a DescribeTransactions while the transaction is almost complete at the same time,

Re: Requesting to add to contributor list and write access for wiki

2020-08-20 Thread Boyang Chen
Added both permissions, you are good to go. On Thu, Aug 20, 2020 at 8:05 AM sasilekha wrote: > Hello Team, > > I am interested in contributing to Apache Kafka codebase. Kindly add me to > contributor list. It would help me in assigning Jiras to myself on Kafka. > > Also, kindly grant me wiki

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

2020-08-19 Thread Boyang Chen
e: > > > > > > > > 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 tw

Re: Permission to create a KIP

2020-08-18 Thread Boyang Chen
Done, thanks for the interest in Kafka! On Tue, Aug 18, 2020 at 10:20 AM Neemias Junior wrote: > Hello, could you please give me permission to create a Kafka Improvement > Proposal? My ID is neemiasjnr. If you want I can explain what I'm going to > propose beforehand. Thank you! > > Best

Re: [EXTERNAL] Re: Request for access to create KIP

2020-08-14 Thread Boyang Chen
Done! On Thu, Aug 13, 2020 at 2:25 PM Koushik Chitta wrote: > Accountid: koushikchitta > > Cheers, > Koushik > > -Original Message----- > From: Boyang Chen > Sent: Sunday, August 9, 2020 6:02 PM > To: dev > Subject: [EXTERNAL] Re: Request for access to c

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

2020-08-13 Thread Boyang Chen
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

[jira] [Created] (KAFKA-10400) Add a customized Kafka Streams logo

2020-08-13 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10400: --- Summary: Add a customized Kafka Streams logo Key: KAFKA-10400 URL: https://issues.apache.org/jira/browse/KAFKA-10400 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-10042) Make INVALID_PRODUCER_EPOCH abortable from Produce response

2020-08-12 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10042. - Resolution: Fixed > Make INVALID_PRODUCER_EPOCH abortable from Produce respo

[jira] [Resolved] (KAFKA-9911) Implement new producer fenced error

2020-08-12 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-9911. Resolution: Fixed > Implement new producer fenced er

Re: [ANNOUNCE] New Kafka PMC Member: John Roesler

2020-08-10 Thread Boyang Chen
Congrats Mr. John! On Mon, Aug 10, 2020 at 3:02 PM Adam Bellemare wrote: > Congratulations John! You have been an excellent help to me and many > others. I am pleased to see this! > > > On Aug 10, 2020, at 5:54 PM, Bill Bejeck wrote: > > > > Congrats! > > > >> On Mon, Aug 10, 2020 at 4:52 PM

Re: Request for access to create KIP

2020-08-09 Thread Boyang Chen
Have you created the account already? What's your account id? On Sat, Aug 8, 2020 at 4:36 PM Koushik Chitta wrote: > Hi Team, > > Can you please grant me access to create KIP ? > > Thanks, > Koushik >

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-08-06 Thread Boyang Chen
. The are only two of them: 1. AlterConfig 2. AlterClientQuotas Let me know if you have any questions. Boyang On Fri, Jul 31, 2020 at 11:42 AM Boyang Chen wrote: > Hey David, > > After discussing with Colin offline, I would like to correct one case in > the described wor

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

2020-08-04 Thread Boyang Chen
Thanks for the KIP Jason, +1 (binding) from me as well for sure :) On Tue, Aug 4, 2020 at 2:46 PM Colin McCabe wrote: > 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

[jira] [Created] (KAFKA-10345) Redirect AlterPartitionReassignment to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10345: --- Summary: Redirect AlterPartitionReassignment to the controller Key: KAFKA-10345 URL: https://issues.apache.org/jira/browse/KAFKA-10345 Project: Kafka Issue

[jira] [Created] (KAFKA-10344) Redirect Create/Renew/ExpireDelegationToken to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10344: --- Summary: Redirect Create/Renew/ExpireDelegationToken to the controller Key: KAFKA-10344 URL: https://issues.apache.org/jira/browse/KAFKA-10344 Project: Kafka

[jira] [Created] (KAFKA-10343) Redirect AlterClientQuotas to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10343: --- Summary: Redirect AlterClientQuotas to the controller Key: KAFKA-10343 URL: https://issues.apache.org/jira/browse/KAFKA-10343 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-10342) Redirect CreateAcls/DeleteAcls to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10342: --- Summary: Redirect CreateAcls/DeleteAcls to the controller Key: KAFKA-10342 URL: https://issues.apache.org/jira/browse/KAFKA-10342 Project: Kafka Issue Type

[jira] [Created] (KAFKA-10350) Add redirect request monitoring metrics

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10350: --- Summary: Add redirect request monitoring metrics Key: KAFKA-10350 URL: https://issues.apache.org/jira/browse/KAFKA-10350 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10349) Deprecate client side controller access

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10349: --- Summary: Deprecate client side controller access Key: KAFKA-10349 URL: https://issues.apache.org/jira/browse/KAFKA-10349 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10348) Redirect UpdateFeatures to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10348: --- Summary: Redirect UpdateFeatures to the controller Key: KAFKA-10348 URL: https://issues.apache.org/jira/browse/KAFKA-10348 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10347) Redirect Create/DeleteTopics to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10347: --- Summary: Redirect Create/DeleteTopics to the controller Key: KAFKA-10347 URL: https://issues.apache.org/jira/browse/KAFKA-10347 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-10346) Redirect CreatePartition to the controller

2020-08-03 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10346: --- Summary: Redirect CreatePartition to the controller Key: KAFKA-10346 URL: https://issues.apache.org/jira/browse/KAFKA-10346 Project: Kafka Issue Type: Sub

Re: [VOTE] KIP-648: Renaming getter method for Interactive Queries

2020-08-03 Thread Boyang Chen
Thanks for the KIP, +1 (binding) from me! On Sun, Aug 2, 2020 at 4:49 PM Matthias J. Sax wrote: > Thanks for the KIP, John! > > +1 (binding) > > > > On 8/2/20 11:04 AM, John Thomas wrote: > > Hello , > > > > The vote has been open for >72 hours, yet to get some binding votes. > Can everyone

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-31 Thread Boyang Chen
Boyang Chen wrote: > > > On Thu, Jul 30, 2020 at 7:18 AM David Jacot wrote: > >> Hi Boyang, >> >> Thanks for your answers. >> >> > The point for using the listener name is more of a security purpose, to >> > detect any forged request to our

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-31 Thread Boyang Chen
an interface. I think this was my plan before. I don't > > think > > > > > I realized at the time that it's possible to replace the entire > > class with > > > > > an interface. Now I realize it is possible, hence the motivation to > > do it. > > > > > &g

Re: [ANNOUNCE] New Kafka PMC Member: Mickael Maison

2020-07-31 Thread Boyang Chen
Congrats Mickael! On Fri, Jul 31, 2020 at 8:47 AM Bill Bejeck wrote: > Congrats Mickael! > > -Bill > > On Fri, Jul 31, 2020 at 11:34 AM Tom Bentley wrote: > > > Congratulations Mickael! > > > > On Fri, Jul 31, 2020 at 4:23 PM Jun Rao wrote: > > > > > Hi, Everyone, > > > > > > Mickael Maison

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-30 Thread Boyang Chen
rsion of routing into the design. > - For the record, should we put the previous proposal in the rejected > alternatives as > well? > > We do have big changes in this KIP, our current strategy is to rely on wiki revisions if people are interested to figure out our previous design

[jira] [Resolved] (KAFKA-10270) Add a broker to controller channel manager to redirect AlterConfig

2020-07-29 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10270. - Resolution: Fixed > Add a broker to controller channel manager to redirect AlterCon

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-29 Thread Boyang Chen
e to me. I will work on drafting an addendum > to KIP-599 to > alter the design to cope with these changes. At a first glance, that seems > doable if 1.1, 3 > and 4 are OK. > > Thank you for the help! > Thanks, > David > > On Wed, Jul 29, 2020 at 5:29 AM Boyang C

[jira] [Created] (KAFKA-10321) shouldDieOnInvalidOffsetExceptionWhileRunning would block forever on JDK11

2020-07-29 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10321: --- Summary: shouldDieOnInvalidOffsetExceptionWhileRunning would block forever on JDK11 Key: KAFKA-10321 URL: https://issues.apache.org/jira/browse/KAFKA-10321 Project

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-28 Thread Boyang Chen
RE(92, "Authorization failed for the > > request during forwarding, this indicates an internal error on the broker > > cluster security setup.", BrokerAuthorizationFailureException::new); > > Grammar nitpick: It would be good to have a period between "forwar

Re: Request for access to create KIP

2020-07-28 Thread Boyang Chen
Permission granted On Tue, Jul 28, 2020 at 9:01 AM John Thomas wrote: > johnthotekat > > > From: John Thomas > Sent: 27 July 2020 23:29 > To: dev@kafka.apache.org > Subject: Request for access to create KIP > > Hello Team, > > Could you please grant me access

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

2020-07-27 Thread Boyang Chen
t of the Raft log, > > there is no way to solve this? > > I know LogCabin implementation does replicate client heartbeats. I > suspect > > that the same issue is there in Zookeeper, which does not replicate > client > > Ping requests.. &

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-27 Thread Boyang Chen
, I'm open for re-vote after discussions converge. Boyang On Wed, Jul 1, 2020 at 2:17 PM Boyang Chen wrote: > Hey folks, > > I have also synced on the KIP-578 which was doing the partition limit, to > make sure the partition limit error code would be properly propagated once > it

[jira] [Created] (KAFKA-10311) Flaky test KafkaAdminClientTest#testMetadataRetries

2020-07-26 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10311: --- Summary: Flaky test KafkaAdminClientTest#testMetadataRetries Key: KAFKA-10311 URL: https://issues.apache.org/jira/browse/KAFKA-10311 Project: Kafka Issue Type

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

2020-07-26 Thread Boyang Chen
ally defer fsync until it knows > "quorum.size - > > > 1" > > > > has get to a certain entry offset." Why is that "quorum.size - 1" > > instead > > > > of the majority of the quorum? > > > > > > > > Th

[jira] [Created] (KAFKA-10307) Topology cycles in KTableKTableForeignKeyInnerJoinMultiIntegrationTest#shouldInnerJoinMultiPartitionQueryable

2020-07-25 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10307: --- Summary: Topology cycles in KTableKTableForeignKeyInnerJoinMultiIntegrationTest#shouldInnerJoinMultiPartitionQueryable Key: KAFKA-10307 URL: https://issues.apache.org/jira/browse

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-24 Thread Boyang Chen
Thanks for the KIP John. I share a similar concern with the motivation, it would be good if you could cast light on the actual downside of using a base class vs the interface, is it making the code fragile, or requiring redundant implementation, etc. Boyang On Tue, Jul 21, 2020 at 2:19 PM Sophie

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-24 Thread Boyang Chen
020, at 11:08, Boyang Chen wrote: > > Hey Colin, some quick questions, > > > > 1. I looked around and didn't find a config for broker heartbeat > interval, > > are we piggy-back on some existing configs? > > > > Good point. I meant to add this, but I f

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-07-18 Thread Boyang Chen
ternal topics marked for deletion > by >the default behaviour (do a dry-run > without this >option to view these topics). > > > - Joel > > On 11 Jul 2020, at 8:41 AM, Boyang Chen <mailto:reluctanthero...@gmail.com&g

[jira] [Created] (KAFKA-10284) Group membership update due to static member rejoin should be persisted

2020-07-16 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10284: --- Summary: Group membership update due to static member rejoin should be persisted Key: KAFKA-10284 URL: https://issues.apache.org/jira/browse/KAFKA-10284 Project: Kafka

Re: Requesting permission to create KIP

2020-07-15 Thread Boyang Chen
Should be good to go now. On Wed, Jul 15, 2020 at 8:12 AM Nikhil kumar wrote: > Hi kafka-devs, > I am planning to add support for sources which can't give out changed > records in kafka connect. > Please, provide me permission to create KIP. > > Wiki id - nikhil578 > Email id -

Re: Permissions to create and assign JIRA issues

2020-07-13 Thread Boyang Chen
Added. Happy contributing! On Mon, Jul 13, 2020 at 2:48 PM Sankalp Bhatia wrote: > Hi devs, > > I would like to contribute and will be grateful if someone can grant me the > permissions to create and assign JIRA issues. > > my id is : sankalpbhatia > > Thanks in advance! >

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-13 Thread Boyang Chen
Hey Colin, some quick questions, 1. I looked around and didn't find a config for broker heartbeat interval, are we piggy-back on some existing configs? 2. We only mentioned that the lease time is 10X of the heartbeat interval, could we also include why we chose this value? On Mon, Jul 13, 2020

[jira] [Created] (KAFKA-10270) Add a broker to controller channel manager to redirect AlterConfig

2020-07-13 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10270: --- Summary: Add a broker to controller channel manager to redirect AlterConfig Key: KAFKA-10270 URL: https://issues.apache.org/jira/browse/KAFKA-10270 Project: Kafka

Re: [VOTE] KIP-554: Add Broker-side SCRAM Config API

2020-07-13 Thread Boyang Chen
just one > question: > > since we don't return the password 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 >

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-07-10 Thread Boyang Chen
thinking about adding > something to view just internal topics? > > Thanks Bruno for the suggestion. I will close this vote for now, and we > can continue the discussion on another thread. (P.S. apologies for > misspelling your name previously) > > - Joel > >

Re: [VOTE] KIP-620 Deprecate ConsumerConfig#addDeserializerToConfig(Properties, Deserializer, Deserializer) and ProducerConfig#addSerializerToConfig(Properties, Serializer, Serializer)

2020-07-10 Thread Boyang Chen
deprecated and we add package-private variety of them. > > I will update KIP ! > > On 2020/07/07 16:12:49, Boyang Chen wrote: > > Ok, after a second thought, keeping a function which still has production > > reference is ok. We probably should not make it public in the first

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-10 Thread Boyang Chen
Thanks Leah and Sophie for the KIP. 1. I'm a bit surprised that we don't have an advance time. Could we elaborate how the storage layer is structured? 2. IIUC, there will be extra cost in terms of fetching aggregation results, since we couldn't pre-aggregate until the user asks for it. Would be

Re: [VOTE] KIP-554: Add Broker-side SCRAM Config API

2020-07-10 Thread Boyang Chen
Hey Colin, thanks for the KIP. One question I have about AlterScramUsers RPC is whether we could consolidate the deletion list and alteration list, since in response we only have a single list of results. The further benefit is to reduce unintentional duplicate entries for both deletion and

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-07 Thread Boyang Chen
back > c) > >>> > is not compatible > >>> > with punctuator call. The idea is that we could access clock using > >>> > uniform API. > >>> > For completness we should have same API for system and stream time. > >>> > &

Re: [VOTE] KIP-620 Deprecate ConsumerConfig#addDeserializerToConfig(Properties, Deserializer, Deserializer) and ProducerConfig#addSerializerToConfig(Properties, Serializer, Serializer)

2020-07-07 Thread Boyang Chen
cessary. > > I don't think my question gets answered, why would deprecating the map based `addSerializerToConfig` break user's recompilation? If you worry about warnings, we could refactor out the content and create a package-private `attachSerializersToConfig` or something similar. On 20

Re: KIP-560 Discuss

2020-07-07 Thread Boyang Chen
; > >> Hi all, thanks for the explanation. I was also not sure how the > > >> kip would be possible to implement. > > >> > > >> No that it does seem plausible, my only feedback is that the > > >> command line option could align better with the

Re: [VOTE] KIP-620 Deprecate ConsumerConfig#addDeserializerToConfig(Properties, Deserializer, Deserializer) and ProducerConfig#addSerializerToConfig(Properties, Serializer, Serializer)

2020-07-07 Thread Boyang Chen
Thanks for the KIP. One question I have is that when we refer to the two methods as useless, do we just suggest they no longer have any production use case? If this is the case, Producer#addSerializerToConfig(Map configs, keySerializer, valueSerializer) is only used in KafkaProducer internal only.

[jira] [Resolved] (KAFKA-10239) The groupInstanceId field in DescribeGroup response should be ignorable

2020-07-06 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10239. - Resolution: Fixed > The groupInstanceId field in DescribeGroup response should be ignora

[jira] [Created] (KAFKA-10241) Pursue a better way to cover ignorable RPC fields

2020-07-06 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10241: --- Summary: Pursue a better way to cover ignorable RPC fields Key: KAFKA-10241 URL: https://issues.apache.org/jira/browse/KAFKA-10241 Project: Kafka Issue Type

[jira] [Created] (KAFKA-10237) Properly handle in-memory stores OOM

2020-07-06 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10237: --- Summary: Properly handle in-memory stores OOM Key: KAFKA-10237 URL: https://issues.apache.org/jira/browse/KAFKA-10237 Project: Kafka Issue Type: Improvement

Re: Contribution Permission

2020-07-06 Thread Boyang Chen
Hey Jonas, I have added you the permission in both JIRA and wiki. Happy contributing! On Sun, Jul 5, 2020 at 11:12 PM Jonas Amslinger wrote: > Hi, > > I would like to start contributing as a developer and kindly ask to be > added to the contributor list. > > Jira and Wiki username: puhlerblet

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-07-01 Thread Boyang Chen
Hey folks, I have also synced on the KIP-578 which was doing the partition limit, to make sure the partition limit error code would be properly propagated once it is done on top of KIP-590. Let me know if you have further questions or concerns. Boyang On Tue, Jun 23, 2020 at 5:08 PM Boyang Chen

Re: Kafka Exactly-Once Semantics in .NET support

2020-07-01 Thread Boyang Chen
Thanks for the interest Saher. TBH, I don't know how the .NET confluent library works, but I would assume it is based on the https://github.com/edenhill/librdkafka. Maybe propose an issue there as well to attract some expertes. Boyang On Tue, Jun 30, 2020 at 3:22 PM Saher Ahwal wrote: > Hi > >

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-06-30 Thread Boyang Chen
Thanks Will for the KIP. A couple questions and suggestions: 1. I think for new APIs to make most sense, we should add a minimal example demonstrating how it could be useful to structure unit tests w/o the new APIs. 2. If this is a testing-only feature, could we only add it to

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-30 Thread Boyang Chen
d inadvertently > deleting topics of other applications? > > I know it is a backward incompatible change if users use it in > scripts, but I think it is still worth discussing it. I would to hear > what committers think about it. > > Best, > Bruno > > On Tue, Jun 30, 2020 at

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-29 Thread Boyang Chen
l when there are > prefix > > > > conflicts between applications, e.g. "app" and "app-v2". In this > case, if > > > > we want to reset "app", the reset tool’s default behaviour will > delete both > > > > the internal topi

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-06-28 Thread Boyang Chen
Hey Gokul, thanks for the reply. It is true that the Metadata API will call CreateTopic under the cover. The key guarantee we need to provide is to properly propagate the exact error code to the original client. So either we are going to introduce a new error code or reuse an existing one, the

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-26 Thread Boyang Chen
Thanks for driving the proposal Joel, I have a minor suggestion: we should be more clear about why we introduce this flag, so it would be better to also state clearly in the document for the default behavior as well, such like: Comma-separated list of internal topics to be deleted. By default,

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-06-24 Thread Boyang Chen
Hey Randal, There was another spotted blocker: https://issues.apache.org/jira/browse/KAFKA-10173 As of current, John is working on a fix. Boyang On Wed, Jun 24, 2020 at 4:08 PM Sophie Blee-Goldman wrote: > Hey all, > > Just a heads up that we discovered a new blocker. The fix is pretty >

[jira] [Resolved] (KAFKA-10135) Extract Task#executeAndMaybeSwallow to be a general utility function into TaskManager

2020-06-24 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10135. - Resolution: Fixed > Extract Task#executeAndMaybeSwallow to be a general utility funct

Re: [DISCUSS] KIP-487: Automatic Topic Creation on Producer

2020-06-23 Thread Boyang Chen
Hey Justin and Jiamei, I read the KIP and skimmed over the discussion. One thing I'm not fully convinced of is why we need to deprecate the server side auto topic creation logic, which seems orthogonal towards whether a client wants to create the topic or not. Won't it be more natural to assume

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-06-23 Thread Boyang Chen
Hi Gokul, Thanks for the excellent KIP. I was recently driving the rollout of KIP-590 and proposed to fix the hole of the bypassing of topic creation policy when applying

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-23 Thread Boyang Chen
Thanks for the clarification, Colin and Ismael. Personally I also feel Option A is better to prioritize fixing the gap. Just to be clear, the proposed solution would be: 1. Bump the Metadata RPC version to return POLICY_VIOLATION. In the application level, we should swap the error message with

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-22 Thread Boyang Chen
> Congratulations Boyang! Well deserved. > > > > -Bill > > > > On Mon, Jun 22, 2020 at 7:35 PM Colin McCabe wrote: > > > >> Congratulations, Boyang! > >> > >> cheers, > >> Colin > >> > >> On Mon, Jun 22, 2020, at

[jira] [Created] (KAFKA-10192) Flaky test BlockingConnectorTest#testBlockInConnectorStop

2020-06-22 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10192: --- Summary: Flaky test BlockingConnectorTest#testBlockInConnectorStop Key: KAFKA-10192 URL: https://issues.apache.org/jira/browse/KAFKA-10192 Project: Kafka

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-22 Thread Boyang Chen
On Mon, Jun 22, 2020 at 9:47 AM Ismael Juma wrote: > Thanks for the reply Boyang. Comments inline. > > On Mon, Jun 22, 2020 at 9:31 AM Boyang Chen > wrote: > > > Thanks for the questions Ismael. > > > > On Fri, Jun 19, 2020 at 3:18 PM Ismael Juma wrote: > &

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-22 Thread Boyang Chen
a bug. It's hard to imagine how a create > topic > > policy could be useful 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 > &

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-18 Thread Boyang Chen
ng, +1 from me. > > > > > > > > > > > > Guozhang > > > > > > > > On Wed, Jun 17, 2020 at 1:40 PM Colin McCabe > > wrote: > > > > > > > > > Thanks, Boyang! +1 (binding) > > > > > > > > > > bes

[jira] [Created] (KAFKA-10181) AlterConfig/IncrementalAlterConfig should go to controller

2020-06-17 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10181: --- Summary: AlterConfig/IncrementalAlterConfig should go to controller Key: KAFKA-10181 URL: https://issues.apache.org/jira/browse/KAFKA-10181 Project: Kafka

[jira] [Resolved] (KAFKA-10087) Properly throw LogTruncation exception from OffsetForLeaderEpoch future

2020-06-17 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10087?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10087. - Resolution: Fixed > Properly throw LogTruncation exception from OffsetForLeaderEpoch fut

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-15 Thread Boyang Chen
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 15:30, Boyang Chen wrote: > > Thanks Colin for the suggestions! > > > > On Fri, Jun 12, 2020 at 2:40 PM Colin McCabe

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-12 Thread Boyang Chen
ummary of > the main points. As it is, the old 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 ne

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

2020-06-12 Thread Boyang Chen
quot;. > 3.c) When the leader's fetch timeout elapsed, it should send metadata > request. > > Make sense, will add to the KIP. > > Guozhang > > > On Wed, Jun 10, 2020 at 5:20 PM Boyang Chen > wrote: > > > Hey all, > > > > follow-up on the previous

Re: [DISCUSS] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-11 Thread Boyang Chen
Thanks for the KIP Joel! Some quick questions and suggestions: 1. The KIP links to a wrong JIRA, which should be https://issues.apache.org/jira/browse/KAFKA-6435 2. Typo: *deletes all topic that *-> *deletes all topics that* 3. We need to explain in the Motivation section that we want to

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

2020-06-10 Thread Boyang Chen
for the version one. Let me know if you have any questions. Boyang On Tue, Jun 9, 2020 at 11:01 PM Boyang Chen wrote: > Hey all, > > Thanks for the great discussions so far. I'm posting some KIP updates from > our working group discussion: > > 1. We will be changing the core RPCs fr

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-10 Thread Boyang Chen
attributes including an accumulated total and rates ( > > https://metrics.dropwizard.io/2.2.0/apidocs/com/yammer/metrics/core/Meter.html > ). > > Jun > > On Wed, Jun 10, 2020 at 10:38 AM Boyang Chen > wrote: > > > Thanks Jun for the suggestions! I have addr

[jira] [Created] (KAFKA-10139) Add operational guide for failure recovery

2020-06-10 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10139: --- Summary: Add operational guide for failure recovery Key: KAFKA-10139 URL: https://issues.apache.org/jira/browse/KAFKA-10139 Project: Kafka Issue Type: Sub

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-10 Thread Boyang Chen
cking the current unique client set > somehow. An alternative approach is to maintain a > num-requests-redirected-rate metric with a client tag. > The clientId tag approach makes sense, will add to the KIP. Jun > > > > On Mon, Jun 8, 2020 at 9:36 AM Boyang Chen > wrote: > > &g

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

2020-06-10 Thread Boyang Chen
t; > > > Hi Jason, > > > > > > > > > > > > > > > > It's amazing to see this coming together :) > > > > > > > > > > > > > > > > I haven't had a chance to read in detail, but I read the > > outline &g

[jira] [Created] (KAFKA-10135) Extract Task#executeAndMaybeSwallow to be a general utility function into TaskManager

2020-06-09 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10135: --- Summary: Extract Task#executeAndMaybeSwallow to be a general utility function into TaskManager Key: KAFKA-10135 URL: https://issues.apache.org/jira/browse/KAFKA-10135

[jira] [Created] (KAFKA-10130) Rewrite ZkData struct with auto-generated protocol

2020-06-09 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-10130: --- Summary: Rewrite ZkData struct with auto-generated protocol Key: KAFKA-10130 URL: https://issues.apache.org/jira/browse/KAFKA-10130 Project: Kafka Issue Type

[VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-08 Thread Boyang Chen
Hey all, I would like to start the vote for KIP-590: https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller Thanks!

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-08 Thread Boyang Chen
Hey there, If no more question is raised, I will go ahead and start the vote shortly. On Thu, Jun 4, 2020 at 12:39 PM Boyang Chen wrote: > Hey there, > > bumping this thread for any further KIP-590 discussion, since it's been > quiet for a while. > > Boyang > > On Thu,

[jira] [Resolved] (KAFKA-10097) Avoid passing task checkpoint file externally

2020-06-06 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10097?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-10097. - Resolution: Fixed > Avoid passing task checkpoint file externa

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-04 Thread Boyang Chen
Hey there, bumping this thread for any further KIP-590 discussion, since it's been quiet for a while. Boyang On Thu, May 21, 2020 at 10:31 AM Boyang Chen wrote: > Thanks David, I agree the wording here is not clear, and the fellow broker > should just send a new CreateTopicR

  1   2   3   4   5   6   7   >