Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-08-08 Thread Boyang Chen
Thank you Jason. We had some offline discussion on properly keeping group metadata up to date, and here are some of our options brainstormed: 1. Let the caller of `sendOffsetsToTransaction(offset, metadata)` maintain the ever-changing group metadata. This could be done on stream side, but for non-s

Re: [DISCUSS] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-08-08 Thread Arjun Satish
Cool! Couple of nits: - In public interfaces, typo: *json.decimal.serialization.fromat* - In public interfaces, you use the term "HEX" instead of "BASE64". On Wed, Aug 7, 2019 at 9:51 AM Almog Gavra wrote: > EDIT: everywhere I've been using "HEX" I meant to be using "BASE64". I will > update

[DISCUSS] KIP-505 : Add new public method to only update assignment metadata in consumer

2019-08-08 Thread Jungtaek Lim
Hi devs, I'd like to initiate discussion around KIP-505, exposing new public method to only update assignment metadata in consumer. `poll(0)` has been misused as according to Kafka doc it doesn't guarantee that it doesn't pull any records, and new method `poll(Duration)` doesn't have same semanti

[jira] [Created] (KAFKA-8776) Add new public method to only update assignment metadata in consumer

2019-08-08 Thread Jungtaek Lim (JIRA)
Jungtaek Lim created KAFKA-8776: --- Summary: Add new public method to only update assignment metadata in consumer Key: KAFKA-8776 URL: https://issues.apache.org/jira/browse/KAFKA-8776 Project: Kafka

Re: [DISCUSS] KIP-401 TransformerSupplier/ProcessorSupplier enhancements

2019-08-08 Thread Paul Whalen
Matthias, You did summarize my thinking correctly, thanks for writing it out. I think the disconnect on opinion is due to a couple things influenced by my habits while writing streams code: 1) I don't see state stores that are "individually owned" versus "shared" as that much different at all, a

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2019-08-08 Thread Vito Jeng
Thanks, Matthias! > About `StreamThreadNotStartedException`: Thank you for explanation. I agree with your opinion. `CompositeReadOnlyXxxStore#get()` would never throw `StreamThreadNotStartedException`. For the case that corresponding thread crashes after we handed out the store handle. We may th

Re: [VOTE] KIP-499 - Unify connection name flag for command line tool

2019-08-08 Thread Harsha Chintalapani
+1 (binding). much needed!! On Thu, Aug 08, 2019 at 6:43 PM, Gwen Shapira wrote: > +1 (binding) THANK YOU. It would be +100 if I could. > > On Thu, Aug 8, 2019 at 6:37 PM Mitchell wrote: > > Hello Dev, > After the discussion I would like to start the vote for KIP-499 > > The following command

Re: Ask for contributor access and write permission on wiki

2019-08-08 Thread Jungtaek Lim
Thanks Harsha for granting permission, now I can create subpage on wiki. -Jungtaek Lim (HeartSaVioR) On Fri, Aug 9, 2019 at 11:22 AM Harsha Chintalapani wrote: > Hi Jungtaek, > Gave you permissions on wiki. Please check. > Thansk, > Harsha > > > On Thu, Aug 08, 2019 at 7:03 PM, Jun

Re: Ask for contributor access and write permission on wiki

2019-08-08 Thread Harsha Chintalapani
Hi Jungtaek, Gave you permissions on wiki. Please check. Thansk, Harsha On Thu, Aug 08, 2019 at 7:03 PM, Jungtaek Lim wrote: > Hi devs, > > I'd like to give a shot to make first contribution on Kafka community, as > I initiated thread on needs a new public API for metadata update o

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

2019-08-08 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8179: Part 3, Add PartitionsLost API for resetGenerations and -- [...truncated 2.59 MB...] org.apache.kafka.trogdor.agent.AgentTest > testAgent

[jira] [Created] (KAFKA-8775) test case pretty happen error the errMsg that zookeeper start failure while execute test case

2019-08-08 Thread zedong.Huang (JIRA)
zedong.Huang created KAFKA-8775: --- Summary: test case pretty happen error the errMsg that zookeeper start failure while execute test case Key: KAFKA-8775 URL: https://issues.apache.org/jira/browse/KAFKA-8775

Ask for contributor access and write permission on wiki

2019-08-08 Thread Jungtaek Lim
Hi devs, I'd like to give a shot to make first contribution on Kafka community, as I initiated thread on needs a new public API for metadata update only [1]. Could you please grant me contributor in JIRA as well as write permission on wiki page? Thanks in advance! Jungtaek Lim (HeartSaVioR) 1,

