Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-08 Thread Guozhang Wang
Hello community, We have made a few changes based on the comments in this thread as well as the DISCUSS thread. Summary of the changes can be found in the update history (2017/02/03 and 2017/02/07) of the design doc: https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF

[jira] [Assigned] (KAFKA-4716) Fix logic for re-checking if internal topic is ready

2017-02-08 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-4716: --- Assignee: Eno Thereska > Fix logic for re-checking if internal topic is ready >

[jira] [Updated] (KAFKA-4716) Fix logic for re-checking if internal topic is ready

2017-02-08 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4716: Fix Version/s: (was: 0.10.3.0) 0.10.2.0 > Fix logic for re-checking if

[jira] [Updated] (KAFKA-4716) Fix logic for re-checking if internal topic is ready

2017-02-08 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4716: Priority: Blocker (was: Major) > Fix logic for re-checking if internal topic is ready >

Re: Correct prefetching of data to KTable-like structure on application startup

2017-02-08 Thread Matthias J. Sax
Jan, you scenario is quite complex and I am not sure if I understood every part of it. I try to break it down: > In my scenario on startup, I want to read all data from a topic (or a subset > of its partitions), > wait until all the old data has been cached and then start processing of a >

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-08 Thread Mayuresh Gharat
Hi Jun, Thanks for the review. Please find the responses inline. 1. It seems the problem that you are trying to address is that java principal returned from KafkaChannel may have additional fields than name that are needed during authorization. Have you considered a customized PrincipleBuilder

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-08 Thread Matthias J. Sax
+1 On 2/8/17 4:51 PM, Gwen Shapira wrote: > +1 (binding) > > On Wed, Feb 8, 2017 at 4:45 PM, Steven Schlansker > wrote: >> Hi everyone, >> >> Thank you for constructive feedback on KIP-121, >> KStream.peek(ForeachAction) ; >> it seems like it is time to call a

[jira] [Commented] (KAFKA-4709) Error message from Struct.validate() should include the name of the offending field.

