Re: [DISCUSS] KIP-363: Allow performance tools to print final results to output file

2018-08-24 Thread Kevin Lu
Hi Attila, Thanks for the KIP. I think this would be a useful feature. Every time I have to benchmark using these performance tools, I end up redirecting the output to a file anyways. Just a couple minor questions... 1. If the configured file already exists, what would be the outcome? My

Re: [DISCUSS] KIP-357: Add support to list ACLs per principal

2018-08-24 Thread Satish Duggana
Hi Mani, Just a minor comment on the output of the command as given in KIP-357, you may want to remove "User:User1 has " as it is redundant for each ACL. It may be good to accept multiple principals option to avoid running this script multiple times with each principal to achieve the same. >>

[jira] [Created] (KAFKA-7336) Kafka Connect source task when producing record with invalid topic name

2018-08-24 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-7336: - Summary: Kafka Connect source task when producing record with invalid topic name Key: KAFKA-7336 URL: https://issues.apache.org/jira/browse/KAFKA-7336 Project:

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-24 Thread Viktor Somogyi-Vass
I think in the first draft I didn't provide an implementation for them as it seemed very simple and straightforward. I looked up a couple of implementations of the ExtendedSerializers on github and the general behavior seems to be that they delegate to the 2 argument (headerless) method:

Re: [VOTE] KIP-346 - Improve LogCleaner behavior on error

2018-08-24 Thread Stanislav Kozlovski
Hi Jun, Yes, my intention was to have them per logDir but I failed to mention it in the KIP. Updated. Thanks, Stan On Fri, Aug 24, 2018 at 12:00 AM Jun Rao wrote: > Hi, Stan, > > Thanks for the KIP. Looks good to me overall. Just one comment below. > > uncleanable-partitions-count is per

[VOTE] KIP-348 Deprecate null from SourceTask#poll()

2018-08-24 Thread Chia-Ping Tsai
hi kafka, I'm starting a vote on KIP-348 https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89065853 -- chia-ping

Re: [DISCUSS] KIP-363: Allow performance tools to print final results to output file

2018-08-24 Thread Attila Sasvári
Thanks for your feedback, Kevin & Viktor! I will update the KIP next week. @Kevin - I also had to write sed based one-liners when I ran performance tests to extract results (that were displayed in some charts later). 1. Initially, I wanted to overwrite any exisisting file and document this

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-08-24 Thread John Roesler
Quick afterthought: I guess that `Window` is exposed to the API via `Windowed` keys. I think it would be fine to not deprecate the `long` start and end, but add `Instant` variants for people preferring that interface. On Fri, Aug 24, 2018 at 11:10 AM John Roesler wrote: > Hey Matthias, > >

Re: Access to Jira

2018-08-24 Thread Jun Rao
Hi, Jonathan, Thanks for your interest. Added you to the contributor list. Jun On Fri, Aug 24, 2018 at 6:18 AM, Jonathan Santilli < jonathansanti...@gmail.com> wrote: > Hello, > > am working on KAFKA-7165 >, > one of the steps is to assign

Re: [DISCUSS] KIP-351: Add --critical-partitions option to describe topics command

2018-08-24 Thread Kevin Lu
Hi All, I am having some trouble re-formulating this KIP to output partitions that are under the configured "min.insync.replicas" as I am not sure how to reliably get the configured "min.insync.replicas" in all cases. The challenge I am facing is when "min.insync.replicas" is configured to

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-08-24 Thread Jason Gustafson
Hey All, Nice to see some solid progress on this. It sounds like one of the complications is allowing static and dynamic registration to coexist. I'm wondering if we can do something like the following: 1. Statically registered members (those joining the group with a non-null ` member.name`)

Re: Build failed in Jenkins: kafka-trunk-jdk10 #429

