[jira] [Updated] (KAFKA-5111) Improve internal Task APIs

2017-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5111?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5111: --- Status: Patch Available (was: In Progress) > Improve internal Task APIs > ---

[jira] [Commented] (KAFKA-5111) Improve internal Task APIs

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5111?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979768#comment-15979768 ] ASF GitHub Bot commented on KAFKA-5111: --- GitHub user mjsax opened a pull request:

[GitHub] kafka pull request #2895: KAFKA-5111: Improve internal Task APIs

2017-04-21 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2895 KAFKA-5111: Improve internal Task APIs Refactors Task with proper interface methods `init()`, `resume()`, `commit()`, `suspend()`, and `close()`. All other methods for task handling are internal now

Re: Recording - Storm & Kafka Meetup on April 20th 2017

2017-04-21 Thread Naresh Kosgi
Thank you for posting On Fri, Apr 21, 2017 at 9:08 PM, Roshan Naik wrote: > It was a great meetup and for the benefit of those interested but unable > to attend it, here is a link to the recording : > > > > https://www.youtube.com/watch?v=kCRv6iEd7Ow > > > > List of Talks: > > - *Introd

[jira] [Commented] (KAFKA-4912) Add check for topic name length

2017-04-21 Thread Soumabrata Chakraborty (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979744#comment-15979744 ] Soumabrata Chakraborty commented on KAFKA-4912: --- Hi [~mjsax] - I checked wit

Re: Recording - Storm & Kafka Meetup on April 20th 2017

2017-04-21 Thread Andrew Psaltis
Fantastic!! Thanks for sharing, wish I could have been there. On Fri, Apr 21, 2017 at 21:08 Roshan Naik wrote: > It was a great meetup and for the benefit of those interested but unable > to attend it, here is a link to the recording : > > https://www.youtube.com/watch?v=kCRv6iEd7Ow > > List of T

Recording - Storm & Kafka Meetup on April 20th 2017

2017-04-21 Thread Roshan Naik
It was a great meetup and for the benefit of those interested but unable to attend it, here is a link to the recording : https://www.youtube.com/watch?v=kCRv6iEd7Ow List of Talks: - Introduction – Suresh Srinivas (Hortonworks) - [4m:31sec] – Overview of Storm 1.1 - Hugo

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Stephane Maarek
Hi Ismael, Good points I think I was headed in that direction: https://github.com/apache/kafka/pull/2894 1. That’s a possibility. I’m just unsure about how the message format will evolve in future versions, because adding constructors is painful if more parameters come into play. The approach

[GitHub] kafka pull request #2894: [KAFKA-5092] [WIP] changed ProducerRecord interfac...

2017-04-21 Thread simplesteph
GitHub user simplesteph opened a pull request: https://github.com/apache/kafka/pull/2894 [KAFKA-5092] [WIP] changed ProducerRecord interface - KIP 141 You can merge this pull request into a Git repository by running: $ git pull https://github.com/simplesteph/kafka producer-rec

[jira] [Commented] (KAFKA-5092) KIP 141 - ProducerRecord Interface Improvements

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5092?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979656#comment-15979656 ] ASF GitHub Bot commented on KAFKA-5092: --- GitHub user simplesteph opened a pull reque

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Ismael Juma
Thanks for the KIP. A possible alternative: 1. Add constructor ProducerRecord(String topic, K key, V value, Long timestamp). This provides an unambiguous constructor that allows one to pass a timestamp without a partition, which is the main requirement of the KIP. We could also consider: 2. Add

[jira] [Updated] (KAFKA-5092) KIP 141 - ProducerRecord Interface Improvements

2017-04-21 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5092?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stephane Maarek updated KAFKA-5092: --- Description: See KIP here: https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+Produce

[jira] [Updated] (KAFKA-5092) KIP 141 - ProducerRecord Interface Improvements

2017-04-21 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5092?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stephane Maarek updated KAFKA-5092: --- Summary: KIP 141 - ProducerRecord Interface Improvements (was: KIP 141 - ProducerRecordBuilde

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Stephane Maarek
Agreed! I’ll update the KIP shortly I’m self-taught, so I guess I still have a lot to learn ( If anything, I think the withForcedPartition method could be just removed, and if users need to force partition, shouldn’t they just mandatorily use a custom Partitioner? It would achieve the same purp

[jira] [Comment Edited] (KAFKA-4795) Confusion around topic deletion

2017-04-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979597#comment-15979597 ] Vahid Hashemian edited comment on KAFKA-4795 at 4/21/17 11:40 PM: --

[jira] [Commented] (KAFKA-4795) Confusion around topic deletion

2017-04-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979597#comment-15979597 ] Vahid Hashemian commented on KAFKA-4795: [~ecomar] Thanks for your feedback. What

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

2017-04-21 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-5112) Trunk compatibility tests should test against 0.10.2

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979584#comment-15979584 ] ASF GitHub Bot commented on KAFKA-5112: --- GitHub user cmccabe opened a pull request:

[jira] [Commented] (KAFKA-5112) Trunk compatibility tests should test against 0.10.2

2017-04-21 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979585#comment-15979585 ] Colin P. McCabe commented on KAFKA-5112: https://github.com/apache/kafka/pull/2893

[GitHub] kafka pull request #2893: KAFKA-5112: Trunk compatibility tests should test ...

2017-04-21 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2893 KAFKA-5112: Trunk compatibility tests should test against 0.10.2 You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka KAFKA-5112 Alter

[jira] [Commented] (KAFKA-5110) ConsumerGroupCommand error handling improvement

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979581#comment-15979581 ] ASF GitHub Bot commented on KAFKA-5110: --- GitHub user hachikuji opened a pull request

[GitHub] kafka pull request #2892: KAFKA-5110: Check for errors when fetching the log...

2017-04-21 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2892 KAFKA-5110: Check for errors when fetching the log end offset in ConsumerGroupCommand You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachiku

[jira] [Created] (KAFKA-5112) Trunk compatibility tests should test against 0.10.2

2017-04-21 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5112: -- Summary: Trunk compatibility tests should test against 0.10.2 Key: KAFKA-5112 URL: https://issues.apache.org/jira/browse/KAFKA-5112 Project: Kafka Issue

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

2017-04-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5090: Fix Kafka Streams SessionStore.findSessions javadoc broken [wangguoz] MINOR: fix javadoc comment of Transformer -- [...truncated 791.

[jira] [Resolved] (KAFKA-523) OffsetRequest handler does not handle errors

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-523?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-523. - Resolution: Fixed > OffsetRequest handler does not handle errors >

[jira] [Commented] (KAFKA-523) OffsetRequest handler does not handle errors

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-523?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979526#comment-15979526 ] Guozhang Wang commented on KAFKA-523: - I think this issue has already been fixed in {{h

[jira] [Created] (KAFKA-5111) Improve internal Task APIs

2017-04-21 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5111: -- Summary: Improve internal Task APIs Key: KAFKA-5111 URL: https://issues.apache.org/jira/browse/KAFKA-5111 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-454) ProducerSendThread calls ListBuffer.size a whole bunch. That is a O(n) operation

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-454. - Resolution: Won't Fix > ProducerSendThread calls ListBuffer.size a whole bunch. That is a O(n) > o

[jira] [Work started] (KAFKA-5111) Improve internal Task APIs

2017-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5111?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5111 started by Matthias J. Sax. -- > Improve internal Task APIs > -- > >

[jira] [Updated] (KAFKA-617) Refactor KafkaApis.handleListOffsetRequest

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-617?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-617: Summary: Refactor KafkaApis.handleListOffsetRequest (was: Refactor KafkaApis.handleOffsetRequest) >

[jira] [Commented] (KAFKA-617) Refactor KafkaApis.handleOffsetRequest

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-617?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979524#comment-15979524 ] Guozhang Wang commented on KAFKA-617: - [~jozi-k] I think it has been named to {{handleL

[jira] [Commented] (KAFKA-454) ProducerSendThread calls ListBuffer.size a whole bunch. That is a O(n) operation

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979525#comment-15979525 ] Guozhang Wang commented on KAFKA-454: - Thanks for pointing out. > ProducerSendThread c

[jira] [Commented] (KAFKA-5110) ConsumerGroupCommand error handling improvement

2017-04-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979502#comment-15979502 ] Vahid Hashemian commented on KAFKA-5110: [~cotedm] Could you please share the step

[jira] [Updated] (KAFKA-5107) remove preferred replica election state from ControllerContext

2017-04-21 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5107?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-5107: Summary: remove preferred replica election state from ControllerContext (was: investigate removing

[jira] [Updated] (KAFKA-5016) Consumer hang in poll method while rebalancing is in progress

2017-04-21 Thread Domenico Di Giulio (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5016?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Domenico Di Giulio updated KAFKA-5016: -- I am currently out of the office, with limited access to my e-mail. I will be back at work o

[jira] [Commented] (KAFKA-5016) Consumer hang in poll method while rebalancing is in progress

2017-04-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979482#comment-15979482 ] Vahid Hashemian commented on KAFKA-5016: [~domenico74] I think you are seeing the

[jira] [Comment Edited] (KAFKA-5055) Kafka Streams skipped-records-rate sensor producing nonzero values even when FailOnInvalidTimestamp is used as extractor

2017-04-21 Thread Nikki Thean (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979433#comment-15979433 ] Nikki Thean edited comment on KAFKA-5055 at 4/21/17 9:38 PM: -

[jira] [Comment Edited] (KAFKA-5055) Kafka Streams skipped-records-rate sensor producing nonzero values even when FailOnInvalidTimestamp is used as extractor

2017-04-21 Thread Nikki Thean (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979433#comment-15979433 ] Nikki Thean edited comment on KAFKA-5055 at 4/21/17 9:38 PM: -

[jira] [Commented] (KAFKA-5055) Kafka Streams skipped-records-rate sensor producing nonzero values even when FailOnInvalidTimestamp is used as extractor

2017-04-21 Thread Nikki Thean (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979433#comment-15979433 ] Nikki Thean commented on KAFKA-5055: [~guozhang] Sure. Streams config properties were

Re: [DISCUSS] KIP 130: Expose states of active tasks to KafkaStreams public API

2017-04-21 Thread Eno Thereska
Ok, I’m convinced. Let’s depreciate. Thanks Eno > On Apr 21, 2017, at 9:20 PM, Guozhang Wang wrote: > > I'd agree with Matthias. > > @Eno do you have any specific use case in mind to better keep the `toString` > function? > > > Guozhang > > > On Fri, Apr 21, 2017 at 11:41 AM, Matthias J. S

[jira] [Created] (KAFKA-5110) ConsumerGroupCommand error handling improvement

2017-04-21 Thread Dustin Cote (JIRA)
Dustin Cote created KAFKA-5110: -- Summary: ConsumerGroupCommand error handling improvement Key: KAFKA-5110 URL: https://issues.apache.org/jira/browse/KAFKA-5110 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #2884: MINOR: fix javadoc comment

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2884 --- 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 enab

[DISCUSS] KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs

2017-04-21 Thread Colin McCabe
Hi all, As part of the AdminClient work, we would like to add methods for adding, deleting, and listing access control lists (ACLs). I wrote up a KIP to discuss implementing requests for those operations, as well as AdminClient APIs. Take a look at: https://cwiki.apache.org/confluence/display/KA

[jira] [Commented] (KAFKA-5090) Kafka Streams SessionStore.findSessions javadoc broken

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979324#comment-15979324 ] ASF GitHub Bot commented on KAFKA-5090: --- Github user asfgit closed the pull request

[GitHub] kafka pull request #2874: KAFKA-5090 Kafka Streams SessionStore.findSessions...

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2874 --- 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 enab

[jira] [Updated] (KAFKA-5090) Kafka Streams SessionStore.findSessions javadoc broken

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-5090: - Resolution: Fixed Fix Version/s: 0.11.0.0 Status: Resolved (was: Patch Available

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

2017-04-21 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP 130: Expose states of active tasks to KafkaStreams public API

2017-04-21 Thread Guozhang Wang
I'd agree with Matthias. @Eno do you have any specific use case in mind to better keep the `toString` function? Guozhang On Fri, Apr 21, 2017 at 11:41 AM, Matthias J. Sax wrote: > It would not give the same information as the new API. Thus, it would be > inconsistent (and this would be reall

[GitHub] kafka pull request #2788: MINOR: improve test cleanup

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2788 --- 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 enab

[jira] [Updated] (KAFKA-5109) validate the output of ProducerPerformance and ConsumerPerformance in integration test

2017-04-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5109?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-5109: --- Description: Our system tests parse the output of ProducerPerformance and ConsumerPerformance. So, it would b

[jira] [Commented] (KAFKA-5100) ProducerPerformanceService failing due to parsing error

2017-04-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979291#comment-15979291 ] Jun Rao commented on KAFKA-5100: Filed https://issues.apache.org/jira/browse/KAFKA-5109 so

[GitHub] kafka pull request #2891: kafka 3355

2017-04-21 Thread garmes-gdev
GitHub user garmes-gdev opened a pull request: https://github.com/apache/kafka/pull/2891 kafka 3355 You can merge this pull request into a Git repository by running: $ git pull https://github.com/garmes-gdev/kafka trunk Alternatively you can review and apply these changes as

[jira] [Created] (KAFKA-5109) validate the output of ProducerPerformance and ConsumerPerformance in integration test

2017-04-21 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-5109: -- Summary: validate the output of ProducerPerformance and ConsumerPerformance in integration test Key: KAFKA-5109 URL: https://issues.apache.org/jira/browse/KAFKA-5109 Project: Kaf

[jira] [Commented] (KAFKA-5100) ProducerPerformanceService failing due to parsing error

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979285#comment-15979285 ] ASF GitHub Bot commented on KAFKA-5100: --- Github user asfgit closed the pull request

[jira] [Resolved] (KAFKA-5100) ProducerPerformanceService failing due to parsing error

2017-04-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-5100. Resolution: Fixed Issue resolved by pull request 2890 [https://github.com/apache/kafka/pull/2890] > Produce

[GitHub] kafka pull request #2890: KAFKA-5100: ProducerPerformanceService failing due...

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2890 --- 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 enab

Re: Kafka-Streams: Cogroup

2017-04-21 Thread Eno Thereska
Hi Kyle, Sorry for the delay in replying. I think it's worth doing a KIP for this one. One super helpful thing with KIPs is to list a few more scenarios that would benefit from this approach. In particular it seems the main benefit is from reducing the number of state stores. Does this necessarily

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-21 Thread Eno Thereska
Added the .tostream() discussion in the Wiki. Thanks Eno > On 21 Apr 2017, at 18:48, Matthias J. Sax wrote: > > I agree with Eno about the renaming. > > @Eno: can you add this to the Wiki Discussion page? > > > -Matthias > > > On 4/21/17 1:11 AM, Eno Thereska wrote: >> Hi Guozhang, >> >> T

Re: [DISCUSS] KIP 130: Expose states of active tasks to KafkaStreams public API

2017-04-21 Thread Matthias J. Sax
It would not give the same information as the new API. Thus, it would be inconsistent (and this would be really bad IMHO.) I would really like to remove (ie, deprecate) it. It was a "hot fix" to give some runtime information to the user. But with this KIP, we get a proper first class API and thus

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

2017-04-21 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Remove redundant CRC validation for non-compressed records in -- [...truncated 790.94 KB...] kafka.integration.SaslPlaintextTopicMetadataTest >

[jira] [Created] (KAFKA-5108) Add support for reading PID snapshot files to DumpLogSegments

2017-04-21 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5108: -- Summary: Add support for reading PID snapshot files to DumpLogSegments Key: KAFKA-5108 URL: https://issues.apache.org/jira/browse/KAFKA-5108 Project: Kafka

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

2017-04-21 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Remove redundant CRC validation for non-compressed records in -- [...truncated 788.24 KB...] kafka.log.LogTest > testReopenThenTruncate PASSED

Re: [VOTE] KIP-114: KTable state stores and improved semantics

2017-04-21 Thread Sriram Subramanian
+1 On Fri, Apr 21, 2017 at 11:06 AM, Guozhang Wang wrote: > +1 > > On Fri, Apr 21, 2017 at 10:52 AM, Bill Bejeck wrote: > > > +1 > > On Fri, Apr 21, 2017 at 1:48 PM Matthias J. Sax > > wrote: > > > > > +1 > > > > > > On 4/21/17 10:39 AM, Eno Thereska wrote: > > > > Hi there, > > > > > > > > Un

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Matthias J. Sax
Ismael's comment is quite reasonable. And I actually like the idea about `withForcedPartition` -- it helps newcomers to understand the API better. Even if I don't agree with some of you reasoning: > I was always told that the key is solely the determining factor Bad teacher? > I find it incredi

Re: [VOTE] KIP-114: KTable state stores and improved semantics

2017-04-21 Thread Guozhang Wang
+1 On Fri, Apr 21, 2017 at 10:52 AM, Bill Bejeck wrote: > +1 > On Fri, Apr 21, 2017 at 1:48 PM Matthias J. Sax > wrote: > > > +1 > > > > On 4/21/17 10:39 AM, Eno Thereska wrote: > > > Hi there, > > > > > > Unless there are more issues on the discuss thread, I'd like to start > > the vote on KIP

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-21 Thread Guozhang Wang
Sounds good. Thanks. On Fri, Apr 21, 2017 at 10:48 AM, Matthias J. Sax wrote: > I agree with Eno about the renaming. > > @Eno: can you add this to the Wiki Discussion page? > > > -Matthias > > > On 4/21/17 1:11 AM, Eno Thereska wrote: > > Hi Guozhang, > > > > Thanks for the feedback. Comments in

[jira] [Created] (KAFKA-5107) investigate removing preferred replica leader election state from ControllerContext

2017-04-21 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-5107: --- Summary: investigate removing preferred replica leader election state from ControllerContext Key: KAFKA-5107 URL: https://issues.apache.org/jira/browse/KAFKA-5107 Proje

[jira] [Commented] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979128#comment-15979128 ] ASF GitHub Bot commented on KAFKA-4564: --- Github user asfgit closed the pull request

[GitHub] kafka pull request #2837: KAFKA-4564: Add system test for pre-0.10 brokers

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2837 --- 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 enab

[jira] [Updated] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4564: - Resolution: Fixed Fix Version/s: 0.11.0.0 Status: Resolved (was: Patch Available

Re: [VOTE] KIP-114: KTable state stores and improved semantics

2017-04-21 Thread Bill Bejeck
+1 On Fri, Apr 21, 2017 at 1:48 PM Matthias J. Sax wrote: > +1 > > On 4/21/17 10:39 AM, Eno Thereska wrote: > > Hi there, > > > > Unless there are more issues on the discuss thread, I'd like to start > the vote on KIP-114. > https://cwiki.apache.org/confluence/display/KAFKA/KIP-114%3A+KTable+stat

Re: [VOTE] KIP-114: KTable state stores and improved semantics

2017-04-21 Thread Matthias J. Sax
+1 On 4/21/17 10:39 AM, Eno Thereska wrote: > Hi there, > > Unless there are more issues on the discuss thread, I'd like to start the > vote on KIP-114. > https://cwiki.apache.org/confluence/display/KAFKA/KIP-114%3A+KTable+state+stores+and+improved+semantics > >

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-04-21 Thread Matthias J. Sax
I agree with Eno about the renaming. @Eno: can you add this to the Wiki Discussion page? -Matthias On 4/21/17 1:11 AM, Eno Thereska wrote: > Hi Guozhang, > > Thanks for the feedback. Comments inline: > >> 1. Regarding the user-facing semantics, I thought we will claim that >> "KTables genera

[GitHub] kafka pull request #2868: Minor fix record collector

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2868 --- 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 enab

[jira] [Updated] (KAFKA-5106) Improve test coverage for broker CRC validation

2017-04-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5106: --- Description: Test cases should ensure that CRC errors are detected across all message formats

[jira] [Created] (KAFKA-5106) Improve test coverage for broker CRC validation

2017-04-21 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5106: -- Summary: Improve test coverage for broker CRC validation Key: KAFKA-5106 URL: https://issues.apache.org/jira/browse/KAFKA-5106 Project: Kafka Issue Type:

[VOTE] KIP-114: KTable state stores and improved semantics

2017-04-21 Thread Eno Thereska
Hi there, Unless there are more issues on the discuss thread, I'd like to start the vote on KIP-114. https://cwiki.apache.org/confluence/display/KAFKA/KIP-114%3A+KTable+state+stores+and+improved+semantics

[jira] [Updated] (KAFKA-5105) ReadOnlyKeyValueStore range scans are not ordered

2017-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5105: --- Issue Type: Improvement (was: Bug) > ReadOnlyKeyValueStore range scans are not ordered >

[GitHub] kafka pull request #2881: MINOR: Remove redundant CRC validation for non-com...

2017-04-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2881 --- 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 enab

[jira] [Updated] (KAFKA-5105) ReadOnlyKeyValueStore range scans are not ordered

2017-04-21 Thread Dmitry Minkovsky (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitry Minkovsky updated KAFKA-5105: Description: Following up with this thread https://www.mail-archive.com/users@kafka.apache.

[jira] [Created] (KAFKA-5105) ReadOnlyKeyValueStore range scans are not ordered

2017-04-21 Thread Dmitry Minkovsky (JIRA)
Dmitry Minkovsky created KAFKA-5105: --- Summary: ReadOnlyKeyValueStore range scans are not ordered Key: KAFKA-5105 URL: https://issues.apache.org/jira/browse/KAFKA-5105 Project: Kafka Issue T

[VOTE] 0.10.2.1 RC3

2017-04-21 Thread Gwen Shapira
Hello Kafka users, developers, friends, romans, countrypersons, This is the fourth (!) candidate for release of Apache Kafka 0.10.2.1. It is a bug fix release, so we have lots of bug fixes, some super important. Release notes for the 0.10.2.1 release: http://home.apache.org/~gwenshap/kafka-0.10.

[jira] [Commented] (KAFKA-5016) Consumer hang in poll method while rebalancing is in progress

2017-04-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979030#comment-15979030 ] Vahid Hashemian commented on KAFKA-5016: [~domenico74] Thanks for confirming. I'll

[jira] [Commented] (KAFKA-4755) SimpleBenchmark test fails for streams

2017-04-21 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15979000#comment-15979000 ] Gwen Shapira commented on KAFKA-4755: - removed 0.10.2.1 from fixVersion since it isn't

Re: [VOTE] 0.10.2.1 RC2

2017-04-21 Thread Gwen Shapira
Stevo, This is a sensible fix to backport, but since it isn't a regression and since we are on our fourth RC, I want to be super conservative here and not add anything that isn't absolutely critical. Gwen On Fri, Apr 21, 2017 at 3:05 AM, Stevo Slavić wrote: > Please consider including https://

[jira] [Updated] (KAFKA-4755) SimpleBenchmark test fails for streams

2017-04-21 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-4755: Fix Version/s: (was: 0.10.2.1) > SimpleBenchmark test fails for streams > --

[jira] [Commented] (KAFKA-5100) ProducerPerformanceService failing due to parsing error

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15978977#comment-15978977 ] ASF GitHub Bot commented on KAFKA-5100: --- GitHub user junrao opened a pull request:

[GitHub] kafka pull request #2890: KAFKA-5100: ProducerPerformanceService failing due...

2017-04-21 Thread junrao
GitHub user junrao opened a pull request: https://github.com/apache/kafka/pull/2890 KAFKA-5100: ProducerPerformanceService failing due to parsing error You can merge this pull request into a Git repository by running: $ git pull https://github.com/junrao/kafka kafka-5100 Alte

[jira] [Created] (KAFKA-5104) DumpLogSegments should not open index files with `rw`

2017-04-21 Thread Yeva Byzek (JIRA)
Yeva Byzek created KAFKA-5104: - Summary: DumpLogSegments should not open index files with `rw` Key: KAFKA-5104 URL: https://issues.apache.org/jira/browse/KAFKA-5104 Project: Kafka Issue Type: Imp

Re: [VOTE] KIP 130: Expose states of active tasks to KafkaStreams public API

2017-04-21 Thread Damian Guy
+1 On Fri, 21 Apr 2017 at 09:06 Eno Thereska wrote: > +1 (non-binding) > > Thanks > Eno > > > On 21 Apr 2017, at 05:58, Guozhang Wang wrote: > > > > +1. Thanks a lot for the KIP! > > > > Guozhang > > > > On Wed, Apr 5, 2017 at 1:57 PM, Florian Hussonnois < > fhussonn...@gmail.com> > > wrote: >

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-21 Thread Damian Guy
Thanks Michal, I agree Transformer.punctuate should also be void, but we can deprecate that too in favor of the new interface. Thanks for the javadoc PR! Cheers, Damian On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki < michal.borowie...@openbet.com> wrote: > Yes, that looks better to me. > > Not

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

2017-04-21 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Fix logging in ControlRecordType -- [...truncated 1.55 MB...] kafka.log.LogTest > testReopenThenTruncate PASSED kafka.log.LogTest > testOldPr

[jira] [Commented] (KAFKA-5100) ProducerPerformanceService failing due to parsing error

2017-04-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15978863#comment-15978863 ] Jun Rao commented on KAFKA-5100: It seems the issue is that we now print the total before

[jira] [Commented] (KAFKA-4928) Add integration test for DumpLogSegments

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15978846#comment-15978846 ] ASF GitHub Bot commented on KAFKA-4928: --- GitHub user original-brownbear opened a pul

[GitHub] kafka pull request #2889: KAFKA-4928: Add integration test for DumpLogSegmen...

2017-04-21 Thread original-brownbear
GitHub user original-brownbear opened a pull request: https://github.com/apache/kafka/pull/2889 KAFKA-4928: Add integration test for DumpLogSegments Adding tests for `kafka.tools.DumpLogSegments` You can merge this pull request into a Git repository by running: $ git pull https

[jira] [Commented] (KAFKA-5103) Refactor AdminUtils to use zkUtils methods instad of zkUtils.zkClient

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15978786#comment-15978786 ] ASF GitHub Bot commented on KAFKA-5103: --- GitHub user baluchicken opened a pull reque

[GitHub] kafka pull request #2888: KAFKA-5103 Refactor AdminUtils to use zkUtils meth...

2017-04-21 Thread baluchicken
GitHub user baluchicken opened a pull request: https://github.com/apache/kafka/pull/2888 KAFKA-5103 Refactor AdminUtils to use zkUtils methods instad of zkUti… @ijuma plz review. You can merge this pull request into a Git repository by running: $ git pull https://github.com/

[jira] [Commented] (KAFKA-5097) KafkaConsumer.poll throws IllegalStateException

2017-04-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5097?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15978766#comment-15978766 ] ASF GitHub Bot commented on KAFKA-5097: --- GitHub user ijuma opened a pull request:

[GitHub] kafka pull request #2887: KAFKA-5097: Add testFetchAfterPartitionWithFetched...

2017-04-21 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2887 KAFKA-5097: Add testFetchAfterPartitionWithFetchedRecordsIsUnassigned You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-5097-unit-

[jira] [Created] (KAFKA-5103) Refactor AdminUtils to use zkUtils methods instad of zkUtils.zkClient

2017-04-21 Thread Balint Molnar (JIRA)
Balint Molnar created KAFKA-5103: Summary: Refactor AdminUtils to use zkUtils methods instad of zkUtils.zkClient Key: KAFKA-5103 URL: https://issues.apache.org/jira/browse/KAFKA-5103 Project: Kafka

  1   2   >