2017-02-08 Thread Tegan Snyder (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15859017#comment-15859017 ] Tegan Snyder commented on KAFKA-4709: - Is there any way to figure out the field that is causing this

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-08 Thread Bill Bejeck
+1 On Wed, Feb 8, 2017 at 7:51 PM, Gwen Shapira wrote: > +1 (binding) > > On Wed, Feb 8, 2017 at 4:45 PM, Steven Schlansker > wrote: > > Hi everyone, > > > > Thank you for constructive feedback on KIP-121, > KStream.peek(ForeachAction) ; > >

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

2017-02-08 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4749; Fix join-time-max and sync-time-max MeasurableStat type -- [...truncated 4108 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Dong Lin
I am not aware of any semantics that will be caused by sharing NetworkClient between producer/consumer and AdminClient. But I agree that there is currently no good way to share such an internal class between them. And yes, goal is to reduce number of connections. For example, say we want to enable

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

2017-02-08 Thread Apache Jenkins Server
See Changes: [junrao] MINOR: changes to the production broker configuration docs. -- [...truncated 4106 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Jason Gustafson
I'm not too sure sharing NetworkClient is a good idea. The consumer and the producer both have request semantics which would be more difficult to reason about if the connections are shared with another client. Also, the NetworkClient is an internal class which is not really meant for users. Do we

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Dong Lin
BTW, the idea to share NetworkClient is suggested by Radai and I like this idea. On Wed, Feb 8, 2017 at 6:39 PM, Dong Lin wrote: > Hey Colin, > > Thanks for updating the KIP. I have two followup questions: > > - It seems that setCreationConfig(...) is a bit redundant given

[jira] [Commented] (KAFKA-4749) fix join-time-max and sync-time-max MeasurableStat type

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4749?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858907#comment-15858907 ] ASF GitHub Bot commented on KAFKA-4749: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2520: KAFKA-4749: fix join-time-max and sync-time-max Me...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2520 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Resolved] (KAFKA-4749) fix join-time-max and sync-time-max MeasurableStat type

2017-02-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4749?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4749. Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2520

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Dong Lin
Hey Colin, Thanks for updating the KIP. I have two followup questions: - It seems that setCreationConfig(...) is a bit redundant given that most arguments (e.g. topic name, partition num) are already passed to TopicsContext.create(...) when user creates topic. Should we pass the creationConfig

[GitHub] kafka pull request #2519: MINOR: changes to the production broker configurat...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2519 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Commented] (KAFKA-4749) fix join-time-max and sync-time-max MeasurableStat type

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4749?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858816#comment-15858816 ] ASF GitHub Bot commented on KAFKA-4749: --- GitHub user onurkaraman opened a pull request:

[GitHub] kafka pull request #2520: KAFKA-4749: fix join-time-max and sync-time-max Me...

2017-02-08 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/2520 KAFKA-4749: fix join-time-max and sync-time-max MeasurableStat type GroupCoordinatorMetrics currently sets up join-time-max and sync-time-max incorrectly as a "new Avg()" MeasurableStat instead

[jira] [Created] (KAFKA-4749) fix join-time-max and sync-time-max MeasurableStat type

2017-02-08 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-4749: --- Summary: fix join-time-max and sync-time-max MeasurableStat type Key: KAFKA-4749 URL: https://issues.apache.org/jira/browse/KAFKA-4749 Project: Kafka Issue

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Colin McCabe
Hi all, I made some major revisions to the proposal on the wiki, so please check it out. The new API is based on Ismael's suggestion of grouping related APIs. There is only one layer of grouping. I think that it's actually pretty intuitive. It's also based on the idea of using Futures, which

[GitHub] kafka pull request #2519: MINOR: changes to the production broker configurat...

2017-02-08 Thread alexlod
GitHub user alexlod opened a pull request: https://github.com/apache/kafka/pull/2519 MINOR: changes to the production broker configuration docs. You can merge this pull request into a Git repository by running: $ git pull https://github.com/alexlod/kafka

[jira] [Commented] (KAFKA-4747) add metrics for KafkaConsumer.poll

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858800#comment-15858800 ] ASF GitHub Bot commented on KAFKA-4747: --- GitHub user onurkaraman opened a pull request:

[GitHub] kafka pull request #2518: KAFKA-4747: add metrics for KafkaConsumer.poll

2017-02-08 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/2518 KAFKA-4747: add metrics for KafkaConsumer.poll KafkaConsumer heavily depends on KafkaConsumer.poll yet we don't have metrics directly associated with it. We probably want to add two

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-08 Thread Gwen Shapira
+1 (binding) On Wed, Feb 8, 2017 at 4:45 PM, Steven Schlansker wrote: > Hi everyone, > > Thank you for constructive feedback on KIP-121, KStream.peek(ForeachAction V>) ; > it seems like it is time to call a vote which I hope will pass easily :) > >

KIP-121 [VOTE]: Add KStream peek method

2017-02-08 Thread Steven Schlansker
Hi everyone, Thank you for constructive feedback on KIP-121, KStream.peek(ForeachAction) ; it seems like it is time to call a vote which I hope will pass easily :) https://cwiki.apache.org/confluence/display/KAFKA/KIP-121%3A+Add+KStream+peek+method I believe the PR attached is already in

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

2017-02-08 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: renamed test so it is picked up by ducktape [wangguoz] MINOR: Add logging when commitSync fails in StreamTask [wangguoz] KAFKA-4702: Parametrize streams benchmarks to run at scale

[jira] [Commented] (KAFKA-4746) Offsets can be committed for the offsets topic

2017-02-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4746?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858753#comment-15858753 ] Jason Gustafson commented on KAFKA-4746: Thanks. I was thinking of cases where you're

[jira] [Commented] (KAFKA-4746) Offsets can be committed for the offsets topic

2017-02-08 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4746?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858709#comment-15858709 ] Grant Henke commented on KAFKA-4746: I just mean that often when working with a compacted topic you

[jira] [Created] (KAFKA-4748) Need a way to shutdown all workers in a Streams application at the same time

2017-02-08 Thread Elias Levy (JIRA)
Elias Levy created KAFKA-4748: - Summary: Need a way to shutdown all workers in a Streams application at the same time Key: KAFKA-4748 URL: https://issues.apache.org/jira/browse/KAFKA-4748 Project: Kafka

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-08 Thread Jun Rao
Hi, Mayuresh, Thanks for the KIP. A few comments below. 1. It seems the problem that you are trying to address is that java principal returned from KafkaChannel may have additional fields than name that are needed during authorization. Have you considered a customized PrincipleBuilder that

[jira] [Commented] (KAFKA-4745) KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame

2017-02-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858686#comment-15858686 ] Ismael Juma commented on KAFKA-4745: Your improvement suggestion makes sense to me, would you like to

[jira] [Commented] (KAFKA-4745) KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame

2017-02-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858687#comment-15858687 ] Ismael Juma commented on KAFKA-4745: Also `writeEndMark` sets finished=true as well. >

[jira] [Commented] (KAFKA-4746) Offsets can be committed for the offsets topic

2017-02-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4746?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858659#comment-15858659 ] Jason Gustafson commented on KAFKA-4746: I never thought of someone trying this, but it makes

[jira] [Created] (KAFKA-4747) add metrics for KafkaConsumer.poll

2017-02-08 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-4747: --- Summary: add metrics for KafkaConsumer.poll Key: KAFKA-4747 URL: https://issues.apache.org/jira/browse/KAFKA-4747 Project: Kafka Issue Type: Improvement

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

2017-02-08 Thread Apache Jenkins Server
See

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-08 Thread Jorge Esteban Quilcate Otoya
Great. I think I got the idea. What about this options: Scenarios: 1. Current status ´kafka-consumer-groups.sh --reset-offset --group cg1´ 2. To Datetime ´kafka-consumer-groups.sh --reset-offset --group cg1 --reset-to-datetime 2017-01-01T00:00:00.000´ 3. To Period ´kafka-consumer-groups.sh

Jenkins build is back to normal : kafka-trunk-jdk7 #1922

2017-02-08 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-08 Thread Jun Rao
Hi, Mani, Thanks for the responses. A few more comments. 101.2/101.3. Could we just remove owner and renewer from DelegationTokenResponse if we don't have a use case? 111. ExpireTokenResponse: Should we return the new expiration time in the response? 112. DescribeTokenRequest: A common use

[jira] [Created] (KAFKA-4746) Offsets can be committed for the offsets topic

2017-02-08 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-4746: -- Summary: Offsets can be committed for the offsets topic Key: KAFKA-4746 URL: https://issues.apache.org/jira/browse/KAFKA-4746 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858581#comment-15858581 ] Guozhang Wang commented on KAFKA-4738: -- I'd suggest we only remove Generic T in this JIRA, as for

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-08 Thread Ben Stopford
Yes - using a tool like this to skip a set of consumer groups over a corrupt/bad message is definitely appealing. B On Wed, Feb 8, 2017 at 9:37 PM Gwen Shapira wrote: > I like the --reset-to-earliest and --reset-to-latest. In general, > since the JSON route is the most

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-08 Thread Gwen Shapira
I like the --reset-to-earliest and --reset-to-latest. In general, since the JSON route is the most challenging for users, we want to provide a lot of ways to do useful things without going there. Two things that can help: 1. A lot of times, users want to skip few messages that cause issues and

[jira] [Commented] (KAFKA-4702) Parametrize streams benchmarks to run at scale

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858546#comment-15858546 ] ASF GitHub Bot commented on KAFKA-4702: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2478: KAFKA-4702: Parametrize streams benchmarks to run ...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2478 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Resolved] (KAFKA-4702) Parametrize streams benchmarks to run at scale

2017-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4702?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4702. -- Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2478

[GitHub] kafka pull request #2514: MINOR: Add logging when commitSync fails in Stream...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2514 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[GitHub] kafka pull request #2517: HOTFIX: renamed test so it is picked up by ducktap...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2517 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Resolved] (KAFKA-4745) KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame

2017-02-08 Thread Will Droste (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Will Droste resolved KAFKA-4745. Resolution: Duplicate Fix Version/s: 0.10.1.1 > KafkaLZ4BlockOutputStream.java incorrectly

[jira] [Commented] (KAFKA-4745) KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame

2017-02-08 Thread Will Droste (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858450#comment-15858450 ] Will Droste commented on KAFKA-4745: Excellent thanks for the quick reply, next time I'll check the

[jira] [Updated] (KAFKA-4716) Fix logic for re-checking if internal topic is ready

2017-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4716: - Labels: architecture (was: ) > Fix logic for re-checking if internal topic is ready >

[jira] [Commented] (KAFKA-4716) Fix logic for re-checking if internal topic is ready

2017-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858439#comment-15858439 ] Guozhang Wang commented on KAFKA-4716: -- Coming from this commit:

[jira] [Commented] (KAFKA-4745) KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame

2017-02-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858434#comment-15858434 ] Ismael Juma commented on KAFKA-4745: This code has changed since 0.10.1.1 to be closer to what you

[jira] [Updated] (KAFKA-4648) Improve test coverage StreamTask

2017-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4648?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4648: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-4648) Improve test coverage StreamTask

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858429#comment-15858429 ] ASF GitHub Bot commented on KAFKA-4648: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2451: KAFKA-4648: Improve test coverage StreamTask

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2451 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Created] (KAFKA-4745) KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame

2017-02-08 Thread Will Droste (JIRA)
Will Droste created KAFKA-4745: -- Summary: KafkaLZ4BlockOutputStream.java incorrectly finishes the last frame Key: KAFKA-4745 URL: https://issues.apache.org/jira/browse/KAFKA-4745 Project: Kafka

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-08 Thread Steven Schlansker
Yes, thank you everyone for your input! I will incorporate the latest round of test revisions and submit a VOTE thread later today :) > On Feb 8, 2017, at 10:48 AM, Eno Thereska wrote: > > Steven, > > Sounds like we can start a VOTE thread on this? Is the KIP up to

