Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread David Jacot
+1 (non-binding) Thanks for the KIP, Anna! On Tue, May 19, 2020 at 7:12 AM Satish Duggana wrote: > +1 (non-binding) > Thanks Anna for the nice feature to control the connection creation rate > from the clients. > > On Tue, May 19, 2020 at 8:16 AM Gwen Shapira wrote: > > > +1 (binding) > > > >

Build failed in Jenkins: kafka-trunk-jdk11 #1474

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10004: ConfigCommand fails to find default broker configs without -- [...truncated 3.09 MB...] org.apache.kafka.streams.TopologyTestDriverTes

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-18 Thread Yuriy Badalyantc
Hi John, Your suggestion looks interesting. I think it's technically doable. But I'm not sure that this is the better solution. I will try to explain. From the scala developers' perspective, `Serde` looks really like a typeclass. Typical typeclass in pure scala will look like this: ``` trait Serd

Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Satish Duggana
+1 (non-binding) Thanks Anna for the nice feature to control the connection creation rate from the clients. On Tue, May 19, 2020 at 8:16 AM Gwen Shapira wrote: > +1 (binding) > > Thank you for driving this, Anna > > On Mon, May 18, 2020 at 4:55 PM Anna Povzner wrote: > > > Hi All, > > > > I wou

Jenkins build is back to normal : kafka-trunk-jdk14 #101

2020-05-18 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk11 #1473

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] MINOR: updated MacOS compatibility statement for RocksDB (#8687) [github] KIP-551: Expose disk read and write metrics (#8569) [github] MINOR: consolidate processor context for active/stan

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-18 Thread John Roesler
Hi Yuriy, Thanks so much for the KIP! I didn’t anticipate the problem you laid out in the KIP, but I find it very plausible. Thanks for pushing back on the “convention” and raising the issue, and also volunteering a solution! I’m wondering if we can “fix” it in one shot by just deprecating the

[jira] [Created] (KAFKA-10020) KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-18 Thread Yuriy Badalyantc (Jira)
Yuriy Badalyantc created KAFKA-10020: Summary: KIP-616: Rename implicit Serdes instances in kafka-streams-scala Key: KAFKA-10020 URL: https://issues.apache.org/jira/browse/KAFKA-10020 Project: Kaf

[DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-18 Thread Yuriy Badalyantc
Hi, I would like to propose KIP-616 to fix naming clash in the kafka streams scala API: https://cwiki.apache.org/confluence/display/KAFKA/KIP-616%3A+Rename+implicit+Serdes+instances+in+kafka-streams-scala Looking forward to your feedback. -Yuriy

[jira] [Created] (KAFKA-10019) MirrorMaker 2 did not function properly after restart (message lost, messages arriving slowly)

2020-05-18 Thread Kay (Jira)
Kay created KAFKA-10019: --- Summary: MirrorMaker 2 did not function properly after restart (message lost, messages arriving slowly) Key: KAFKA-10019 URL: https://issues.apache.org/jira/browse/KAFKA-10019 Project:

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-05-18 Thread Guozhang Wang
Bruno, Matthias: Thanks for your inputs. After some thoughts I've decide to update my proposal in the following way: 1. Store#serdes() would return a "Map" 2. Topology's description would be independent of whether it is generated from `StreamsBuilder#build(props)` or `StreamsBuilder#build()`, an

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-05-18 Thread Guozhang Wang
Hello Andy, Thanks a lot for your comments! I do not mind at all :) I think that's a valid point, what I have in mind is to expose an interface which can be optionally overridden in the overridden describe() call: Topology#describe(final TopologyDescriber) Interface TopologyDescriber { def

Build failed in Jenkins: kafka-trunk-jdk8 #4542

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10004: ConfigCommand fails to find default broker configs without -- [...truncated 3.08 MB...] org.apache.kafka.streams.TopologyTestDriverTes

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Aakash Shah
Hi Randall, I really appreciate the highly detailed explanation. It clears up the advantages of an asynchronous design using Futures, specifically because get() does not necessarily need to be called due to the guarantee put in place by the framework that you mentioned. I think that if this guaran

Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Gwen Shapira
+1 (binding) Thank you for driving this, Anna On Mon, May 18, 2020 at 4:55 PM Anna Povzner wrote: > Hi All, > > I would like to start the vote on KIP-612: Ability to limit connection > creation rate on brokers. > > For reference, here is the KIP wiki: > > https://cwiki.apache.org/confluence/dis

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Konstantine Karantasis
Thanks for the detailed explanation Randall. I think it highlights nicely how the common practice of overlapping communication with computation (or other communication) in concurrent systems can be useful and practical in this case. I also agree with the amendment around `preCommit` and the guara

Re: [VOTE] KIP-572: Improve timeouts and retires in Kafka Streams

2020-05-18 Thread Matthias J. Sax
No worries Guozhang, any feedback is always very welcome! My reply is going to be a little longer... Sorry. > 1) There are some inconsistent statements in the proposal regarding what to > deprecated: The proposal of the KIP is to deprecate `retries` for producer, admin, and Streams. Maybe the c

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-18 Thread Konstantine Karantasis
Thanks Michael. I think it's useful to enable specialized message formatters by adding this interface to the public API. You have my vote: +1 (binding) Just a few optional comments below: 1. Would you mind adding the equivalent command line example in the places where you have an example output?

