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

2019-08-07 Thread Jörn Franke
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. You may learn from those protocols on how they handle keys, how long they keep them etc. to implement your E2e solution . >

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

2019-08-07 Thread Maulin Vasavada
Bump! Can somebody please review this? On Tue, Jul 16, 2019 at 1:51 PM Maulin Vasavada wrote: > Bump! Can somebody please review this? >

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

2019-08-07 Thread Guozhang Wang
1. Sounds good, just wanted to clarify; and it may worth documenting it so that users would not be surprised when monitoring their footprint. 2. Hmm I see... I think the trade-off can be described as "how much imbalance would bother you to be willing to pay another rebalance, along with potentiall

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

2019-08-07 Thread Maulin Vasavada
Hi Sönke Liebau Thanks for the great detailed documentation. However, I feel by leaving the KMS outside of Kafka might simplify the whole thing to a great extent. If the broker is not going to touch the enc

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

2019-08-07 Thread Don Bosco Durai
Rajini Thanks for putting this together. It is looking good. I have few questions... 1. List authorize(..., List actions). Do you see a scenario where the broker will call authorize for multiple topics at the same time? I can understand that during creating/deleting ACLS, multiple permissions

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

2019-08-07 Thread Matthias J. Sax
I am not sure if I full understand, hence, I try to rephrase: > I can't think of an example that would require both ways, or would > even be more readable using both ways. Example: There are two processor A and B, and one store S that both need to access and one store S_b that only B needs to ac

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-08-07 Thread George Li
Hi Colin, > In your example, I think we're comparing apples and oranges.  You started by > outlining a scenario where "an empty broker... comes up... [without] any > > leadership[s]."  But then you criticize using reassignment to switch the > order of preferred replicas because it "would not a

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

2019-08-07 Thread Paul Whalen
My thinking on restricting the API to enforce only one way of connecting stores would make it more simple to use and end up with more readable code. I can't think of an example that would require both ways, or would even be more readable using both ways. Shared stores can be passed from the outsi

Build failed in Jenkins: kafka-2.0-jdk8 #286