[jira] [Commented] (KAFKA-4195) support throttling on request rate

2017-02-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4195?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858410#comment-15858410 ] Jun Rao commented on KAFKA-4195: Ismael brought up a good point that it may not be easy to set a request

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-08 Thread Eno Thereska
Steven, Sounds like we can start a VOTE thread on this? Is the KIP up to date with all the latest comments? Thanks Eno > On 8 Feb 2017, at 18:05, Matthias J. Sax wrote: > > I like this idea. But to get clean and concise PRs, I would prefer to > have a JIRA and extra PR

Re: Streams: TTLCacheStore

2017-02-08 Thread Elias Levy
The use case is a simple one. You can think of it as an update mechanism. One stream is a set of tuples consisting of consumer id, an object id, the value of some property of the object, and a timestamp. This stream represents a record of what we told some consumer the value of some property of

Build failed in Jenkins: kafka-trunk-jdk7 #1921

2017-02-08 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4741; Fix potential buffer leak in RecordAccumulator in case of -- [...truncated 19627 lines...] org.apache.kafka.streams.processor.TopologyBuilderTest >

[jira] [Updated] (KAFKA-4462) Improved Kafka Client Compatibility Policy

2017-02-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-4462: --- Fix Version/s: 0.10.2.0 > Improved Kafka Client Compatibility Policy >

