Re: [DISCUSS] KIP-54 Sticky Partition Assignment Strategy

2016-04-29 Thread Edoardo Comar
Hi why is the calculation of the partition assignments to group member being executed by the client (leader of the group), rather than server (eg by the group Coordinator) ? This question came up working with Vahid Hashemian on https://issues.apache.org/jira/browse/KAFKA-2273 We have impleme

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2016-04-29 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15263678#comment-15263678 ] Stig Rohde Døssing commented on KAFKA-2729: --- We hit this on 0.9.0.1 today {code}

[jira] [Updated] (KAFKA-1880) Add support for checking binary/source compatibility

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1880?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-1880: --- Attachment: compatibilityReport-only-incompatible.html Adding a more complete sample report that only

[DISCUSS] mbeans overwritten with identical clients on a single jvm

2016-04-29 Thread Onur Karaman
Hey everyone. I think we might need to have an actual discussion on an issue I brought up a while ago in https://issues.apache.org/jira/browse/KAFKA-3494. It seems like client-ids are being used for too many things today: 1. kafka-request.log. This helps if you ever want to associate a client with

[jira] [Assigned] (KAFKA-3637) Add method that checks if streams are initialised

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei reassigned KAFKA-3637: - Assignee: Liquan Pei > Add method that checks if streams are initialised > --

[jira] [Assigned] (KAFKA-3213) [CONNECT] It looks like we are not backing off properly when reconfiguring tasks

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3213?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei reassigned KAFKA-3213: - Assignee: Liquan Pei (was: Ewen Cheslack-Postava) > [CONNECT] It looks like we are not backing o

[GitHub] kafka pull request: KAFKA-3613: Consolidate TumblingWindows and Ho...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1277 --- 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] [Commented] (KAFKA-3613) Consolidate tumbling windows and hopping windows

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

[jira] [Resolved] (KAFKA-3613) Consolidate tumbling windows and hopping windows

2016-04-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3613. -- Resolution: Fixed Fix Version/s: 0.10.0.0 Issue resolved by pull request 1277 [https://gi

[GitHub] kafka pull request: KAFKA-3598: Improve JavaDoc of public API

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1250 --- 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] [Commented] (KAFKA-3598) Improve JavaDoc of public API

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

[jira] [Updated] (KAFKA-3598) Improve JavaDoc of public API

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

[jira] [Updated] (KAFKA-2693) Run relevant ducktape tests with SASL/PLAIN and multiple mechanisms

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2693: - Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.10

[GitHub] kafka pull request: KAFKA-2693: Ducktape tests for SASL/PLAIN and ...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1282 --- 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] [Commented] (KAFKA-2693) Run relevant ducktape tests with SASL/PLAIN and multiple mechanisms

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

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-29 Thread Jason Gustafson
Hey Harsha, One issue with adding back subscribe(List), but marking it deprecated is that it may confuse some users if they use the typical Arrays.asList() pattern. You'd have to cast to a Collection to avoid the deprecation warning, which is awkward. Maybe it would be better in that case to keep

[GitHub] kafka pull request: KAFKA-3559: lazy initialisation of state store...

2016-04-29 Thread enothereska
Github user enothereska closed the pull request at: https://github.com/apache/kafka/pull/1223 --- 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-3559) Task creation time taking too long in rebalance callback

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264350#comment-15264350 ] ASF GitHub Bot commented on KAFKA-3559: --- Github user enothereska closed the pull req

[jira] [Updated] (KAFKA-3559) Task creation time taking too long in rebalance callback

2016-04-29 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3559?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-3559: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Task creation time taking too long

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

2016-04-29 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-29 Thread Grant Henke
I think you are right Jason. People were definitely on the fence about this and we went back and forth for quite some time. I think the main point in the KIP discussion that made this decision, is that the Consumer was annotated with the Unstable annotation. Given how new the Consumer is, we wante

[GitHub] kafka pull request: KAFKA-3418: add javadoc section describing con...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1129 --- 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-3418) Add section on detecting consumer failures in new consumer javadoc

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3418: - Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.10

[jira] [Commented] (KAFKA-3418) Add section on detecting consumer failures in new consumer javadoc

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

[GitHub] kafka pull request: KAFKA-3615: Exclude test jars in kafka-run-cla...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1263 --- 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] [Created] (KAFKA-3642) Fix NPE from ProcessorStateManager when the changelog topic not exists

2016-04-29 Thread Yuto Kawamura (JIRA)
Yuto Kawamura created KAFKA-3642: Summary: Fix NPE from ProcessorStateManager when the changelog topic not exists Key: KAFKA-3642 URL: https://issues.apache.org/jira/browse/KAFKA-3642 Project: Kafka