2019-08-07 Thread Apache Jenkins Server
See Changes: [matthias] KAFKA-8602: Backport bugfix for standby task creation (#7146) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building r

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-08-07 Thread Colin McCabe
On Wed, Aug 7, 2019, at 12:48, George Li wrote: > Hi Colin, > > Thanks for your feedbacks.  Comments below: > > Even if you have a way of blacklisting an entire broker all at once, you > >still would need to run a leader election > for each partition where you > >want to move the leader off of

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

2019-08-07 Thread Colin McCabe
On Wed, Aug 7, 2019, at 15:41, George Li wrote: > This email seemed to get lost in the dev email server. Resending. > > > On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li > wrote: > > > The pending reassignments partitions would be reported as URP (Under > Replicated Partitions). or

Re: KIP-352: Distinguish URPs caused by reassignment

2019-08-07 Thread Jason Gustafson
Hi All, Since KIP-455 is passed, I would like to revive this proposal. I have reduced the scope so that it is just changing the way we compute URP and adding a new metric for the number of reassigning partitions. Please take a look: https://cwiki.apache.org/confluence/display/KAFKA/KIP-352%3A+Dist

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

2019-08-07 Thread Matthias J. Sax
Sorry for the long silence on this KIP Paul! I guess the 2.3 release distracted us somewhat. Overall, I am +1. With regard to John's point about owned vs shared state stores, I think it describe a valid use case, and throwing an exception if people want to mix both features might be too restricti

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

2019-08-07 Thread George Li
This email seemed to get lost in the dev email server. Resending. On Tuesday, August 6, 2019, 10:16:57 PM PDT, George Li wrote: The pending reassignments partitions would be reported as URP (Under Replicated Partitions). or maybe reported as a separate metrics of RURP (Reassignment URP)

[jira] [Created] (KAFKA-8767) Optimize StickyAssignor for Cooperative mode

2019-08-07 Thread Sophie Blee-Goldman (JIRA)
Sophie Blee-Goldman created KAFKA-8767: -- Summary: Optimize StickyAssignor for Cooperative mode Key: KAFKA-8767 URL: https://issues.apache.org/jira/browse/KAFKA-8767 Project: Kafka Issue

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

2019-08-07 Thread Jason Gustafson
Thanks for the KIP. This is useful. The controller also maintains a set for topics which are awaiting deletion, but currently ineligible. A topic which is undergoing reassignment, for example, is ineligible for deletion. Would it make sense to have a metric for this as well? -Jason On Wed, Aug 7,

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

2019-08-07 Thread David Arthur
Updated the KIP with a count of replicas awaiting deletion. On Wed, Aug 7, 2019 at 9:37 AM David Arthur wrote: > Thanks for the feedback, Stan. That's a good point about the partition > count -- I'll poke around and see if I can surface this value in the > Controller. > > On Tue, Aug 6, 2019 at

[jira] [Created] (KAFKA-8766) Allow a custom offset policy for Kafka Streams applications

2019-08-07 Thread Patrik Kleindl (JIRA)
Patrik Kleindl created KAFKA-8766: - Summary: Allow a custom offset policy for Kafka Streams applications Key: KAFKA-8766 URL: https://issues.apache.org/jira/browse/KAFKA-8766 Project: Kafka

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

2019-08-07 Thread Rajini Sivaram
Hi Ron/Harsha/Satish, Thanks for reviewing the KIP! We should perhaps have a wider discussion outside this KIP for refactoring clients so that others who are not following this KIP also notice the discussion. Satish, would you like to start a discussion thread on dev? Regards, Rajini On Wed,

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-08-07 Thread George Li
Hi Colin, Thanks for your feedbacks.  Comments below: > Even if you have a way of blacklisting an entire broker all at once, you >still would need to run a leader election > for each partition where you want >to move the leader off of the blacklisted broker.  So the operation is still >O(N) in

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

2019-08-07 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-8599: Use automatic RPC generation in ExpireDelegationToken -- [...truncated 2.60 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

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

2019-08-07 Thread John Roesler
Hey Guozhang, Thanks for the review! 1. Yes, even with `num.standby.replicas := 0`, we will still temporarily allocate standby tasks to accomplish a no-downtime task migration. Although, I'd argue that this doesn't really violate the config, as the task isn't a true hot standby. As soon as it cat

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

2019-08-07 Thread Jason Gustafson
Hi All, I'd like to start a vote on KIP-497: https://cwiki.apache.org/confluence/display/KAFKA/KIP-497%3A+Add+inter-broker+API+to+alter+ISR. +1 from me. -Jason

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

2019-08-07 Thread Jason Gustafson
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-487: Automatic Topic Creation on Producer

2019-08-07 Thread Justine Olshan
Hi Harsha, I think my message may have gotten lost in all the others. Two of the goals of this KIP are to 1) allow auto-creation on specific clients when the broker default is false and 2) eventually replace the broker config. In order to accomplish these two goals, we need the producer to be ab

Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-08-07 Thread Jason Gustafson
Thanks Mickael, +1 from me. Just a couple more comments: > Yes mapping it to a friendlier error would be nice but I'm not sure if we can do that because UNKNOWN_MEMBER_ID is also returned if the group is dead. I think the logic was chanced recently to return `COORDINATOR_NOT_AVAILABLE` if the gr

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-08-07 Thread Colin McCabe
On Fri, Aug 2, 2019, at 20:02, George Li wrote: > Hi Colin, > Thanks for looking into this KIP.  Sorry for the late response. been busy.  > > If a cluster has MAMY topic partitions, moving this "blacklist" broker > to the end of replica list is still a rather "big" operation, involving > submit

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

2019-08-07 Thread Harsha Chintalapani
On Wed, Aug 07, 2019 at 9:50 AM, Colin McCabe wrote: > On Wed, Aug 7, 2019, at 09:24, Harsha Ch wrote: > > On Tue, Aug 06, 2019 at 11:46 PM, Colin McCabe < cmcc...@apache.org > > wrote: > > On Tue, Aug 6, 2019, at 21:38, Harsha Ch wrote: > > Hi Colin, > "Hmm... I'm not sure I follow. Users don't