2018-08-24 Thread Joan Goyeau
Yeah it's was all good in the end :) Thanks Ted On Thu, 23 Aug 2018 at 21:42 Ted Yu wrote: > I ran streams unit tests as of > commit 4156ea0a9bcca67d209fd3b43d2268c9abd5a0b5 . > > All tests passed locally. > > FYI > > On Thu, Aug 23, 2018 at 12:23 PM Joan Goyeau wrote: > > > I'm looking into

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-08-24 Thread John Roesler
Hi Nikolay, First: I wanted to let you know that we have dropped the `grace(long)` method from the Windows interface, but we do still need to transition the same method on TimeWindows and JoinWindows ( https://github.com/apache/kafka/pull/5536) I have also been thinking it would be nice to

[jira] [Created] (KAFKA-7337) Enhance Producer Performance tool to generate keys

2018-08-24 Thread Zenifer Cheruveettil (JIRA)
Zenifer Cheruveettil created KAFKA-7337: --- Summary: Enhance Producer Performance tool to generate keys Key: KAFKA-7337 URL: https://issues.apache.org/jira/browse/KAFKA-7337 Project: Kafka

Re: [DISCUSS] KIP-362: Dynamic Session Window Support

2018-08-24 Thread Guozhang Wang
Hello Lei, Thanks for the proposal. I've just made a quick pass over it and there is a question I have: The session windows are defined per key, i.e. does that mean that each incoming record of the key can dynamically change the gap of the window? For example, say you have the following record

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-08-24 Thread Matthias J. Sax
It's tricky... :) Some APIs have "dual use" as I mentioned in my first reply. I agree that it would be good to avoid abstract class and use interfaces if possible. As long as the change is source code compatible, it should be fine IMHO -- we need to document binary incompatibility of course. I

Re: [DISCUSS] KIP-363: Make FunctionConversions private

2018-08-24 Thread John Roesler
I'm also in favor of this. I don't think it's controversial either. Should we just move to a vote? On Thu, Aug 23, 2018 at 7:01 PM Guozhang Wang wrote: > +1. > > On Thu, Aug 23, 2018 at 12:47 PM, Ted Yu wrote: > > > +1 > > > > In the Motivation section, you can quote the comment from pull

Re: [VOTE] KIP-363: Make FunctionConversions private

2018-08-24 Thread Guozhang Wang
+1 from me (binding). On Fri, Aug 24, 2018 at 11:24 AM, Joan Goyeau wrote: > Hi, > > As pointed out in this comment #5539 (comment) > "This > class was already defaulted to public visibility, and we can't retract it > now,

Re: Request for contributor permissions

2018-08-24 Thread Matthias J. Sax
Done. On 8/23/18 8:19 PM, 王又田 wrote: > JIRA ID: tony80720 > > Cwiki ID: Yu Tien Wang > > Thanks in advance! > > Tony > > > > --- > This email has been checked for viruses by Avast antivirus software. > https://www.avast.com/antivirus > signature.asc Description: OpenPGP digital

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-24 Thread Joel Koshy
I had some offline discussions with Lucas on this KIP. While it is much more work than the original proposals, separating the control plane entirely removes any interference with the data plane as summarized under the rejected alternatives section. Just a few minor comments: - Can you update

unable to build schema registry

2018-08-24 Thread Simon Nunn
Getting the following error when trying to build the schema registry. I have tried various versions of kafka, but not sure what I need to do. Any help would be appreciated. [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.6.1:compile (default-compile) on

Request for contributor permissions

2018-08-24 Thread LEE, Tung-Yu
JIRA ID: leetungyu Cwiki ID: leetungyu Thanks a lot!

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-08-24 Thread Jason Gustafson
Hey Guozhang, Responses below: Originally I was trying to kill more birds with one stone with KIP-345, > e.g. to fix the multi-rebalance issue on starting up / shutting down a > multi-instance client (mentioned as case 1)/2) in my early email), and > hence proposing to have a pure

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-08-24 Thread John Roesler
Hey Matthias, Thanks for pointing that out. I agree that we only really need to change methods that are API-facing, and we probably want to avoid using Duration/Instant for Streams-facing members. Like I said in my last email, I think the whole Windows interface is Streams-facing, and the

Access to Jira

2018-08-24 Thread Jonathan Santilli
Hello, am working on KAFKA-7165 , one of the steps is to assign the Jira issue to myself, but am not allowed, I would like to have Jira access to do it, please. Cheers! -- Santilli Jonathan

Request for contributor permissions

2018-08-24 Thread 陳映彤
JIRA ID: Chloe chen Cwiki ID: Chloe chen Thanks in advance! 從我的 iPhone 傳送

Re: Request for contributor permissions

2018-08-24 Thread Jun Rao
Hi, Yingtung, Thanks for your interest. Gave you permission to both jira and wiki. Jun On Thu, Aug 23, 2018 at 8:58 PM, 陳映彤 wrote: > JIRA ID: Chloe chen > Cwiki ID: Chloe chen > > Thanks in advance! > > 從我的 iPhone 傳送

[VOTE] KIP-363: Make FunctionConversions private

2018-08-24 Thread Joan Goyeau
Hi, As pointed out in this comment #5539 (comment) "This class was already defaulted to public visibility, and we can't retract it now, without a KIP.", the object FunctionConversions is only of internal use and therefore should be

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-08-24 Thread Guozhang Wang
Hey Jason, I like your idea to simplify the upgrade protocol to allow co-exist of static and dynamic members. Admittedly it may make the coordinator-side logic a bit more complex, but I think it worth doing it. Originally I was trying to kill more birds with one stone with KIP-345, e.g. to fix