[jira] [Resolved] (KAFKA-3615) Exclude test jars in CLASSPATH of kafka-run-class.sh

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-3615. -- Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.

[jira] [Commented] (KAFKA-3615) Exclude test jars in CLASSPATH of kafka-run-class.sh

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

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-29 Thread Grant Henke
If anyone wants to review the KIP call discussion we had on this just before the vote, here is a link to the relevant session (6 minutes in): https://youtu.be/Hcjur17TjBE?t=6m On Fri, Apr 29, 2016 at 12:21 PM, Grant Henke wrote: > I think you are right Jason. People were definitely on the fence

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

2016-04-29 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-56 Allow cross origin HTTP requests on all HTTP methods

2016-04-29 Thread Ewen Cheslack-Postava
This passes w/ 5 binding and 2 non-binding votes (it wasn't explicit, but I'm assuming Liquan was an implicit +1 by starting the thread). Thanks for voting everyone! -Ewen On Wed, Apr 27, 2016 at 10:04 PM, Ismael Juma wrote: > +1 > > On Wed, Apr 27, 2016 at 1:54 PM, Grant Henke wrote: > > > +

[GitHub] kafka pull request: KAFKA-3597: Query ConsoleConsumer and Verifiab...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1278 --- 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-3597) Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3597: Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.10.1.0 St

[jira] [Commented] (KAFKA-3597) Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly

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

[jira] [Resolved] (KAFKA-3578) Allow cross origin HTTP requests on all HTTP methods

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-3578. -- Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.

[GitHub] kafka pull request: KAFKA-3642: Fix NPE from ProcessorStateManager...

2016-04-29 Thread kawamuray
GitHub user kawamuray opened a pull request: https://github.com/apache/kafka/pull/1289 KAFKA-3642: Fix NPE from ProcessorStateManager when the changelog topic not exists Issue: https://issues.apache.org/jira/browse/KAFKA-3642 You can merge this pull request into a Git repository by

[GitHub] kafka pull request: KAFKA-3578: Allow cross origin HTTP requests o...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1288 --- 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] [Commented] (KAFKA-3578) Allow cross origin HTTP requests on all HTTP methods

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

[jira] [Updated] (KAFKA-3642) Fix NPE from ProcessorStateManager when the changelog topic not exists

2016-04-29 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-3642: - Status: Patch Available (was: Open) > Fix NPE from ProcessorStateManager when the changelog topic

[jira] [Commented] (KAFKA-3642) Fix NPE from ProcessorStateManager when the changelog topic not exists

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

[GitHub] kafka pull request: KAFKA-3634: Upgrade tests for SASL authenticat...

2016-04-29 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1290 KAFKA-3634: Upgrade tests for SASL authentication Add a test for changing SASL mechanism using rolling upgrade and a test for rolling upgrade from 0.9.0.x to 0.10.0 with SASL/GSSAPI. You can

[jira] [Commented] (KAFKA-3634) Add ducktape tests for upgrade with SASL

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3634?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264439#comment-15264439 ] ASF GitHub Bot commented on KAFKA-3634: --- GitHub user rajinisivaram opened a pull req

[jira] [Commented] (KAFKA-3128) Add metrics for ZooKeeper events

2016-04-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264445#comment-15264445 ] Jun Rao commented on KAFKA-3128: [~fpj], yes, it's a good idea to track both session expir

[jira] [Updated] (KAFKA-1880) Add support for checking binary/source compatibility

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1880?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-1880: --- Description: Recent discussions around compatibility shows how important compatibility is to users. Ka

[jira] [Commented] (KAFKA-3634) Add ducktape tests for upgrade with SASL

2016-04-29 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3634?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264450#comment-15264450 ] Rajini Sivaram commented on KAFKA-3634: --- [~ijuma] Upgrade test from 0.9.0.x was bein

[jira] [Updated] (KAFKA-3634) Add ducktape tests for upgrade with SASL

2016-04-29 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3634?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3634: -- Status: Patch Available (was: Open) > Add ducktape tests for upgrade with SASL > --

[jira] [Commented] (KAFKA-1880) Add support for checking binary/source compatibility

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1880?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264455#comment-15264455 ] ASF GitHub Bot commented on KAFKA-1880: --- GitHub user granthenke opened a pull reques

[GitHub] kafka pull request: WIP - KAFKA-1880: Add support for checking bin...

2016-04-29 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1291 WIP - KAFKA-1880: Add support for checking binary/source compatibility This is a WIP pull request to show how I am generating the reports attached to the Jira. I am putting it up now so that we u