[jira] [Created] (KAFKA-8765) Remove "unstable" annotations

2019-08-07 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-8765: -- Summary: Remove "unstable" annotations Key: KAFKA-8765 URL: https://issues.apache.org/jira/browse/KAFKA-8765 Project: Kafka Issue Type: Task Co

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-07 Thread Mickael Maison
Thank Colin for kickstarting this initiative. Just one question. - A nice feature of Zookeeper is the ability to use chroots and have several Kafka clusters use the same Zookeeper ensemble. Is this something we should keep? Thanks On Mon, Aug 5, 2019 at 7:44 PM Colin McCabe wrote: > > On Mon, A

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

2019-08-07 Thread Satish Duggana
I felt the same need when we want to add a pluggable API for core server functionality. This does not need to be part of this KIP, it can be a separate KIP. I can contribute those refactoring changes if others are OK with that. It is better to have a structure like below. kafka-common: common cla

Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-08-07 Thread Mickael Maison
Hi Jason, Thanks for the feedback 1. Yes listOffsets() should be able to retrieve earliest, latest and by-timestamp offsets. I agree, it's better to avoid exposing magic values. I've updated the KIP to use an OffsetSpec object as suggested. 2. Yes let's expose the leader epoch in ListOffsetsResu

[jira] [Resolved] (KAFKA-8716) broker cannot join the cluster after upgrading kafka binary from 2.1.1 to 2.2.1 or 2.3.0

2019-08-07 Thread Yu Yang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yu Yang resolved KAFKA-8716. Resolution: Not A Problem > broker cannot join the cluster after upgrading kafka binary from 2.1.1 to > 2.

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

2019-08-07 Thread Almog Gavra
EDIT: everywhere I've been using "HEX" I meant to be using "BASE64". I will update the KIP to reflect this. On Wed, Aug 7, 2019 at 9:44 AM Almog Gavra wrote: > Thanks for the feedback Arjun! I'm happy changing the default config to > HEX instead of BINARY, no strong feelings there. > > I'll also

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

2019-08-07 Thread Colin McCabe
On Wed, Aug 7, 2019, at 09:24, Harsha Ch wrote: > On Tue, Aug 06, 2019 at 11:46 PM, Colin McCabe < cmcc...@apache.org > wrote: > > > On Tue, Aug 6, 2019, at 21:38, Harsha Ch wrote: > >> > >> Hi Colin, > >> "Hmm... I'm not sure I follow. Users don't have to build their own > >> tooling, right? The

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

2019-08-07 Thread Almog Gavra
Thanks for the feedback Arjun! I'm happy changing the default config to HEX instead of BINARY, no strong feelings there. I'll also clarify the example in the KIP to be clearer: - serialize the decimal field "foo" with value "10.2345" with the HEX setting: {"foo": "D3J5"} - serialize the decimal f

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

2019-08-07 Thread Harsha Ch
On Tue, Aug 06, 2019 at 11:46 PM, Colin McCabe < cmcc...@apache.org > wrote: > > > > On Tue, Aug 6, 2019, at 21:38, Harsha Ch wrote: > > > >> >> >> Hi Colin, >> "Hmm... I'm not sure I follow. Users don't have to build their own >> tooling, right? They can use any of the shell scripts that

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

2019-08-07 Thread Jason Gustafson
Hi Boyang, > We already persist member.id, instance.id and generation.id in the offset topic, what extra fields we need to store? Yeah, you're right. I was a little confused and thought this information was needed by the transaction coordinator. > This should be easily done on the stream side as

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

2019-08-07 Thread Andrew Schofield
Hi, I think this is a useful KIP and it looks good in principle. While it can all be done using interceptors, if the brokers do not know anything about it, you need to maintain the mapping from topics to key ids somewhere external. I'd prefer the way you've done it. I'm not sure whether you'll

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