[jira] [Updated] (KAFKA-4506) Refactor AbstractRequest to contain version information

2017-02-08 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4506?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-4506: --- Fix Version/s: 0.10.2.0 > Refactor AbstractRequest to contain version information >

[jira] [Assigned] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-4740: -- Assignee: Sébastien Launay > Using new consumer API with a Deserializer that throws

Re: Streams: TTLCacheStore

2017-02-08 Thread Guozhang Wang
Hello Elias, I would love to solicit more feedbacks from the community on how commonly used a TTL persistent KV store. Maybe you can share your use cases first here in this thread? As for its implementation, I think leveraging rocksdb's TTL feature would be a good option. One tricky part though,

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-08 Thread Roger Hoover
Thanks. I found the other discussion thread. Sorry for being behind on this. I'm interested in the future impersonation use cases. This seems to get us closer. +1 (non-binding) On Wed, Feb 8, 2017 at 4:41 AM, Manikumar wrote: > Hi Roger, > > In the current

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-08 Thread Manikumar
Hi Jason, As noticed by you, the current proposal does not support rotation of secret. We also discussed about maintaining a list of secret keys. Other option could be using the controller to generate and rotate secret and distribute it to all brokers. I will update the possible alternatives to

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

2017-02-08 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4741; Fix potential buffer leak in RecordAccumulator in case of -- [...truncated 33580 lines...] kafka.security.auth.SimpleAclAuthorizerTest >

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-08 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858165#comment-15858165 ] Sébastien Launay commented on KAFKA-4740: - [~ijuma] Sure, I can take a peek at it. I'll submit a

[GitHub] kafka pull request #2517: HOTFIX: renamed test so it is picked up by ducktap...

2017-02-08 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2517 HOTFIX: renamed test so it is picked up by ducktape You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka hotfix-broker-test

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-08 Thread Ismael Juma
Thanks for elaborating Jay. I totally agree that we have to be very careful in how we use our complexity budget. Easier said than done when people don't agree on what is complex and what is simple. :) For example, I think batch APIs are a significant source of complexity as you have to do a bunch