Re: Request for contributor permissions

2018-08-24 Thread Jun Rao
Hi, Tung-Yu, Thanks for your interest. Gave you permission to both jira and wiki. Jun On Fri, Aug 24, 2018 at 12:41 AM, LEE, Tung-Yu wrote: > JIRA ID: leetungyu > Cwiki ID: leetungyu > > Thanks a lot! >

Jenkins build is back to normal : kafka-trunk-jdk10 #435

2018-08-24 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-363: Make FunctionConversions private

2018-08-24 Thread Attila Sasvári
Hi there, There is a conflicting KIP with the same number, see https://cwiki.apache.org/confluence/display/KAFKA/KIP-363%3A+Allow+performance+tools+to+print+final+results+to+output+file Its discussion was started earlier, on August 23

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

2018-08-24 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5975; No response when deleting topics and [github] MINOR: replace deprecated remove with delete (#5565) -- [...truncated 2.47 MB...]

Build failed in Jenkins: kafka-trunk-jdk10 #434

2018-08-24 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7240: -total metrics in Streams are incorrect (#5467) -- [...truncated 1.53 MB...] kafka.coordinator.transaction.TransactionCoordinatorTest

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-24 Thread Jason Gustafson
Hey Viktor, Good summary. I agree that option 1) seems like the simplest choice and, as you note, we can always add the default implementation later. I'll leave Ismael to make a case for the circular forwarding approach ;) -Jason On Fri, Aug 24, 2018 at 3:02 AM, Viktor Somogyi-Vass <

Re: Request for contributor permissions

2018-08-24 Thread LEE, Tung-Yu
Thanks! Jun Rao 於 2018年8月25日 週六 01:40 寫道: > Hi, Tung-Yu, > > Thanks for your interest. Gave you permission to both jira and wiki. > > Jun > > On Fri, Aug 24, 2018 at 12:41 AM, LEE, Tung-Yu wrote: > >> JIRA ID: leetungyu >> Cwiki ID: leetungyu >> >> Thanks a lot! >> > >

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-08-24 Thread Jason Gustafson
Hey Mike, Yeah, that's a good point. A long "registration timeout" may not be a great idea. Perhaps in practice you'd set it long enough to be able to detect a failure and provision a new instance. Maybe on the order of 10 minutes is more reasonable. In any case, it's probably a good idea to

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-24 Thread Ismael Juma
I'm OK with 1 too. It makes me a bit sad that we don't have a path for removing the method without headers, but it seems like the simplest and least confusing option (I am assuming that headers are not needed in the serializers in the common case). Ismael On Fri, Aug 24, 2018 at 2:42 PM Jason

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

2018-08-24 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7240: -total metrics in Streams are incorrect (#5467) -- [...truncated 2.47 MB...]

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-08-24 Thread Mike Freyberger
Jason, Regarding step 4 in your proposal which suggests beginning a long timer (30 minutes) when a static member leaves the group, would there also be the ability for an admin to force a static membership expiration? I’m thinking that during particular types of outages or upgrades users would

Re: [VOTE] KIP-363: Make FunctionConversions private

2018-08-24 Thread Ted Yu
+1 On Fri, Aug 24, 2018 at 5:17 PM Attila Sasvári wrote: > Hi there, > > There is a conflicting KIP with the same number, see > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-363%3A+Allow+performance+tools+to+print+final+results+to+output+file > > Its discussion was started earlier, on

Re: unable to build schema registry

2018-08-24 Thread Ted Yu
The first error was complaining about this class from Kafka clients: clients/src/main/java/org/apache/kafka/common/utils/Timer.java It seems you haven't installed kafka clients jar into local maven repo. You can run the following command under kafka working dir: ./gradlew installAll I just

Jenkins build is back to normal : kafka-trunk-jdk8 #2925

2018-08-24 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-357: Add support to list ACLs per principal

2018-08-24 Thread Manikumar
Hi Satish, Thanks for the review. 1. Currently we are just printing toSring() of Acl class. I prefer to have similar output with or without "--principal" option. 2. Yes, we can accept multiple principals. Updated the KIP. @all If there are no more comments, I will start vote on this KIP early

Re: [DISCUSS] KIP-363: Allow performance tools to print final results to output file

2018-08-24 Thread Viktor Somogyi-Vass
Hi Attila, Thanks for the KIP, I think overall it looks good. I have three comments: 1. Would you mind adding an example? (Later on we'd need anyway for the public doc.) 2. Do you want to add any 3rd party CSV reader/writer library or will you implement that too? 3. What is the separator or is