2019-08-07 Thread Apache Jenkins Server
See Changes: [github] MINOR: Upgrade jackson-databind to 2.9.9.3 (#7125) -- [...truncated 1.79 MB...] org.apache.kafka.streams.state.internals.TimestampedKeyValueStoreBui

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

2019-08-07 Thread Justine Olshan
Hello all, Thank you for all the feedback! It seems that one of the main issues is how the client-side auto-creation can act on its own and does not simply block or allow auto-creation as configured by the broker. I think I was a bit unclear about this, but the idea of this KIP is to eventually

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

2019-08-07 Thread Harsha Chintalapani
Thanks for the KIP Rajini. Quick thought, it would be good to have a common module outside of clients that only applies to server side interfaces & changes. It looks like we are increasingly in favor of using Java interface for pluggable modules on the broker side. Thanks, Harsha On Tue, Aug 06

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

2019-08-07 Thread David Arthur
Thanks for the feedback, Stan. That's a good point about the partition count -- I'll poke around and see if I can surface this value in the Controller. On Tue, Aug 6, 2019 at 8:13 AM Stanislav Kozlovski wrote: > Thanks for the KIP David, > > As you mentioned in the KIP - "when a large number of

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

2019-08-07 Thread Sönke Liebau
Hi Rajini, looks great and addresses a few gripes I had in the past, thanks for that! One idea that I had while reading, but I am not sure if this is taking "being flexible" a step too far maybe.. Would it make sense to make the decision at which severity to log a decision pluggable/configurable?

[jira] [Created] (KAFKA-8764) LogCleanerManager endless loop while compacting/cleaning segments

2019-08-07 Thread Tomislav Rajakovic (JIRA)
Tomislav Rajakovic created KAFKA-8764: - Summary: LogCleanerManager endless loop while compacting/cleaning segments Key: KAFKA-8764 URL: https://issues.apache.org/jira/browse/KAFKA-8764 Project: Ka

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

2019-08-07 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-8599: Use automatic RPC generation in ExpireDelegationToken -- [...truncated 2.58 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Jenkins build is back to normal : kafka-2.2-jdk8 #157

2019-08-07 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-2.3-jdk8 #82

2019-08-07 Thread Apache Jenkins Server
See

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

2019-08-07 Thread Ron Dagostino
Looks great, Rajini — a detailed and complete KIP with a great backwards-compatibility plan. Nothing came to mind aside from how easy it was to read and understand. Thanks for writing it so clearly. Ron > On Aug 6, 2019, at 5:31 PM, Rajini Sivaram wrote: > > Hi all, > > I have created a KI

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

2019-08-07 Thread Apache Jenkins Server
See Changes: [github] MINOR: Upgrade jackson-databind to 2.9.9.3 (#7125) -- [...truncated 2.60 MB...] org.apache.kafka.connect.converters.FloatConverterTest > testDeseri

[jira] [Resolved] (KAFKA-8599) Replace ExpireDelegationToken request/response with automated protocol

2019-08-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8599?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-8599. -- Resolution: Fixed Fix Version/s: 2.4.0 Issue resolved by pull request 7098 [https://github.com/a

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-08-07 Thread Satish Duggana
Hi George, Thanks for addressing the comments. I do not have any more questions. On Wed, Aug 7, 2019 at 11:08 AM George Li wrote: > > Hi Colin, Satish, Stanislav, > > Did I answer all your comments/concerns for KIP-491 ? Please let me know if > you have more questions regarding this feature.

Re: Alternative of poll(0) without pulling records

2019-08-07 Thread Jungtaek Lim
If we just wanted to remove deprecation and let both co-exist, that would be also viable, though `poll(0)` is still a hack and it would be ideal to provide official approach to do so. On Wed, Aug 7, 2019 at 4:24 PM Jungtaek Lim wrote: > Hi devs, > > I'm trying to replace deprecated poll(long) wi

Alternative of poll(0) without pulling records

2019-08-07 Thread Jungtaek Lim
Hi devs, I'm trying to replace deprecated poll(long) with poll(Duration), and realized there's no alternative which behaves exactly same as poll(0), as poll(0) has been used as a hack to only update metadata instead of pulling records. poll(Duration.ZERO) wouldn't behave same since even updating m