[jira] [Created] (KAFKA-4744) Streams_bounce test failing occassionally

2017-02-08 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-4744: --- Summary: Streams_bounce test failing occassionally Key: KAFKA-4744 URL: https://issues.apache.org/jira/browse/KAFKA-4744 Project: Kafka Issue Type: Bug

[jira] [Assigned] (KAFKA-4741) Memory leak in RecordAccumulator.append

2017-02-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-4741: -- Assignee: Satish Duggana Done. > Memory leak in RecordAccumulator.append >

[jira] [Commented] (KAFKA-4741) Memory leak in RecordAccumulator.append

2017-02-08 Thread Satish Duggana (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858059#comment-15858059 ] Satish Duggana commented on KAFKA-4741: --- [~ijuma] Thanks for merging the PR. Can you set me as

FINAL REMINDER: CFP for ApacheCon closes February 11th

2017-02-08 Thread Rich Bowen
Dear Apache Enthusiast, This is your FINAL reminder that the Call for Papers (CFP) for ApacheCon Miami is closing this weekend - February 11th. This is your final opportunity to submit a talk for consideration at this event. This year, we are running several mini conferences in conjunction with

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15858013#comment-15858013 ] Ismael Juma commented on KAFKA-4740: This is a tricky one. It may be worth prototyping the option

[jira] [Commented] (KAFKA-4741) Memory leak in RecordAccumulator.append

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15857996#comment-15857996 ] ASF GitHub Bot commented on KAFKA-4741: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2509: KAFKA-4741 Fix for buffer leaks in RecordAccumulat...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2509 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Resolved] (KAFKA-4741) Memory leak in RecordAccumulator.append

2017-02-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4741. Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2509

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-08 Thread Manikumar
Hi Roger, In the current proposal, we only allow a user to get delegation token for that user only. Anyone who gets that token can impersonate the user on the broker. Yes, In future we can extend the support to allow a user to acquire delegation tokens for other users. Pl refer discuss mail

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-08 Thread Jorge Esteban Quilcate Otoya
Thanks for the feedback! @Onur, @Gwen: Agree. Actually at the first draft I considered to have it inside ´kafka-consumer-groups.sh´, but I decide to propose it as a standalone tool to describe it clearly and focus it on reset functionality. But now that you mentioned, it does make sense to have

Re: [DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2017-02-08 Thread Ismael Juma
Hi Roger, Sorry for the delay. SCRAM is specified by: https://tools.ietf.org/html/rfc5802 The following quote is relevant: A SCRAM mechanism name is a string "SCRAM-" followed by the > uppercased name of the underlying hash function taken from the IANA > "Hash Function Textual Names" registry

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

2017-02-08 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2516: MINOR: add GlobalKTable doc to streams.html

2017-02-08 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2516 MINOR: add GlobalKTable doc to streams.html Update streams.html with GlobalKTable docs You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka

Jenkins build is back to normal : kafka-trunk-jdk7 #1920

2017-02-08 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-08 Thread Manikumar
Hi Jun, > If a token expires, then every broker will potentially try to delete it > around the same time, but only one will succeed. So, we will have to deal > with failures in that case? Another way is to let just one broker (say, the > controller) deletes expired tokens. > > Agree, we can run

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-08 Thread Damian Guy
Matthias, Fair point. I'll update it the KIP. Thanks On Wed, 8 Feb 2017 at 05:49 Matthias J. Sax wrote: > Damian, > > I am not strict about it either. However, if there is no advantage in > disabling it, we might not want to allow it. This would have the > advantage to

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-08 Thread Dong Lin
Hi all, Thank you all for the helpful suggestion. I have updated the KIP to address the comments received so far. See here to read the changes of the KIP. Here is a summary of change: - Updated the Proposed

[jira] [Commented] (KAFKA-4733) Improve Streams Reset Tool console output

2017-02-08 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15857618#comment-15857618 ] ASF GitHub Bot commented on KAFKA-4733: --- Github user asfgit closed the pull request at:

[jira] [Resolved] (KAFKA-4733) Improve Streams Reset Tool console output

2017-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4733?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4733. -- Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2503

[GitHub] kafka pull request #2503: KAFKA-4733: Improve Streams Reset Tool console out...

2017-02-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2503 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is