[GitHub] kafka pull request: KAFKA-3618: Handle ApiVersionsRequest before S...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1286 --- 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] [Commented] (KAFKA-3618) Handle ApiVersionRequest before SASL handshake

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

[jira] [Updated] (KAFKA-3618) Handle ApiVersionRequest before SASL handshake

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3618?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3618: Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.10.1.0 St

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

2016-04-29 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2693: Ducktape tests for SASL/PLAIN and multiple mechanisms [me] KAFKA-3418: add javadoc section describing consumer failure detection [me] KAFKA-3615: Exclude test jars in kafka-run-class.sh -

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1292 KAFKA-3641: Fix RecordMetadata constructor backward compatibility You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka recordmeta

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264502#comment-15264502 ] ASF GitHub Bot commented on KAFKA-3641: --- GitHub user granthenke opened a pull reques

Re: [DISCUSS] mbeans overwritten with identical clients on a single jvm

2016-04-29 Thread Onur Karaman
fixing the cc for navina. On Fri, Apr 29, 2016 at 1:06 AM, Onur Karaman wrote: > Hey everyone. I think we might need to have an actual discussion on an > issue I brought up a while ago in > https://issues.apache.org/jira/browse/KAFKA-3494. It seems like > client-ids are being used for too many t

[jira] [Updated] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3641: --- Affects Version/s: 0.10.0.0 > Fix RecordMetadata constructor backward compatibility > ---

[jira] [Updated] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3641: --- Status: Patch Available (was: Open) > Fix RecordMetadata constructor backward compatibility > --

[jira] [Updated] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3641: --- Fix Version/s: 0.10.0.0 > Fix RecordMetadata constructor backward compatibility > ---

[GitHub] kafka pull request: HOTFIX: wrong keyvalue equals logic when keys ...

2016-04-29 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/1293 HOTFIX: wrong keyvalue equals logic when keys not equal but values equal With the previous logic, if key does NOT equal, but value DOES equal, then equals returns TRUE. You can merge this pull

[jira] [Created] (KAFKA-3643) Data Duplication on clean restart of Kafka Broker

2016-04-29 Thread Arun Mathew (JIRA)
Arun Mathew created KAFKA-3643: -- Summary: Data Duplication on clean restart of Kafka Broker Key: KAFKA-3643 URL: https://issues.apache.org/jira/browse/KAFKA-3643 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3643) Data Duplication on clean restart of Kafka Broker

2016-04-29 Thread Arun Mathew (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264543#comment-15264543 ] Arun Mathew commented on KAFKA-3643: [~gwenshap] This is the issue I talked to you abo

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-04-29 Thread Nisarg Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264548#comment-15264548 ] Nisarg Shah commented on KAFKA-3209: Hey, I'm looking to contribute to open source and

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

2016-04-29 Thread Apache Jenkins Server
See

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

2016-04-29 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2693: Ducktape tests for SASL/PLAIN and multiple mechanisms [me] KAFKA-3418: add javadoc section describing consumer failure detection [me] KAFKA-3615: Exclude test jars in kafka-run-class.sh

"nag" PR 1143

2016-04-29 Thread Zack Dever
Just a friendly reminder for this minor PR https://github.com/apache/kafka/pull/1143 as per the instructions on http://kafka.apache.org/contributing.html. It received a +1, but had a request for a minor test change which I made. Thanks! Zack

[GitHub] kafka pull request: KAFKA-3440: update JavaDoc

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1287 --- 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] [Commented] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

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

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264627#comment-15264627 ] Gwen Shapira commented on KAFKA-3209: - Thank you Nisarg. I believe [~ewencp] already s