Re: Alternative of poll(0) without pulling records

2019-08-08 Thread Jungtaek Lim
Thanks Viktor for guiding me through this! I would initiate new thread to ask edit permission on wiki. Once I got permission I'll come up with simple KIP page and initiate discussion thread. Thanks again, Jungtaek Lim On Thu, Aug 8, 2019 at 9:42 PM Viktor Somogyi-Vass wrote: > Hey Jungtaek, >

Re: [VOTE] KIP-499 - Unify connection name flag for command line tool

2019-08-08 Thread Gwen Shapira
+1 (binding) THANK YOU. It would be +100 if I could. On Thu, Aug 8, 2019 at 6:37 PM Mitchell wrote: > > Hello Dev, > After the discussion I would like to start the vote for KIP-499 > > > The following command line tools will have the `--bootstrap-server` > command line argument added: kafka-conso

[VOTE] KIP-499 - Unify connection name flag for command line tool

2019-08-08 Thread Mitchell
Hello Dev, After the discussion I would like to start the vote for KIP-499 The following command line tools will have the `--bootstrap-server` command line argument added: kafka-console-producer.sh, kafka-consumer-groups.sh, kafka-consumer-perf-test.sh, kafka-verifiable-consumer.sh, kafka-verifia

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

2019-08-08 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8179: Part 3, Add PartitionsLost API for resetGenerations and -- [...truncated 2.58 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

[jira] [Created] (KAFKA-8774) Connect REST API exposes plaintext secrets in tasks endpoint

2019-08-08 Thread Oleksandr Diachenko (JIRA)
Oleksandr Diachenko created KAFKA-8774: -- Summary: Connect REST API exposes plaintext secrets in tasks endpoint Key: KAFKA-8774 URL: https://issues.apache.org/jira/browse/KAFKA-8774 Project: Kafka

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-08-08 Thread Guozhang Wang
I think I agree with you Sophie. My gut feeling is that 1) it should not be the major concern in assignor's algorithm for standby tasks not catching up, but rather be tackled in different modules, and 2) a lot of optimization can be down at the stream thread itself, like dedicated threading and lar

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-08-08 Thread Sophie Blee-Goldman
> we may have other ways to not starving the standby tasks, for example, by > using dedicate threads for standby tasks or even consider having *higher> priority for standby than active* so that we always try to caught up standby > first, then process active This is an interesting idea, but seems l

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-08-08 Thread Sophie Blee-Goldman
Stateful tasks with logging disabled seem to be an interesting edge case. On the one hand, for balancing purposes they should be considered stateful since as Guozhang pointed out they are still "heavy" in IO costs. But for "catching up" purposes, ie when allocating standby tasks that will become ac

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Maulin Vasavada
Hi Harsha Let me try to write samples and will let you know. Thanks Maulin On Thu, Aug 8, 2019 at 4:00 PM Harsha Ch wrote: > Hi Maulin, > With java security providers can be as custom you would like it to > be. If you only want to to implement a custom way of loading the > keystore an

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Harsha Ch
Hi Maulin, With java security providers can be as custom you would like it to be. If you only want to to implement a custom way of loading the keystore and truststore and not implement any protocol/encryption handling you can leave them empty and no need to implement. Have you looked into

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Maulin Vasavada
Hi Colin, To your point - "Also, it seems like most people who want a custom truststore / keystore will also want a custom SSL provider," I don't think so. Take our own example. We have a fairly large Kafka eco-system (500B+ messages a day flowing through with poly-glot client-base) with strict In

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Maulin Vasavada
Imagine a scenario like - We know we have a custom KMS and as a Kafka owner we want to comply to using that KMS source to load keys/certs. As a Kafka owner we know how to integrate with KMS but doesn't necessarily have to know anything about cipher suites, algorithms, and SSL/TLS implementation. Go

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-08-08 Thread Guozhang Wang
Regarding 3) above: I think for active task they should still be considered stateful since the processor would still pay IO cost accessing the store, but they would not have standby tasks? On Thu, Aug 8, 2019 at 7:49 AM Bruno Cadonna wrote: > Hi, > > Thank you for the KIP! > > Some questions/com

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Colin McCabe
Harsha made a good point that you can achieve your goals through KIP-492. Security configuration is starting to get pretty complex-- is there a reason not to use the existing configurations? Also, it seems like most people who want a custom truststore / keystore will also want a custom SSL pro

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-08 Thread Don Bosco Durai
Hi Rajini Thanks for clarifying. This is very helpful... #1 - On the Ranger side, we should be able to handle multiple requests at the same time. I was just not sure how much processing overhead will be there on the Broker side to split and then consolidate the results. If it is negligible, t

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Maulin Vasavada
Hi Harsha We don't have spire (or similar) agents and we do not have keys/certs locally on any brokers. To elaborate more on my previous email, I agree that Java security Providers are used in much broader sense - to have a particular implementation of an algorithm, use specific cipher suites for

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Harsha Chintalapani
Hi Maulin, On Thu, Aug 08, 2019 at 2:04 PM, Maulin Vasavada wrote: > Hi Harsha > > The reason we rejected the SslProvider route is that - we only needed a > custom way to load keys/certs. Not touch any policy that existing Providers > govern like SunJSSE Provider. > We have exactly the same req