Build failed in Jenkins: kafka-trunk-jdk8 #4541

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] KIP-551: Expose disk read and write metrics (#8569) [github] MINOR: consolidate processor context for active/standby (#8669) -- [...truncated 6.15

Build failed in Jenkins: kafka-trunk-jdk11 #1472

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] Remove redundant TOC and introduction in Running Streams Applications -- [...truncated 4.81 MB...] kafka.controller.ReplicaStateMachineTest > test

[jira] [Created] (KAFKA-10018) Change sh to bash

2020-05-18 Thread jiamei xie (Jira)
jiamei xie created KAFKA-10018: -- Summary: Change sh to bash Key: KAFKA-10018 URL: https://issues.apache.org/jira/browse/KAFKA-10018 Project: Kafka Issue Type: Bug Components: admin

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2020-05-18 Thread Maulin Vasavada
+1 (non-binding) On Mon, May 18, 2020 at 9:41 AM Ryanne Dolan wrote: > Bump. Looks like we've got 6 non-binding votes and 1 binding. > > On Thu, Feb 20, 2020 at 11:25 AM Ning Zhang > wrote: > > > Hello committers, > > > > I am the author of the KIP-545 and if we still miss votes from the > > co

Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Harsha Ch
+1 (binding) On Mon, May 18 2020 at 4:54 PM, Anna Povzner < a...@confluent.io > wrote: > > > > Hi All, > > > > I would like to start the vote on KIP-612: Ability to limit connection > creation rate on brokers. > > > > For reference, here is the KIP wiki: > https://cwiki.apache.org/conflu

[VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Hi All, I would like to start the vote on KIP-612: Ability to limit connection creation rate on brokers. For reference, here is the KIP wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers And discussion thread: https://lists.apa

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
I realized the KIP freeze is on May 20. I will start the voting thread now. On Mon, May 18, 2020 at 3:19 PM Anna Povzner wrote: > Thanks everyone for the feedback. I will start a voting thread tomorrow > morning if there are no more comments. > > Regards, > Anna > > On Mon, May 18, 2020 at 2:06

Build failed in Jenkins: kafka-trunk-jdk14 #100

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] KIP-551: Expose disk read and write metrics (#8569) [github] MINOR: consolidate processor context for active/standby (#8669) -- [...truncated 3.09

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Randall Hauch
Hi, Chris, Aakash, and others: First of all, apologies for the extremely long email. Secondly, thanks for the input on this KIP. The timing is unfortunately, but I do believe we're agreed on most points. Chris asked earlier: > I'm still unclear on how futures are going to provide any benefit to

Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-05-18 Thread Colin McCabe
On Mon, May 18, 2020, at 14:41, Cheng Tan wrote: > Dear Colin, > > > Thanks for the suggestions. > > > For example, if a new node joins the cluster, it will have 0 failed connect > > attempts, whereas the existing nodes will probably have more than 0. So > > all the clients will ignore every

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-18 Thread Maulin Vasavada
+1 (non-binding) On Mon, May 18, 2020 at 8:49 AM Mickael Maison wrote: > Bumping this thread as KIP freeze is approaching. > > It's a pretty small change and I have a PR ready: > https://github.com/apache/kafka/pull/8604 > > Thanks > > On Mon, May 4, 2020 at 5:26 PM Ryanne Dolan wrote: > > > >

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Thanks everyone for the feedback. I will start a voting thread tomorrow morning if there are no more comments. Regards, Anna On Mon, May 18, 2020 at 2:06 PM Anna Povzner wrote: > Hi Boyang, > > This KIP does not change the protocol with clients. The behavior is the > same as with KIP-402 where

Batch size, linger-ms not working as intended.

2020-05-18 Thread Bibek Shrestha
Hi All, I am working on a research work where I have to tweak a few things about how Kafka works. Goal: Suppose I am to send messages to the broker. Let us consider that each alphabet is a single message that needs to be sent (A-Z). My goal is to be able to send the message in such a way that I c

Build failed in Jenkins: kafka-trunk-jdk8 #4540

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] MINOR: updated MacOS compatibility statement for RocksDB (#8687) -- [...truncated 3.07 MB...] org.apache.kafka.streams.TopologyTestDriverTest > sho

Build failed in Jenkins: kafka-trunk-jdk14 #99

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] MINOR: updated MacOS compatibility statement for RocksDB (#8687) -- [...truncated 3.09 MB...] org.apache.kafka.streams.test.OutputVerifierTest > sh

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Aakash Shah
Hi Chris, I agree with your point. Randall, Konstantine, do you guys mind weighing in on any benefit of adding asynchronous functionality using a Future in the KIP right now? It seems to me that it only provides user control on when the thread will be blocked, and if we are going to process all t

Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-05-18 Thread Cheng Tan
Dear Colin, Thanks for the suggestions. > For example, if a new node joins the cluster, it will have 0 failed connect > attempts, whereas the existing nodes will probably have more than 0. So all > the clients will ignore every other node and pile on to the new one. That's > not good The

[jira] [Resolved] (KAFKA-9292) KIP-551: Expose disk read and write metrics

2020-05-18 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9292?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9292. - Fix Version/s: 2.6.0 Resolution: Fixed > KIP-551: Expose disk read and write metrics > ---

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Hi Boyang, This KIP does not change the protocol with clients. The behavior is the same as with KIP-402 where the broker delays accepting new connections when the limit for the number of connections is reached. This KIP adds another reason for the delay (when the rate is reached). Similarly, when

[jira] [Created] (KAFKA-10017) Flaky Test EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]

