[jira] [Created] (KAFKA-8792) Default ZK configuration to disable AdminServer

2019-08-12 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-8792: --- Summary: Default ZK configuration to disable AdminServer Key: KAFKA-8792 URL: https://issues.apache.org/jira/browse/KAFKA-8792 Project: Kafka Issue Type:

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

2019-08-12 Thread Jungtaek Lim
So in overall, AdminClient covers the necessary to retrieve up-to-date topic-partitions, whereas KIP-396 will cover the necessary to retrieve offset (EARLIEST, LATEST, timestamp) on partition. Gabor, could you please add the input if I'm missing something? I'd like to double-check on this.

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

2019-08-12 Thread Jungtaek Lim
On Tue, Aug 13, 2019 at 10:01 AM Colin McCabe wrote: > On Mon, Aug 12, 2019, at 14:54, Jungtaek Lim wrote: > > Thanks for the feedbacks Colin and Matthias. > > > > I agree with you regarding getting topics and partitions via AdminClient, > > just curious how much the overhead would be. Would it

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

2019-08-12 Thread Jungtaek Lim
For now Spark needs to know about exact offset for EARLIEST, LATEST per partition so that it can handle users' query on EARLIEST/LATEST and write exact offset in checkpoint. I guess Spark would also want to validate the known offset, but I guess that could be covered by knowing range of available

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

2019-08-12 Thread Jungtaek Lim
Hi Harsha, I'm not sure what exactly the class is doing, but if I can't get all the necessary information from that class, I would end up with calling others and go back to same issue. And skimming the class, it seems to be complicated one (end-users unfriendly, as it's designed to be used

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

2019-08-12 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Format AdminUtils::assignReplicasToBrokers java documentation -- [...truncated 6.52 MB...] org.apache.kafka.streams.scala.kstream.JoinedTest

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

2019-08-12 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Format AdminUtils::assignReplicasToBrokers java documentation -- [...truncated 2.59 MB...]

Re: [DISCUSS] KIP-482: The Kafka Protocol should Support Optional Fields

2019-08-12 Thread Colin McCabe
On Mon, Aug 12, 2019, at 11:22, Jason Gustafson wrote: > Hi Colin, > > Thanks for the KIP! This is a significant improvement. One of my personal > interests in this proposal is solving the compatibility problems we have > with the internal schemas used to define consumer offsets and transaction >

Re: Dynamic configuration of interbroker SSL certificates

2019-08-12 Thread Michael Carter
Hi Colin, Thanks for the explanation. Yes, I can see the sense in that, although it was certainly a surprise for me. I’ll just have to be a bit more careful about instantiating AdminClients in the future. Cheers, Michael > On 13 Aug 2019, at 1:36 am, Colin McCabe wrote: > > Hi Michael, > >

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

2019-08-12 Thread Colin McCabe
On Mon, Aug 12, 2019, at 14:54, Jungtaek Lim wrote: > Thanks for the feedbacks Colin and Matthias. > > I agree with you regarding getting topics and partitions via AdminClient, > just curious how much the overhead would be. Would it be lighter, or > heavier? We may not want to list topics in

[jira] [Created] (KAFKA-8791) RocksDBTimestampeStore should open in regular mode for new store

2019-08-12 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-8791: -- Summary: RocksDBTimestampeStore should open in regular mode for new store Key: KAFKA-8791 URL: https://issues.apache.org/jira/browse/KAFKA-8791 Project: Kafka

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

2019-08-12 Thread Matthias J. Sax
Note that `KafkaConsumer` refreshed it's metadata every 5 minutes by default anyway... (parameter `metadata.max.age.ms`). And of course, you can refresh the metadata you get via AdminClient each time you trigger planning. I cannot quantify the overhead of a single request though. Also, what

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

2019-08-12 Thread Harsha Chintalapani
Hi Jungtaek, Have you looked into this interface https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java . Right now its not a public interface but does the methods available in this interface work for your needs? . The

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

2019-08-12 Thread Boyang Chen
Thanks Jason, the intuition behind defining a separate callback function is that, with KIP-429 we no longer guarantee to call OnPartitionsAssigned() or OnPartitionsRevoked() with each rebalance. Our requirement is to be up-to-date with group metadata such as generation information, so callback

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

2019-08-12 Thread Jungtaek Lim
Thanks for the feedbacks Colin and Matthias. I agree with you regarding getting topics and partitions via AdminClient, just curious how much the overhead would be. Would it be lighter, or heavier? We may not want to list topics in regular intervals - in plan phase we want to know up-to-date

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

2019-08-12 Thread Rajini Sivaram
Hi David, Thanks for reviewing the KIP! Since questions about `authorization mode` and `count` have come up multiple times, I have renamed both. 1) Renamed `count` to `resourceReferenceCount`. It is the number of times the resource being authorized is referenced within the request. 2) Renamed