Re: [DISCUSS] KIP-499 - Unify connection name flag for command line tool

2019-08-08 Thread Colin McCabe
I agree that limiting the scope of the KIP would be good. The configuration is actually bootstrap.servers with an S, though. I actually like --bootstrap-servers slightly better than --bootstrap-server, although I don't feel that strongly about either. ;) best, Colin On Thu, Aug 8, 2019, at 14

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-08 Thread Colin McCabe
Hi Rajini, Thanks for the KIP. This will be a great improvement. Why not just pass the cluster ID directly to Authorizer#start, rather than dealing with the ClusterResourceListener interface? That seems like it would be simpler. If authorizers don't need that information, they can ignore tha

[jira] [Resolved] (KAFKA-4600) Consumer proceeds on when ConsumerRebalanceListener fails

2019-08-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4600?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4600. -- Resolution: Fixed > Consumer proceeds on when ConsumerRebalanceListener fails > ---

[jira] [Resolved] (KAFKA-8493) Add PartitionsLost API in RebalanceListener (part 3)

2019-08-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8493?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-8493. -- Resolution: Fixed Fix Version/s: 2.4.0 > Add PartitionsLost API in RebalanceListener (pa

Re: [DISCUSS] KIP-499 - Unify connection name flag for command line tool

2019-08-08 Thread Jason Gustafson
@Dongjin Thanks, you raise some good points. I think the intent here is to try and fix one of the more egregious inconsistencies without increasing the scope too much. We tried the big KIP approach with KIP-14 before and I don't think we made much progress. I think it's reasonable to do this on a c

Re: KIP-352: Distinguish URPs caused by reassignment

2019-08-08 Thread George Li
Hi Jason, Can KIP-352 split the two metrics MaxLag and TotalLag for reassignment replication as well?  From the dashboard of these 2 metrics, one can see whether the replication is stuck (flat line) and estimate how long the reassignments will complete (how fast the Lag line is going down). T

[jira] [Created] (KAFKA-8773) Static membership protocol borks on re-used group id

2019-08-08 Thread Raman Gupta (JIRA)
Raman Gupta created KAFKA-8773: -- Summary: Static membership protocol borks on re-used group id Key: KAFKA-8773 URL: https://issues.apache.org/jira/browse/KAFKA-8773 Project: Kafka Issue Type: Bu

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Maulin Vasavada
Hi Harsha The reason we rejected the SslProvider route is that - we only needed a custom way to load keys/certs. Not touch any policy that existing Providers govern like SunJSSE Provider. The ask here is different than KIP-492. We don't have any need to modify/specify the algorithm parameter. Doe

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

2019-08-08 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update dependencies for Kafka 2.4 (#7126) -- [...truncated 2.60 MB...] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoo

Re: KIP-352: Distinguish URPs caused by reassignment

2019-08-08 Thread Jason Gustafson
Hey Stan, Thanks for the suggestion. I have updated the proposal to include two new meters for reassignment traffic inbound and outbound. -Jason On Thu, Aug 8, 2019 at 12:07 PM Stanislav Kozlovski wrote: > Agreed on not totally spitting the replication incoming/outgoing metric - > that could c

Jenkins build is back to normal : kafka-trunk-jdk11 #739

2019-08-08 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-8772) Flaky Test kafka.api.DelegationTokenEndToEndAuthorizationTest.testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl

2019-08-08 Thread Sophie Blee-Goldman (JIRA)
Sophie Blee-Goldman created KAFKA-8772: -- Summary: Flaky Test kafka.api.DelegationTokenEndToEndAuthorizationTest.testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl Key: KAFKA-8772 URL: https://issues.apache

[jira] [Resolved] (KAFKA-8763) Flaky Test SaslSslAdminClientIntegrationTest.testIncrementalAlterConfigsForLog4jLogLevels

2019-08-08 Thread Sophie Blee-Goldman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-8763. Resolution: Duplicate duplicate of 8756 > Flaky Test > SaslSslAdminClientIntegrat

[jira] [Created] (KAFKA-8771) Flaky Test SaslClientsWithInvalidCredentialsTest.testManualAssignmentConsumerWithAuthenticationFailure

2019-08-08 Thread Sophie Blee-Goldman (JIRA)
Sophie Blee-Goldman created KAFKA-8771: -- Summary: Flaky Test SaslClientsWithInvalidCredentialsTest.testManualAssignmentConsumerWithAuthenticationFailure Key: KAFKA-8771 URL: https://issues.apache.org/jira/bro

Re: KIP-352: Distinguish URPs caused by reassignment

2019-08-08 Thread Stanislav Kozlovski
Agreed on not totally spitting the replication incoming/outgoing metric - that could cause confusion. Just adding a new metric sounds good to me! The throttle follow-up is mentioned as part of future work in KIP-455 and I agree that it is way out of scope for this one. On Thu, Aug 8, 2019 at 8:0

Re: KIP-352: Distinguish URPs caused by reassignment

2019-08-08 Thread Jason Gustafson
Hi Stan, That's an interesting thought. I'm wondering if it would be better to leave the current replication metrics counting for the total replication traffic and add a new metric for reassignment traffic? By the way, a further KIP-455 follow-up that I won't attempt here would be to have a separ

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

2019-08-08 Thread Jun Rao
Hi, Justine, Thanks for the KIP. Overall, it seems to be a good improvement. However, I think Harsha's point seems reasonable. We had auto.create.topics.enable config on the broker to allow admins to disable topic creation from the producer/consumer clients before we had the security feature. The

Re: KIP-352: Distinguish URPs caused by reassignment

2019-08-08 Thread Stanislav Kozlovski
Hi Jason, I like the new ReassigningPartitions metric. Would it be easy to expand the scope of the KIP to split the ReplicationIncoming/Outgoing metric to distringuish between reassigning/non-reassigning traffic, or do you prefer to keep this KIP nice and small? On Thu, Aug 8, 2019 at 12:08 AM Ja

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

2019-08-08 Thread Colin McCabe
Hi Koushik, The vote for this KIP already passed. See https://www.mail-archive.com/dev@kafka.apache.org/msg99636.html best, Colin On Thu, Aug 8, 2019, at 10:50, Koushik Chitta wrote: > Thanks Colin, George. Can we restart the voting for this KIP. > > Thanks, > Koushik > > -Original Mes

RE: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

2019-08-08 Thread Koushik Chitta
Thanks Colin, George. Can we restart the voting for this KIP. Thanks, Koushik -Original Message- From: Colin McCabe Sent: Wednesday, August 7, 2019 5:17 PM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment On Wed, Aug 7, 201

[DISCUSS] Modularization of kafka client separating server related classes/interfaces

2019-08-08 Thread Satish Duggana
Hi, There are many classes in the client module that are not really related to the client. It is good to have common modules structure with respective classes/interfaces. Implementors/providers need to have dependency only on those modules instead of having a dependency on the client module. Belo

Re: [DISCUSS] KIP-503: deleted topics metric

2019-08-08 Thread Stanislav Kozlovski
What do people think if we exposed: * eligible topics/replicas pending delete * ineligible topics/replicas pending delete On Thu, Aug 8, 2019 at 5:16 PM David Arthur wrote: > It looks like topicsIneligibleForDeletion is a subset of topicsToBeDeleted > in the controller. > > On Thu, Aug 8, 2019 a

Re: [DISCUSS] KIP-503: deleted topics metric

2019-08-08 Thread David Arthur
It looks like topicsIneligibleForDeletion is a subset of topicsToBeDeleted in the controller. On Thu, Aug 8, 2019 at 11:16 AM Stanislav Kozlovski wrote: > ineligible replicas/topics are not included in the pending metrics, right? > If so, sounds good to me. > > On Thu, Aug 8, 2019 at 4:12 PM Dav

[jira] [Created] (KAFKA-8770) Either switch to or add an option for emit-on-change

2019-08-08 Thread John Roesler (JIRA)
John Roesler created KAFKA-8770: --- Summary: Either switch to or add an option for emit-on-change Key: KAFKA-8770 URL: https://issues.apache.org/jira/browse/KAFKA-8770 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-8769) Consider computing stream time independently per key

2019-08-08 Thread John Roesler (JIRA)
John Roesler created KAFKA-8769: --- Summary: Consider computing stream time independently per key Key: KAFKA-8769 URL: https://issues.apache.org/jira/browse/KAFKA-8769 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-503: deleted topics metric

2019-08-08 Thread Stanislav Kozlovski
ineligible replicas/topics are not included in the pending metrics, right? If so, sounds good to me. On Thu, Aug 8, 2019 at 4:12 PM David Arthur wrote: > Yes I think exposing ineligible topics would be useful as well. The > controller also tracks this ineligible state for replicas. Would that be

Re: [DISCUSS] KIP-503: deleted topics metric

2019-08-08 Thread David Arthur
Yes I think exposing ineligible topics would be useful as well. The controller also tracks this ineligible state for replicas. Would that be useful to expose as well? In that case, we'd be up to four new metrics: * topics pending delete * replicas pending delete * ineligible topics * ineligible re

[jira] [Resolved] (KAFKA-8578) Add Functionality to Expose RocksDB Metrics

2019-08-08 Thread Bruno Cadonna (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-8578. -- Resolution: Done > Add Functionality to Expose RocksDB Metrics > --

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-08-08 Thread Bruno Cadonna
Hi, Thank you for the KIP! Some questions/comments: 1. I am wondering if the "stand-by" tasks that catch up state before the active task is switched deserve its own name in this KIP and maybe in the code. We have already stated that they are not true stand-by tasks, they are not configured throu

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Harsha Chintalapani
In your KIP you added security. provider as rejected alternative and specified "its not the correct way". Do you mind explaining why its not? I didn't find any evidence in Java docs to say so. Contrary to your statement it does say in the java docs " However, please note that a provider can be used

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-08 Thread Harsha Chintalapani
Hi Maulin, Not sure if you looked at my previous replies. This changes are not required as there is already security Provider to do what you are proposing. This KIP https://cwiki.apache.org/confluence/display/KAFKA/KIP-492%3A+Add+java+security+providers+in+Kafka+Security+config also

Re: Alternative of poll(0) without pulling records

2019-08-08 Thread Viktor Somogyi-Vass
Hey Jungtaek, Thanks for your interest, sometimes I also think such an API would be a good thing. I don't see any strong reasons neither in KIP-288 nor in KIP-266 why such an API shouldn't be created, so go ahead with it, although you'll need to create a short KIP for this as the KafkaConsumer cla

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-08 Thread Satish Duggana
Hi Rajini, Sure, I will start a discussion thread soon on dev mailing list. Thanks, Satish. On Thu, Aug 8, 2019 at 1:29 AM Rajini Sivaram wrote: > > Hi Ron/Harsha/Satish, > > Thanks for reviewing the KIP! > > We should perhaps have a wider discussion outside this KIP for refactoring > clients so

[jira] [Created] (KAFKA-8768) Replace DeleteRecords request/response with automated protocol

2019-08-08 Thread Mickael Maison (JIRA)
Mickael Maison created KAFKA-8768: - Summary: Replace DeleteRecords request/response with automated protocol Key: KAFKA-8768 URL: https://issues.apache.org/jira/browse/KAFKA-8768 Project: Kafka

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-08 Thread Rajini Sivaram
Hi Don, Thanks for reviewing the KIP. 1. I had this originally as a single Action, but thought it may be useful to support batched authorize calls as well and keep it consistent with other methods. Single requests can contain multiple topics. For example a produce request can contain records for

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

2019-08-08 Thread M. Manna
Hi, If I may, perhaps you could simplify everything by using only 'auto.create.topics.enable' as a value along with true. In other words, the public interfaces section should only have [true,auto.create.topics.enable, false]. The reason for this is that auto.create.topics.enable is already known

Re: [DISCUSS] KIP-317: Transparent Data Encryption

2019-08-08 Thread Sönke Liebau
Thanks for your feedback both of you! I've commented inline below. On Thu, 8 Aug 2019 at 08:38, Jörn Franke wrote: > If you are doing batch encryption then you are more similar to a scenario > of file encryption. The more frequent the messages are you are closer to > the ssl/https scenarios. Y