2020-05-18 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-10017: --- Summary: Flaky Test EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true] Key: KAFKA-10017 URL: https://issues.apache.org/jira/browse/KAFKA-10017

Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-05-18 Thread Colin McCabe
Hi Cheng, socket.connection.setup.timeout.ms seems more consistent with our existing configuration names than socket.connections.setup.timeout.ms (with an s). What do you think? > If no connected or connecting node exists, provide the disconnected node which > respects the reconnect backoff wi

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Nikolay Izhikov
Hello, Colin We need hack only because TLSv1.3 not supported in java8. > Java 8 will receive TLS 1.3 support later this year > (https://java.com/en/jre-jdk-cryptoroadmap.html) We can 1. Enable TLSv1.3 for java11 for now. And after java8 get TLSv1.3 support remove it. 2. Or we can wait and e

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Chris Egerton
Hi Aakash, Yep, that's pretty much it. I'd also like to emphasize that we should be identifying practical use cases for whatever API we provide. Giving developers a future that can be made synchronous with little effort seems flexible, but if that's all that developers are going to do with it anyw

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Aakash Shah
Hi all, Chris, I see your points about whether Futures provide much benefit at all as they are not truly fully asynchronous. Correct me if I am wrong, but I think what you are trying to point out is that if we have the option to add additional functionality later (in a simpler way too since we ar

Re: [DISCUSS] Kafka 3.0

2020-05-18 Thread Boyang Chen
One more thing I would like to see deprecated (hopefully no one mentioned before) is the zk based consumer offset support. On Mon, May 11, 2020 at 2:15 PM Colin McCabe wrote: > Hi Michael, > > It would be better to discuss the background behind KIP-500 in a separate > thread, since this thread i

[jira] [Created] (KAFKA-10016) Support For Purge Topic

2020-05-18 Thread David Mollitor (Jira)
David Mollitor created KAFKA-10016: -- Summary: Support For Purge Topic Key: KAFKA-10016 URL: https://issues.apache.org/jira/browse/KAFKA-10016 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Boyang Chen
Hey Anna, thanks for the KIP. Will this change be applied as one type of quota violation, which for client side should be retriable? For EOS model before 2.6, the Streams client creates one producer for each input partition, so it is actually possible to create thousands of producers when the serv

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Ismael Juma
Yeah, agreed. One option is to actually only change this in Apache Kafka 3.0 and avoid the hack altogether. We could make TLS 1.3 the default and have 1.2 as one of the enabled protocols. Ismael On Mon, May 18, 2020 at 12:24 PM Colin McCabe wrote: > Hmm. It would be good to figure out if we ar

Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-18 Thread Colin McCabe
Hi Rajini, I think the idea behind the 10 second default is that if you have three Kafka nodes A, B, C (or whatever), and you can't talk to A within 10 seconds, you'll try again with B or C, and still have plenty of time left over. Whereas currently, if your connection hangs while trying to co

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Chris Egerton
Hi Aakash, I asked this earlier about whether futures were the right way to go, if we wanted to enable asynchronous behavior at all: > I'm still unclear on how futures are going to provide any benefit to developers, though. Blocking on the return of such a future slightly later on in the process

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Colin McCabe
Hmm. It would be good to figure out if we are going to remove this compatibility hack in the next major release of Kafka? In other words, in Kafka 3.0, will we enable TLS 1.3 by default even if the cipher suite is specified? best, Colin On Mon, May 18, 2020, at 09:26, Ismael Juma wrote: > S

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-18 Thread Aakash Shah
Hi Arjun, Thanks for your feedback. I agree with moving to Future, those are good points. I believe an earlier point made for asynchronous functionality were that modern APIs tend to be asynchronous as they result in more expressive and better defined APIs. Additionally, because a lot of Kafka C

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Hi Alexandre, Thanks for your comments. My answers are below: 900. The KIP does not propose any new metrics because we already have metrics that will let us monitor connection attempts and the amount of time the broker delays accepting new connections: 1. We have a per-listener (and per-processor

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Ismael Juma
Hey Colin, Note that the clients expose "Map metrics()". So, some things are already exposed. I generally agree that we should be very clear about the new classes we are exposing as public APIs via this. Ismael On Mon, May 18, 2020 at 11:29 AM Colin McCabe wrote: > KafkaMetrics isn't a public

Build failed in Jenkins: kafka-trunk-jdk8 #4539

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] Remove redundant TOC and introduction in Running Streams Applications -- [...truncated 3.07 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-18 Thread Jun Rao
Hi, Satish, Thanks for the update. Just to clarify. Which doc has the latest updates, the wiki or the google doc? Jun On Thu, May 14, 2020 at 10:38 AM Satish Duggana wrote: > Hi Jun, > Thanks for your comments. We updated the KIP with more details. > > >100. For each of the operations related

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Colin McCabe
Hi Zhiguo, I think it's really important to distinguish between "a class being public" and "a class being part of Kafka's public API." These really are two completely different concepts, unfortunately. The best explanation of interface annotations is probably here: https://github.com/apache/k

Build failed in Jenkins: kafka-trunk-jdk14 #98

2020-05-18 Thread Apache Jenkins Server
See Changes: [github] Remove redundant TOC and introduction in Running Streams Applications -- [...truncated 3.09 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Zhiguo Huang
The reason we have this KIP is that allow public AK to take advantage of this such as AclAuthorizer. Jeff On Mon, May 18, 2020 at 11:29 AM Colin McCabe wrote: > KafkaMetrics isn't a public API currently. Are we willing to make it > one? I think this is a very big change, if so. > > This affect

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Zhiguo Huang
Hi Ismael, The purpose for this KIP allows plugins to use the same Metrics instance from the broker without recreating it. Metrics is public class so we could use it. Current Metrics is not implementing an interface related to any metrics. Expose all fields through methods through interface is big

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Colin McCabe
KafkaMetrics isn't a public API currently. Are we willing to make it one? I think this is a very big change, if so. This affects a huge number of classes. MetricConfig, MetricReporter, MetricName, Sensor, KafkaMetric, and probably more I'm forgetting would need to become public APIs that we

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Ismael Juma
Hi Jeff, I think the idea is good, but I have 2 questions/comments: 1. The monitor method is documented to "get" metrics, but it takes a `Metrics` instance. Either the method name/signature needs to be adjusted or the documentation. 2. Instead of exposing the Metrics class, which is currently not

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Rajini Sivaram
+1 (binding) Thanks for the KIP, Jeff! Regards, Rajini On Mon, May 18, 2020 at 6:41 PM Jason Gustafson wrote: > +1 This looks useful. > > -Jason > > On Mon, May 18, 2020 at 9:59 AM Guozhang Wang wrote: > > > Zhiguo, thanks for the KIP. +1 from me. > > > > > > Guozhang > > > > On Fri, May 15,

[jira] [Resolved] (KAFKA-9994) Catch TaskMigrated exception in task corruption code path

2020-05-18 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-9994. - Resolution: Fixed > Catch TaskMigrated exception in task corruption code path >

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Jason Gustafson
+1 This looks useful. -Jason On Mon, May 18, 2020 at 9:59 AM Guozhang Wang wrote: > Zhiguo, thanks for the KIP. +1 from me. > > > Guozhang > > On Fri, May 15, 2020 at 2:21 PM Zhiguo Huang > wrote: > > > Thanks to everyone for their input. I've incorporated the changes, and I > > think this is

[jira] [Resolved] (KAFKA-10008) Symbol not found when running Kafka Streams with RocksDB dependency on MacOS 10.13.6

2020-05-18 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-10008. -- Resolution: Won't Fix Following the other comments, we've decided just to document it. > Symb

Re: [VOTE] KIP-606: Add Metadata Context to MetricsReporter

2020-05-18 Thread Xavier Léauté
Thank you everyone, KIP-606 has been adopted with 4 +1 binding votes. Xavier On Fri, May 15, 2020 at 10:50 AM Bill Bejeck wrote: > Thanks for the proposal, Xavier. > > +1(binding) > > -Bill > > On Fri, May 15, 2020 at 1:41 PM John Roesler wrote: > > > Thanks for the KIP, Xavier! > > > > I'm

Re: [DISCUSS] KIP-589 Add API to Update Replica State in Controller

2020-05-18 Thread David Arthur
I've updated the KIP with the feedback from this discussion https://cwiki.apache.org/confluence/display/KAFKA/KIP-589+Add+API+to+update+Replica+state+in+Controller. I'll send out the vote thread shortly. Thanks again, David On Tue, May 5, 2020 at 10:34 AM Tom Bentley wrote: > Hi Colin, > > Yeah

Re: [VOTE] KIP-608: Expose Kafka Metrics in Authorizer

2020-05-18 Thread Guozhang Wang
Zhiguo, thanks for the KIP. +1 from me. Guozhang On Fri, May 15, 2020 at 2:21 PM Zhiguo Huang wrote: > Thanks to everyone for their input. I've incorporated the changes, and I > think this is ready for voting. > > To summarize, the KIP simply proposes to add a feature which expose > instance o

Re: [VOTE] KIP-572: Improve timeouts and retires in Kafka Streams

2020-05-18 Thread Guozhang Wang
Hi Matthias, Sorry for flooding the thread, but with this KIP I feel the design scope of https://issues.apache.org/jira/browse/KAFKA-6520 can be simplified a lot and may it the design can be just piggy-backed as part of this KIP, wdyt? Guozhang On Mon, May 18, 2020 at 9:47 AM Guozhang Wang wro

Re: [VOTE] KIP-572: Improve timeouts and retires in Kafka Streams

2020-05-18 Thread Guozhang Wang
Hi Matthias, Just to add one more meta comment: for consumer, if it gets a TimeoutException polling records, would start timing all tasks since that single consumer would affect all tasks? For other blocking calls like `endOffsets()` etc, they are usually also issued on behalf of a batch of tasks,

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2020-05-18 Thread Ryanne Dolan
Bump. Looks like we've got 6 non-binding votes and 1 binding. On Thu, Feb 20, 2020 at 11:25 AM Ning Zhang wrote: > Hello committers, > > I am the author of the KIP-545 and if we still miss votes from the > committers, please review the KIP and vote for it, so that the > corresponding PR will be

Re: [DISCUSS] KIP-613: Add end-to-end latency metrics to Streams

2020-05-18 Thread Guozhang Wang
Thanks Sophie, I'm +1 on the updated KIP. On Mon, May 18, 2020 at 8:32 AM Sophie Blee-Goldman wrote: > @Bruno > No, this would be after inserting in the cache/state store/etc -- basically > once > all operations have completed such that the record would be present if > queried after that time. >

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Ismael Juma
Sounds good. Ismael On Mon, May 18, 2020, 9:03 AM Nikolay Izhikov wrote: > > A safer approach may be to only add TLS 1.3 to the list if the cipher > suite config has not been specified. > > So, if TLS 1.3 is added to the list by Kafka, it would seem that it > would not work if the user specifi

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Ismael Juma
It's not ok for things to break for Java 8 users when they upgrade (even if they can fix it by changing a config). So, I think we need to change the KIP to offer more dynamic behavior: only enable TLS 1.3 if it's safe. Ismael On Mon, May 18, 2020, 8:59 AM Nikolay Izhikov wrote: > > 1. I meant t

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Nikolay Izhikov
> A safer approach may be to only add TLS 1.3 to the list if the cipher suite > config has not been specified. > So, if TLS 1.3 is added to the list by Kafka, it would seem that it would not > work if the user specified a list of cipher suites for previous TLS versions Let’s just add test for th

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Nikolay Izhikov
> 1. I meant that `ssl.protocol` is TLSv1.2 while `ssl.enabled.protocols` is > `TLSv1.2, TLSv1.3`. How do these two configs interact `ssl.protocol` is what will be used, by default, in this KIP is stays unchanged (TLSv1.2) Please, see [1] `ssl.enabled.protocols` is list of protocols that *can*

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Rajini Sivaram
Hi Anna, Thanks for the response, sounds good. Regards, Rajini On Sun, May 17, 2020 at 1:38 AM Anna Povzner wrote: > Hi Rajini, > > Thanks for reviewing the KIP! > > I agree with your suggestion to make per-IP connection rate quota a dynamic > quota for entity name IP. This will allow config

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-18 Thread Mickael Maison
Bumping this thread as KIP freeze is approaching. It's a pretty small change and I have a PR ready: https://github.com/apache/kafka/pull/8604 Thanks On Mon, May 4, 2020 at 5:26 PM Ryanne Dolan wrote: > > +1, non-binding > > On Mon, May 4, 2020, 9:24 AM Christopher Egerton > wrote: > > > +1 (no

Re: [DISCUSS] KIP-613: Add end-to-end latency metrics to Streams

2020-05-18 Thread Sophie Blee-Goldman
@Bruno No, this would be after inserting in the cache/state store/etc -- basically once all operations have completed such that the record would be present if queried after that time. @Guozhang I'm glad you suggested the TRACE level. It seems we've been heading in that direction for a while so I'm

[jira] [Created] (KAFKA-10015) React to Unexpected Errors on Stream Threads

2020-05-18 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10015: - Summary: React to Unexpected Errors on Stream Threads Key: KAFKA-10015 URL: https://issues.apache.org/jira/browse/KAFKA-10015 Project: Kafka Issue Type: Im

Re: [DISCUSS] KIP-609: Use Pre-registration and Blocking Calls for Better Transaction Efficiency

2020-05-18 Thread Boyang Chen
Oh, I see your point! Will add that context to the KIP. Boyang On Sun, May 17, 2020 at 11:39 AM Guozhang Wang wrote: > My point here is only for the first AddPartitionToTxn request of the > transaction, since only that request would potentially be blocked on the > previous txn to complete. By d

Re: [VOTE] KIP-596 Safely abort Producer transactions during application shutdown

2020-05-18 Thread Boyang Chen
Hey Xiang, we already got 3 binding votes from Matthias, John and Guozhang. The KIP is approved and you should be good to go :) On Fri, May 15, 2020 at 6:05 PM Guozhang Wang wrote: > +1. Thanks Xiang! > > > Guozhang > > On Fri, May 15, 2020 at 11:37 AM John Roesler wrote: > > > Thanks for the

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Ismael Juma
To be more concrete, here are the ciphers supported by TLS 1.3: TLS_AES_256_GCM_SHA384:TLS_CHACHA20_POLY1305_SHA256:TLS_AES_128_GCM_SHA256:TLS_AES_128_CCM_SHA256 Compare with TLS 1.2: ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:ECDHE-ECDSA-CHACHA20-POLY1305:ECDHE-RSA-CHACHA20-POLY

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Ismael Juma
Nikolay, Thanks for the comments. More below: 1. I meant that `ssl.protocol` is TLSv1.2 while `ssl.enabled.protocols` is `TLSv1.2, TLSv1.3`. How do these two configs interact? 2. My question is not about obsolete protocols, it is about people using TLS 1.2 with specified cipher suites. How will t

Re: MM2 Message Handlers

2020-05-18 Thread Jamie
Hi Ryanne, Great, thanks  Jamie Sent from AOL Mobile Mail Get the new AOL app: mail.mobile.aol.com On Monday, 18 May 2020, Ryanne Dolan  wrote: Jamie, MM2 uses Connect, so you can use Connect's SMTs for the same effect. Ryanne On Mon, May 18, 2020, 6:17 AM Jamie wrote: > Hi All, > Does MM2 c

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-05-18 Thread Ismael Juma
Sounds good, that's the one I meant to use. :) Ismael On Mon, May 18, 2020, 6:34 AM Nikolay Izhikov wrote: > Hello, Ismael. > > I think we should move ongoing discussion into KIP-573 discussion [1] > > I will respond here and is KIP-573 discussion thread, because, this KIP > already adopted by

Re: [DISCUSS] KIP-573: Enable TLSv1.3 by default

2020-05-18 Thread Nikolay Izhikov
Hello, Ismael. Here is answers to your questions: > Quick question, the following is meant to include TLSv1.3 as well, right? > Change the value of the SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS to «TLSv1.2» I propose to have the following value SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS = «TLSv1.

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-05-18 Thread Nikolay Izhikov
Hello, Ismael. I think we should move ongoing discussion into KIP-573 discussion [1] I will respond here and is KIP-573 discussion thread, because, this KIP already adopted by [2] [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-573%3A+Enable+TLSv1.3+by+default [2] https://github.com

Re: MM2 Message Handlers

2020-05-18 Thread Ryanne Dolan
Jamie, MM2 uses Connect, so you can use Connect's SMTs for the same effect. Ryanne On Mon, May 18, 2020, 6:17 AM Jamie wrote: > Hi All, > Does MM2 currently support message handlers, the same way MM1 did? If not, > are there any plans to support this in future? > Many Thanks, > Jamie

MM2 Message Handlers

2020-05-18 Thread Jamie
Hi All, Does MM2 currently support message handlers, the same way MM1 did? If not, are there any plans to support this in future? Many Thanks,  Jamie

Re: [DISCUSS] KIP-613: Add end-to-end latency metrics to Streams

2020-05-18 Thread Bruno Cadonna
Hi Sophie, Just a question for my understanding. When you say " t_A be the time when it is finished being processed by the aggregator node", this means before caching (or putting into the state store if caching is disabled), right? Best, Bruno On Sat, May 16, 2020 at 6:57 PM Guozhang Wang wrote

[jira] [Created] (KAFKA-10014) Always try to close all channels in Selector#close

2020-05-18 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-10014: -- Summary: Always try to close all channels in Selector#close Key: KAFKA-10014 URL: https://issues.apache.org/jira/browse/KAFKA-10014 Project: Kafka Issue

Re: [VOTE] KIP-572: Improve timeouts and retires in Kafka Streams

2020-05-18 Thread Bruno Cadonna
Hi Matthias, I am +1 (non-binding) on the KIP. Just one final remark: Wouldn't it be better to specify task.timeout.ms to -1 if no retry should be done? IMO it would make the config more intuitive because 0 would not have two possible meanings (i.e. try once and never try) anymore. Best, Bruno

Re: Consumer hang-up in case of unclean leader election

2020-05-18 Thread Dmitry Sorokin
Hi! Done: https://issues.apache.org/jira/browse/KAFKA-10013 пт, 15 мая 2020 г. в 18:36, Sophie Blee-Goldman : > Hey Dmitry, > > Can you open a ticket at https://issues.apache.org/jira/issues/ and > include > all this information so we can track and look into it? > > Thanks! > Sophie > > On Fri, M

[jira] [Created] (KAFKA-10013) Consumer hang-up in case of unclean leader election

2020-05-18 Thread Dmitry (Jira)
Dmitry created KAFKA-10013: -- Summary: Consumer hang-up in case of unclean leader election Key: KAFKA-10013 URL: https://issues.apache.org/jira/browse/KAFKA-10013 Project: Kafka Issue Type: Bug