Re: [VOTE] KIP-497: Add inter-broker API to alter ISR

2019-08-12 Thread Jason Gustafson
Hi Viktor, I originally named the field `CurrentVersion`. I didn't have 'Zk' in the name in anticipation of KIP-500. I thought about it and decided it makes sense to keep naming consistent with other APIs. Even if KIP-500 passes, there will be some time during which it only refers to the zk

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

2019-08-12 Thread Jason Gustafson
Hey Boyang, I favor option 4 as well. It's a little more cumbersome than 3 for this use case, but it seems like a cleaner separation of concerns. The rebalance listener is already concerned with events affecting the assignment lifecycle and group membership. I think the only thing I'm wondering

[jira] [Reopened] (KAFKA-8789) kafka-avro-console-consumer works with 2.0.x, but not 2.3.x

2019-08-12 Thread Raman Gupta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raman Gupta reopened KAFKA-8789: > kafka-avro-console-consumer works with 2.0.x, but not 2.3.x >

Re: [DISCUSS] KIP-482: The Kafka Protocol should Support Optional Fields

2019-08-12 Thread Jason Gustafson
Hi Colin, Thanks for the KIP! This is a significant improvement. One of my personal interests in this proposal is solving the compatibility problems we have with the internal schemas used to define consumer offsets and transaction metadata. Currently we have to guard schema bumps with the

Re: [VOTE] KIP-496: Administrative API to delete consumer offsets

2019-08-12 Thread Guozhang Wang
+1 (binding). Thanks Jason! On Wed, Aug 7, 2019 at 11:18 AM Jason Gustafson wrote: > Hi All, > > I'd like to start a vote on KIP-496: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-496%3A+Administrative+API+to+delete+consumer+offsets > . > +1 > from me of course. > > -Jason > --

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

2019-08-12 Thread David Jacot
Hi Rajini, Thank you for the KIP. Overall, it looks good to me. I have few questions/suggestions: 1. It is hard to grasp what `Action#count` is for. I guess I understand where you want to go with it but it took me a while to figure it out. Perhaps, we could come up with a better name than

[jira] [Created] (KAFKA-8790) [kafka-connect] KafkaBaseLog.WorkThread not recoverable

2019-08-12 Thread Qinghui Xu (JIRA)
Qinghui Xu created KAFKA-8790: - Summary: [kafka-connect] KafkaBaseLog.WorkThread not recoverable Key: KAFKA-8790 URL: https://issues.apache.org/jira/browse/KAFKA-8790 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-8789) kafka-avro-console-consumer works with 2.0.x, but not 2.3.x

2019-08-12 Thread Raman Gupta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raman Gupta resolved KAFKA-8789. Resolution: Invalid UPDATE: The error message may be coming from the schema registry, which would

[jira] [Created] (KAFKA-8789) kafka-avro-console-consumer works with confluent 5.0.3, but not 5.3.0

2019-08-12 Thread Raman Gupta (JIRA)
Raman Gupta created KAFKA-8789: -- Summary: kafka-avro-console-consumer works with confluent 5.0.3, but not 5.3.0 Key: KAFKA-8789 URL: https://issues.apache.org/jira/browse/KAFKA-8789 Project: Kafka

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

2019-08-12 Thread Matthias J. Sax
Thanks for the details Jungtaek! I tend to agree with Colin, that using the AdminClient seems to be the better choice. You can get all topics via `listTopics()` (and you can refresh this information on regular intervals) and match any pattern against the list of available topics in the driver.

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

2019-08-12 Thread Harsha Chintalapani
On Fri, Aug 09, 2019 at 11:12 AM, Ismael Juma wrote: > Hi all, > > A few points: > > 1. I think the way backwards compatibility is being used here is not > correct. Any functionality that is only enabled if set via a config is > backwards compatible. People may disagree with the functionality or

Re: [DISCUSS] KIP-373: Allow users to create delegation tokens for other users

2019-08-12 Thread Colin McCabe
+1 for better access control here. In general, impersonating another user seems like it’s equivalent to super user access. Colin On Mon, Aug 12, 2019, at 05:43, Manikumar wrote: > Hi Viktor, > > As per the KIP, It's not only superuser, any user with required permissions > (CreateTokens on

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

