Re: [VOTE] KIP-33 - Add a time based log index

2016-04-07 Thread Guozhang Wang
Great job Jiangjie. A few comments: 1. "if an offset index entry is inserted, it will also insert a time index entry" what is the motivation for co-inserting offset index and timestamp index? Is it just for simplicity or are there any other considerations? 2. "Search message by timestamp": you m

[jira] [Updated] (KAFKA-725) Broker Exception: Attempt to read with a maximum offset less than start offset

2016-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-725: Reviewer: Guozhang Wang > Broker Exception: Attempt to read with a maximum offset less than start off

[jira] [Updated] (KAFKA-725) Broker Exception: Attempt to read with a maximum offset less than start offset

2016-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-725: Assignee: Stig Rohde Døssing (was: Jay Kreps) > Broker Exception: Attempt to read with a maximum off

[jira] [Commented] (KAFKA-725) Broker Exception: Attempt to read with a maximum offset less than start offset

2016-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15231622#comment-15231622 ] Guozhang Wang commented on KAFKA-725: - [~Srdo] I think your reasoning still makes sense

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Don Bosco Durai
Ranger team would prefer option #2. Right now, we have to access some of the nested constants using constructs like Group$.MODULE$, which is not intuitive in Java. Thanks Bosco On 4/7/16, 4:30 PM, "Ashish Singh" wrote: >Harsha/ Don, > >Are you guys OK with option 2? I am not aware of all

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

2016-04-07 Thread Apache Jenkins Server
See

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

2016-04-07 Thread Apache Jenkins Server
See Changes: [me] MINOR: ensure original use of prop_file in verifiable producer -- [...truncated 1601 lines...] at hudson.scm.SCM.compareRemoteRevisionWith(SCM.java:381) at huds

[jira] [Updated] (KAFKA-3480) Autogenerate metrics documentation

2016-04-07 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-3480: --- Attachment: Screen Shot 2016-04-07 at 6.52.19 PM.png Screenshot of autogenerated metric docs. > Autog

[jira] [Updated] (KAFKA-3480) Autogenerate metrics documentation

2016-04-07 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-3480: --- Attachment: sample_metrics.html Generated docs for the sample metrics from TestSpecificMetrics.java. O

[jira] [Commented] (KAFKA-3480) Autogenerate metrics documentation

2016-04-07 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15231481#comment-15231481 ] James Cheng commented on KAFKA-3480: [~hachikuji] and [~gwenshap], I uploaded a work-i

[GitHub] kafka pull request: [WIP] KAFKA-3480: Autogenerate metrics documen...

2016-04-07 Thread wushujames
GitHub user wushujames opened a pull request: https://github.com/apache/kafka/pull/1202 [WIP] KAFKA-3480: Autogenerate metrics documentation This is a proof of concept for autogenerating documentation for metrics, to see whether this approach is feasible. I applied it to th

[jira] [Commented] (KAFKA-3480) Autogenerate metrics documentation

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

[GitHub] kafka pull request: MINOR: ensure original use of prop_file in ver...

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

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

2016-04-07 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3488; Avoid failing of unsent requests in consumer where possible -- [...truncated 3127 lines...] kafka.server.LogOffsetTest > testEmptyLogsGetOffsets PASSED kafka.serv

[jira] [Updated] (KAFKA-3528) AbstractCoordinator wakeup during rebalance can cause spurious rebalances

2016-04-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3528: --- Status: Patch Available (was: Open) > AbstractCoordinator wakeup during rebalance can cause s

[GitHub] kafka pull request: KAFKA-3528: handle wakeups while rebalancing m...

2016-04-07 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1201 KAFKA-3528: handle wakeups while rebalancing more gracefully You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-3528 Alter

[jira] [Commented] (KAFKA-3528) AbstractCoordinator wakeup during rebalance can cause spurious rebalances

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

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

2016-04-07 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3488; Avoid failing of unsent requests in consumer where possible -- [...truncated 7082 lines...] org.apache.kafka.streams.kstream.internals.KStreamTransformValuesTest >

[jira] [Created] (KAFKA-3528) AbstractCoordinator wakeup during rebalance can cause spurious rebalances

2016-04-07 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-3528: -- Summary: AbstractCoordinator wakeup during rebalance can cause spurious rebalances Key: KAFKA-3528 URL: https://issues.apache.org/jira/browse/KAFKA-3528 Project:

[jira] [Created] (KAFKA-3527) Consumer commitAsync should not expose internal exceptions

2016-04-07 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-3527: -- Summary: Consumer commitAsync should not expose internal exceptions Key: KAFKA-3527 URL: https://issues.apache.org/jira/browse/KAFKA-3527 Project: Kafka

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Ashish Singh
Harsha/ Don, Are you guys OK with option 2? I am not aware of all the existing authorizer implementations, however ranger has one for sure. Getting direct feedback from you guys will be really valuable. On Thu, Apr 7, 2016 at 3:52 PM, Ismael Juma wrote: > Hi Don, > > This is true in Java 7, but

[jira] [Commented] (KAFKA-3488) commitAsync() fails if metadata update creates new SASL/SSL connection

2016-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3488?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15231326#comment-15231326 ] Guozhang Wang commented on KAFKA-3488: -- [~hachikuji] [~rsivaram] I think the original

[jira] [Resolved] (KAFKA-3498) Transient failure in kafka.api.SslConsumerTest.testSimpleConsumption

2016-04-07 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3498?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3498. Resolution: Duplicate > Transient failure in kafka.api.SslConsumerTest.testSimpleConsumption > -

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Ismael Juma
Hi Don, This is true in Java 7, but Java 8 introduces default methods and this workaround is no longer required. During the Interceptor KIP discussion, it was decided that it was fine to stick to interfaces given that we are likely to move to Java 8 in the nearish future (probably no later than th

[jira] [Updated] (KAFKA-3488) commitAsync() fails if metadata update creates new SASL/SSL connection

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

[jira] [Resolved] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2016-04-07 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-2672. Resolution: Fixed Assignee: Rajini Sivaram (was: Jason Gustafson) Fixed as part of KAFKA-3488

[jira] [Commented] (KAFKA-3488) commitAsync() fails if metadata update creates new SASL/SSL connection

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

[GitHub] kafka pull request: KAFKA-3488: Avoid failing of unsent requests i...

2016-04-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1183 --- 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-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-07 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei updated KAFKA-3526: -- Status: Patch Available (was: In Progress) > REST APIs return object representation instead of string f

[jira] [Work started] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-07 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3526 started by Liquan Pei. - > REST APIs return object representation instead of string for config values, > defaul

[GitHub] kafka pull request: KAFKA-3526: Return string instead of object in...

2016-04-07 Thread Ishiihara
GitHub user Ishiihara opened a pull request: https://github.com/apache/kafka/pull/1200 KAFKA-3526: Return string instead of object in ConfigKeyInfo and ConfigValueInfo You can merge this pull request into a Git repository by running: $ git pull https://github.com/Ishiihara/ka

[jira] [Commented] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

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

[jira] [Updated] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-07 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei updated KAFKA-3526: -- Description: In the response of {code} PUT /connector-plugins/{name}/config/validate {code} The value.v

[jira] [Updated] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-07 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei updated KAFKA-3526: -- Description: In the response of {code} PUT /connector-plugins/{name}/config/validate {code} The value.v

[jira] [Created] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-07 Thread Liquan Pei (JIRA)
Liquan Pei created KAFKA-3526: - Summary: REST APIs return object representation instead of string for config values, default values and recommended values Key: KAFKA-3526 URL: https://issues.apache.org/jira/browse/KA

[jira] [Updated] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-07 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei updated KAFKA-3526: -- Description: In the response of PUT /connector-plugins/{name}/config/validate, The value.value, value.

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Don Bosco Durai
Hi Ashish If we are going by option #2, then I can suggest we give an abstract implementation of the Interface and recommend anyone implementing their own plugin to extend from the abstract class, rather than implement the interface? The advantage is, in the future if we add add any new methods

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Harsha
Gwen, Ashish, Rolling upgrade should be fine in this case. Users can bring down single broker at a time and upgrade their kafka binaries along with new authorizer implementation . This looks fine to me. Thanks, Harsha On Thu

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Ashish Singh
Harsha, No, ZK data format for acls or SimplaAclAthorizer implementation is not proposed to be changed. Gwen, The concern I was pointing at is that if we choose to go with Option 2, i.e., move interface to a separate package, authorizer implementation will have to be updated along with broker up

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Harsha
Ashish, Thanks for the details. We are not changing any of the zookeeper data format for acls right? Thanks, Harsha On Thu, Apr 7, 2016, at 11:25 AM, Gwen Shapira wrote: > Can you guys go into details on what will happen during a rolling upgrade > exactly? > > Gwen > > On Thu, Apr 7

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

2016-04-07 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3497: Streams ProcessorContext should support forward() based on -- [...truncated 7608 lines...] org.apache.kafka.streams.kstream.internals.KStreamTransformValuesT

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

2016-04-07 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3497: Streams ProcessorContext should support forward() based on -- [...truncated 1589 lines...] kafka.consumer.ZookeeperConsumerConnectorTest > testConsumerDecode

[jira] [Commented] (KAFKA-3524) ConnectorTaskId should also also include cluster id

2016-04-07 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3524?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15230788#comment-15230788 ] Ewen Cheslack-Postava commented on KAFKA-3524: -- [~fpj] Can you explain the us

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Gwen Shapira
Can you guys go into details on what will happen during a rolling upgrade exactly? Gwen On Thu, Apr 7, 2016 at 11:21 AM, Ashish Singh wrote: > Hello Harsha, > > On Thu, Apr 7, 2016 at 11:03 AM, Harsha wrote: > > "My only ask is to have this in 0.10. As Jay pointed out, right now > > there > >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Gwen Shapira
Harsha and Rajini, We need a good way to advertise to clients whether the broker they are talking to supports the new KIP-43 methods or not. So a Storm client will know whether it can use a delegation token or needs to revert to older implementation. Alternative proposals are more than welcome at

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Ashish Singh
Hello Harsha, On Thu, Apr 7, 2016 at 11:03 AM, Harsha wrote: "My only ask is to have this in 0.10. As Jay pointed out, right now > there > are not many implementations out there, we might want to fix it ASAP." > > Probably there aren't many implementations but there are lot of users > using thes

[jira] [Updated] (KAFKA-3525) max.reserved.broker.id off-by-one error

2016-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3525?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-3525: -- Fix Version/s: 0.10.0.0 > max.reserved.broker.id off-by-one error >

[jira] [Updated] (KAFKA-3525) max.reserved.broker.id off-by-one error

2016-04-07 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3525?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-3525: -- Priority: Blocker (was: Minor) > max.reserved.broker.id off-by-one error >

[jira] [Created] (KAFKA-3525) max.reserved.broker.id off-by-one error

2016-04-07 Thread Alan Braithwaite (JIRA)
Alan Braithwaite created KAFKA-3525: --- Summary: max.reserved.broker.id off-by-one error Key: KAFKA-3525 URL: https://issues.apache.org/jira/browse/KAFKA-3525 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Harsha
Magnus, I am not in favor moving this SASL/SSL hand shake to KafkaApllication level protocol. I am not sure what evolution of the current handshake needs versioning and also merging Application level protocol with SSL/SASL handshake doesn't seem ri

Re: [VOTE] KIP-4 Metadata Schema

2016-04-07 Thread Grant Henke
Thanks for the feedback Guozhang and Gwen. Gwen, I agree with you on this. I am not sure its something we can/should tackle here. Especially before the release. I can leave the delete flag off of the changes. What that means for KIP-4, is that a client won't be able to differentiate between a top

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Harsha
"My only ask is to have this in 0.10. As Jay pointed out, right now there are not many implementations out there, we might want to fix it ASAP." Probably there aren't many implementations but there are lot of users using these implementations in production clusters. Isn't this going to break the

Re: [VOTE] KIP-4 Metadata Schema

2016-04-07 Thread Gwen Shapira
Given that we are very close to the release, if we are changing the Metadata cache + topic deletion logic, I'd like a good number of system tests to appear with the patch. On Thu, Apr 7, 2016 at 10:53 AM, Gwen Shapira wrote: > This will change some logic though, right? > > IIRC, right now produc

Re: [VOTE] KIP-4 Metadata Schema

2016-04-07 Thread Gwen Shapira
This will change some logic though, right? IIRC, right now produce/fetch requests to marked-for-deletion topics fail because the topics are simple not around. You get a generic "doesn't exist" error. If we keep these topics and add a flag, we'll need to find all the places with this implicit logic

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Ashish Singh
We might want to take a call here. Following are the options. 1. Let KIP-50 be the way it is, i.e., just add getDescription() to existing scala authorizer interface. It will break binary compatibility (only when using CLI and/or AdminCommand from >= 0.10 against authorizer implementati

Re: [VOTE] KIP-4 Metadata Schema

2016-04-07 Thread Guozhang Wang
Hmm, I think since in the original protocol, metadata response do not have information for "marked for deleted topics" and hence we want to remove that topic from returning in response by cleaning the metadata cache once it is marked to deletion. With the new format, I think it is OK to delay the

[jira] [Commented] (KAFKA-3497) Streams ProcessorContext should support forward() based on child name

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

[jira] [Resolved] (KAFKA-3497) Streams ProcessorContext should support forward() based on child name

2016-04-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3497. -- Resolution: Fixed Issue resolved by pull request 1194 [https://github.com/apache/kafka/pull/1194

[GitHub] kafka pull request: KAFKA-3497: Streams ProcessorContext should su...

2016-04-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1194 --- 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-3524) ConnectorTaskId should also also include cluster id

2016-04-07 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-3524: --- Summary: ConnectorTaskId should also also include cluster id Key: KAFKA-3524 URL: https://issues.apache.org/jira/browse/KAFKA-3524 Project: Kafka Issue

[jira] [Commented] (KAFKA-3513) Transient failure of OffsetValidationTest

2016-04-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3513?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15230573#comment-15230573 ] Jason Gustafson commented on KAFKA-3513: [~ewencp] Here's a PR with the logging im

[GitHub] kafka pull request: MINOR: improve logging of consumer system test...

2016-04-07 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1199 MINOR: improve logging of consumer system tests You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka improve-consumer-systests Alt

Re: [VOTE] KIP-4 Metadata Schema

2016-04-07 Thread Grant Henke
I am testing the marked for deletion flag in the metadata and ran into some challenges. It turns out that as soon as a topic is marked for deletion it may be purged from the metadata cache. This means that Metadata responses can't/don't return the topic. Though the topic may still exist if its not

[jira] [Work started] (KAFKA-3504) Changelog partition configured to enable log compaction

2016-04-07 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3504 started by Eno Thereska. --- > Changelog partition configured to enable log compaction > ---

[jira] [Assigned] (KAFKA-3504) Changelog partition configured to enable log compaction

2016-04-07 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-3504: --- Assignee: Eno Thereska > Changelog partition configured to enable log compaction > --

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Rajini Sivaram
Magnus, *"Why would it be harder to achieve? And is it harder for the Kafka developeror the user?Wouldnt it be possible to accept both non-SSL-auth and SSL-auth clients onthe same port?"* Maybe I am missing something, so perhaps it would be easier if I explained our scenario and you could tell

Re: KStreams Group Rebalance Commit Error

2016-04-07 Thread Michael D. Coon
One more thing I'm noticing in the logs. I see periodic node disconnection messages due to "timeout". I set my metadata.fetch.timeout.ms to 6, request.timeout.ms to 3 and timeout.ms to 3 and those should be more than enough time waiting for metadata responses. I also set my offset co

[jira] [Updated] (KAFKA-3490) Multiple version support for ducktape performance tests

2016-04-07 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3490: --- Reviewer: Ewen Cheslack-Postava Status: Patch Available (was: Open) > Multiple version support

Re: KStreams Group Rebalance Commit Error

2016-04-07 Thread Michael D. Coon
Guozhang,    Thanks for the advice; however, "max.poll.records" doesn't seem to be supported since it's not affecting how many records are coming back from the consumer.poll requests. However, I agree that the likely culprit in rebalancing is the delay in processing new records. I'm going to try

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-07 Thread Ismael Juma
+1 (non-binding) Ismael On Thu, Apr 7, 2016 at 1:25 AM, Jason Gustafson wrote: > Minor note: I've changed the restart API to be blocking as discussed in the > other thread. > > -Jason > > On Tue, Apr 5, 2016 at 9:02 PM, Guozhang Wang wrote: > > > +1 > > > > On Tue, Apr 5, 2016 at 8:06 PM, Neha

[GitHub] kafka pull request: MINOR: Fix wrong comments

2016-04-07 Thread gyk
GitHub user gyk opened a pull request: https://github.com/apache/kafka/pull/1198 MINOR: Fix wrong comments You can merge this pull request into a Git repository by running: $ git pull https://github.com/gyk/kafka fix-comment Alternatively you can review and apply these change