[jira] [Updated] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3440: - Resolution: Fixed Reviewer: Ewen Cheslack-Postava Status: Resolved (

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264633#comment-15264633 ] Ewen Cheslack-Postava commented on KAFKA-3209: -- [~snisarg] I haven't started

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264645#comment-15264645 ] ASF GitHub Bot commented on KAFKA-3641: --- Github user granthenke closed the pull requ

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread granthenke
Github user granthenke closed the pull request at: https://github.com/apache/kafka/pull/1292 --- 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-3209) Support single message transforms in Kafka Connect

2016-04-29 Thread Nisarg Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264644#comment-15264644 ] Nisarg Shah commented on KAFKA-3209: That does sound good. I realise it is not insanel

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264649#comment-15264649 ] ASF GitHub Bot commented on KAFKA-3641: --- GitHub user granthenke reopened a pull requ

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread granthenke
GitHub user granthenke reopened a pull request: https://github.com/apache/kafka/pull/1292 KAFKA-3641: Fix RecordMetadata constructor backward compatibility You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka recordme

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

2016-04-29 Thread Apache Jenkins Server
See

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

2016-04-29 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3440: Update streams javadocs -- [...truncated 1641 lines...] kafka.log.LogTest > testLogRolls PASSED kafka.log.LogTest > testMessageSizeCheck PASSED kafka.log.LogTest

[jira] [Closed] (KAFKA-3582) remove references to Copcyat from connect property files

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3582?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3582. - > remove references to Copcyat from connect property files > -

[jira] [Closed] (KAFKA-3615) Exclude test jars in CLASSPATH of kafka-run-class.sh

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3615. - > Exclude test jars in CLASSPATH of kafka-run-class.sh >

[jira] [Closed] (KAFKA-3606) Traverse CLASSPATH during herder start to list connectors

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3606. - > Traverse CLASSPATH during herder start to list connectors >

[jira] [Closed] (KAFKA-3578) Allow cross origin HTTP requests on all HTTP methods

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3578. - > Allow cross origin HTTP requests on all HTTP methods >

[jira] [Closed] (KAFKA-3611) Remove WARNs when using reflections

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3611?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3611. - > Remove WARNs when using reflections > > > Key: KAFKA-3

[jira] [Commented] (KAFKA-3128) Add metrics for ZooKeeper events

2016-04-29 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264741#comment-15264741 ] James Cheng commented on KAFKA-3128: +1. These metrics would be super useful. I know

[jira] [Commented] (KAFKA-3128) Add metrics for ZooKeeper events

2016-04-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264750#comment-15264750 ] Ismael Juma commented on KAFKA-3128: Thanks [~junrao], I'll update the PR to use `Kafk

[jira] [Commented] (KAFKA-3643) Data Duplication on clean restart of Kafka Broker

2016-04-29 Thread Arun Mathew (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264820#comment-15264820 ] Arun Mathew commented on KAFKA-3643: Further Details of the Issue. When an event is r

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

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

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1292 --- 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-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3641: Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.10.1.0 St

[GitHub] kafka pull request: HOTFIX: Fix equality semantics of KeyValue

2016-04-29 Thread miguno
GitHub user miguno opened a pull request: https://github.com/apache/kafka/pull/1294 HOTFIX: Fix equality semantics of KeyValue Fixes wrong KeyValue equals logic when keys not equal but values equal. Original hotfix PR at https://github.com/apache/kafka/pull/1293 (/cc @enoth

Re: [DISCUSS] mbeans overwritten with identical clients on a single jvm

2016-04-29 Thread Jay Kreps
The definition for client id has always been "a logical name for an application which (potentially) spans more than one process". >From my point of view the rationalization that is most needed is client id with "user" for the authenticated cases. There not quite the same but they're similar. I th

[GitHub] kafka pull request: KAFKA-3459: Returning zero task configurations...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1248 --- 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-3459) Returning zero task configurations from a connector does not properly clean up existing tasks

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3459: - Resolution: Fixed Fix Version/s: 0.10.1.0 Status: Resolved (was:

[jira] [Commented] (KAFKA-3459) Returning zero task configurations from a connector does not properly clean up existing tasks

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

[RELEASE] Final merge of trunk into 0.10.0

2016-04-29 Thread Gwen Shapira
Hi, I just merged trunk into 0.10.0 branch and pushed. 0.10.0 is updated as of commit d0dedc6 (KAFKA-3459: Returning zero task configurations from a connector does not properly clean up existing tasks). Committers: Please cherry-pick only critical bug fixes and/or low-risk changes (preferably te

[GitHub] kafka pull request: HOTFIX: Fix equality semantics of KeyValue

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1294 --- 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

[GitHub] kafka pull request: KAFKA-3627: consumer fails to execute delayed ...

2016-04-29 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1295 KAFKA-3627: consumer fails to execute delayed tasks in poll when records are available You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachik

[jira] [Commented] (KAFKA-3627) New consumer doesn't run delayed tasks while under load

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

[jira] [Created] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-3644: -- Summary: Use Boolean protocol type for StopReplicaRequest delete_partitions Key: KAFKA-3644 URL: https://issues.apache.org/jira/browse/KAFKA-3644 Project: Kafka

[jira] [Updated] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3644: --- Status: Patch Available (was: Open) > Use Boolean protocol type for StopReplicaRequest delete_partiti

[jira] [Commented] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15264892#comment-15264892 ] ASF GitHub Bot commented on KAFKA-3644: --- GitHub user granthenke opened a pull reques

  1   2   >