2019-08-12 Thread Gabor Somogyi
Hi Colin, Thanks for your suggestion! Which KIPs are you referring to? BR, G On Mon, Aug 12, 2019 at 5:22 PM Colin McCabe wrote: > Hi, > > If there’s no need to consume records in the Spark driver, then the > Consumer is probably the wrong thing to use. Instead, Spark should use >

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-08-12 Thread Ismael Juma
Thanks for volunteering Manikumar. +1 Ismael On Mon, Aug 12, 2019 at 7:54 AM Manikumar wrote: > Hi all, > > I would like to volunteer to be the release manager for our next time-based > feature release (v2.4.0). > > If that sounds good, I'll post the release plan over the next few days. > >

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-08-12 Thread Colin McCabe
+1. Thanks, Manikumar. Colin On Mon, Aug 12, 2019, at 08:25, Matthias J. Sax wrote: > Thanks Manikumar! SGTM. > > > On 8/12/19 7:54 AM, Manikumar wrote: > > Hi all, > > > > I would like to volunteer to be the release manager for our next time-based > > feature release (v2.4.0). > > > > If

Re: Dynamic configuration of interbroker SSL certificates

2019-08-12 Thread Colin McCabe
Hi Michael, The NetworkClient periodically fetches metadata so that it always knows what the cluster topology is. This also helps it to have some open connections when needed to reduce the latency of operations. To be fair, we haven’t thought very much about optimizing this since the overhead

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-08-12 Thread Matthias J. Sax
Thanks Manikumar! SGTM. On 8/12/19 7:54 AM, Manikumar wrote: > Hi all, > > I would like to volunteer to be the release manager for our next time-based > feature release (v2.4.0). > > If that sounds good, I'll post the release plan over the next few days. > > Thanks, > Manikumar >

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

2019-08-12 Thread Colin McCabe
Hi, If there’s no need to consume records in the Spark driver, then the Consumer is probably the wrong thing to use. Instead, Spark should use AdminClient to find out what partitions exist and where, manage their offsets, and so on. There are some KIPs under discussion now that would add the

[DISCUSS] Apache Kafka 2.4.0 release

2019-08-12 Thread Manikumar
Hi all, I would like to volunteer to be the release manager for our next time-based feature release (v2.4.0). If that sounds good, I'll post the release plan over the next few days. Thanks, Manikumar

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

2019-08-12 Thread Jungtaek Lim
My feeling is that I didn't explain the use case for Spark properly and hence fail to explain the needs. Sorry about this. Spark leverages the single instance of KafkaConsumer in the driver which is registered solely on the consumer group. This is used in the plan phase for each micro-batch to

Re: [DISCUSS] KIP-373: Allow users to create delegation tokens for other users

2019-08-12 Thread Manikumar
Hi Viktor, As per the KIP, It's not only superuser, any user with required permissions (CreateTokens on Cluster Resource), can create the tokens for other users. Current proposed permissions defined like, "UserA can create tokens for any user". I am thinking, can we change the permissions like

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-08-12 Thread Mickael Maison
Hi Tom, Thanks for following up on this KIP. This is a great improvement that will make policies more powerful and at the same time easier to manage. I just have one question: In AbstractRequestMetadata.principal() javadoc, it says the principal will be "null" for non authenticated session.

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

2019-08-12 Thread Apache Jenkins Server
See Changes: [vahid.hashemian] MINOR: Remove Utils.notNull, use Objects.requireNonNull instead (#7194) -- [...truncated 6.51 MB...]

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-08-12 Thread Tom Bentley
Hi folks, As far as I can see the motivation for KIP-201 is still valid, and as far as I'm aware the changes I made to the KIP back in April addressed the previous comments. Since the issue still needs to be addressed I intend to start another vote thread in the near future, but before I do I

[DISCUSS] KIP-506: Allow setting SCRAM password via Admin interface

2019-08-12 Thread Tom Bentley
Hi All, I've written KIP-506 proposing an RPC and Admin interface for setting SCRAM user passwords. I think there could be an interesting discussion over the relative merits of hashing on the broker or client. In any case I'd be grateful for any comments you may have:

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

2019-08-12 Thread Apache Jenkins Server
See Changes: [vahid.hashemian] MINOR: Remove Utils.notNull, use Objects.requireNonNull instead (#7194) -- [...truncated 2.60 MB...]

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

2019-08-12 Thread Gabor Somogyi
Hi Guys, Please see the actual implementation, pretty sure it explains the situation well: https://github.com/apache/spark/blob/master/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala To answer one question/assumption which popped up from all of you

Re: Alternative of poll(0) without pulling records

2019-08-12 Thread Gabor Somogyi
@Jungtaek, thanks for the explanation! @Colin, please see the attached code in my previous mail for all the details. On Sun, Aug 11, 2019 at 1:20 PM Jungtaek Lim wrote: > Btw, I'd like to ask you to move on thread for KIP discussion, as it will > make us reaching conclusion faster and have