Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Rajini Sivaram
Jay, Thank you for the feedback. 1. I think it is good to have a single concept of identity, but multiple ways of grouping clients for different functions. Client-id is a logical grouping of clients with a meaningful name that is used in client metrics and logs. User principal is an

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-06-10 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324450#comment-15324450 ] Edoardo Comar commented on KAFKA-3727: -- I think it would be reasonable to have the poll timeout

[jira] [Commented] (KAFKA-3775) Throttle maximum number of tasks assigned to a single KafkaStreams

2016-06-10 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324187#comment-15324187 ] Yuto Kawamura commented on KAFKA-3775: -- Sorry for leaving this discussion for a while and thanks for

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-06-10 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324424#comment-15324424 ] Edoardo Comar commented on KAFKA-3727: -- [~ijuma] [~hachikuji] ... reposting here as more on-topic

[GitHub] kafka pull request #1489: KAFKA-2945: CreateTopic - protocol and server side...

2016-06-10 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1489 KAFKA-2945: CreateTopic - protocol and server side implementation You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka

[jira] [Commented] (KAFKA-2945) CreateTopic - protocol and server side implementation

2016-06-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324731#comment-15324731 ] ASF GitHub Bot commented on KAFKA-2945: --- Github user granthenke closed the pull request at:

[GitHub] kafka pull request #626: KAFKA-2945: CreateTopic - protocol and server side ...

2016-06-10 Thread granthenke
Github user granthenke closed the pull request at: https://github.com/apache/kafka/pull/626 --- 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

Re: [DISCUSS] KIP-4 Create Topic Schema

2016-06-10 Thread Gwen Shapira
Thank you for the clear proposal, Grant! I like the request/response objects and the timeout semantics. Two comments: 1. The replica assignment protocol takes [replicas], there is the implicit assumption that the first replica is the leader. This matches current behavior elsewhere, but lets

[jira] [Commented] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324669#comment-15324669 ] Jay Kreps commented on KAFKA-3811: -- I'm not wild about introducing these levels in an ad hoc way in Kafka

[jira] [Commented] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324847#comment-15324847 ] Greg Fodor commented on KAFKA-3811: --- I've also attached a screenshot of YourKit of the relevant call

[jira] [Updated] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Fodor updated KAFKA-3811: -- Attachment: screenshot-1.png > Introduce Kafka Streams metrics recording levels >

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Jay Kreps
Hey Rajini, 1. That makes sense to me. Is that reflected in the documentation anywhere (I couldn't really find it)? Is there a way to discover that definition? We do way better when we right this stuff down so it has an official definition users and developers can work off of... 2. If client id

[jira] [Commented] (KAFKA-3803) JsonConverter deserialized Struct containing bytes field does not return true for equals()

2016-06-10 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324743#comment-15324743 ] Jay Kreps commented on KAFKA-3803: -- FWIW, past experience with giving users ByteBuffer has been pretty

[jira] [Updated] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Fodor updated KAFKA-3811: -- Attachment: Muon-Snapshot.zip > Introduce Kafka Streams metrics recording levels >

[jira] [Updated] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-10 Thread Alexander Cook (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alexander Cook updated KAFKA-3822: -- Description: I am using the KafkaConsumer java client to consume messages. My application

[jira] [Updated] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Fodor updated KAFKA-3811: -- Attachment: Muon-latency.zip > Introduce Kafka Streams metrics recording levels >

[jira] [Commented] (KAFKA-2945) CreateTopic - protocol and server side implementation

2016-06-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324728#comment-15324728 ] ASF GitHub Bot commented on KAFKA-2945: --- GitHub user granthenke opened a pull request:

[DISCUSS] KIP-4 Create Topic Schema

2016-06-10 Thread Grant Henke
Now that Kafka 0.10 has been released I would like to start work on the new protocol messages and client implementation for KIP-4. In order to break up the discussion and feedback I would like to continue breaking up the content in to smaller pieces. This discussion thread is for the CreateTopic

[jira] [Commented] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread aarti gupta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324755#comment-15324755 ] aarti gupta commented on KAFKA-3811: [~gfodor] Can you please share and outline the profiling and

[jira] [Commented] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread aarti gupta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324760#comment-15324760 ] aarti gupta commented on KAFKA-3811: >>Make the metrics lower overhead (this is an issue in the

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

2016-06-10 Thread Gwen Shapira
>From what I can see, remaining questions are: 1. Who / how are tokens renewed? By original requester only? or using Kerberos auth only? 2. Are tokens stored on each broker or in ZK? 3. How are tokens invalidated / expired? 4. Which encryption algorithm is used? 5. What is the impersonation

[jira] [Commented] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324838#comment-15324838 ] Greg Fodor commented on KAFKA-3811: --- Hey [~aartigupta], I ran an attached yourkit profiler to one of our

[jira] [Commented] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324869#comment-15324869 ] Greg Fodor commented on KAFKA-3811: --- Also, I've attached a screenshot + snapshot of a second run where I

[jira] [Updated] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Fodor updated KAFKA-3811: -- Attachment: screenshot-latency.png > Introduce Kafka Streams metrics recording levels >

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Gwen Shapira
I am not crazy about modes either. An earlier proposal supported both client-ids and users at the same time, and it made more sense to me. I believe it was dropped without proper discussion (Basically, Jun mentioned it is complex and Rajini agreed to drop it). We should probably rethink the

[jira] [Created] (KAFKA-3823) QuickStart documentation is still showing MirrorMakers supports more than one consumer.config

2016-06-10 Thread Chak Lee (JIRA)
Chak Lee created KAFKA-3823: --- Summary: QuickStart documentation is still showing MirrorMakers supports more than one consumer.config Key: KAFKA-3823 URL: https://issues.apache.org/jira/browse/KAFKA-3823

Re: [DISCUSS] scalability limits in the coordinator

2016-06-10 Thread Jason Gustafson
Hey Becket, My suggestion was pretty far from a completely thought-out proposal, but the advantages of having your MM cluster maintain subscriptions/assignments in its own topic are the following: 1. It solves the immediate problem of the size of the group metadata. 2. It distributes the

[jira] [Commented] (KAFKA-3775) Throttle maximum number of tasks assigned to a single KafkaStreams

2016-06-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325258#comment-15325258 ] Guozhang Wang commented on KAFKA-3775: -- Hi [~kawamuray] Thanks for sharing your usage scenarios, it

[jira] [Resolved] (KAFKA-3817) KTableRepartitionMap should handle null inputs

2016-06-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3817?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3817. -- Resolution: Fixed Issue resolved by pull request 1488

[GitHub] kafka pull request #1488: KAFKA-3817: handle null keys in KTableRepartitionM...

2016-06-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1488 --- 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-3801) Provide static serialize() and deserialize() for use as method references

2016-06-10 Thread Jeff Klukas (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3801?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325270#comment-15325270 ] Jeff Klukas commented on KAFKA-3801: I like concise code that you get from a static method reference:

[jira] [Updated] (KAFKA-3753) Add approximateNumEntries() to the StateStore interface for metrics reporting

2016-06-10 Thread Jeff Klukas (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Klukas updated KAFKA-3753: --- Summary: Add approximateNumEntries() to the StateStore interface for metrics reporting (was: Add

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Rajini Sivaram
I do think client-id is a valid and useful grouping for quotas even in secure clusters - but only if clientA of user1 is treated as a different client-id from clientA of user2. Grouping of clients of a user enables users to allocate their quota effectively to their clients (eg. guarantee that

[jira] [Assigned] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2016-06-10 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ishita Mandhan reassigned KAFKA-2857: - Assignee: Ishita Mandhan > ConsumerGroupCommand throws

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Jay Kreps
This sounds a lot better to me--hopefully it isn't too much harder! I do think if it is possible to do this directly that will be better for users than having an intermediate step since we always have to work through migrating people who have setup quotas already from the old way to the new way.

[jira] [Commented] (KAFKA-3801) Provide static serialize() and deserialize() for use as method references

2016-06-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3801?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325209#comment-15325209 ] Guozhang Wang commented on KAFKA-3801: -- I think a general question here, that we are considering, is

[GitHub] kafka pull request #1490: KAFKA-3769: Optimize metrics recording overhead

2016-06-10 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1490 KAFKA-3769: Optimize metrics recording overhead You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K3769-optimize-metrics

[jira] [Commented] (KAFKA-3769) KStream job spending 60% of time writing metrics

2016-06-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325148#comment-15325148 ] ASF GitHub Bot commented on KAFKA-3769: --- GitHub user guozhangwang opened a pull request:

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2016-06-10 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325152#comment-15325152 ] Ishita Mandhan commented on KAFKA-2857: --- Working on this with [~vahid] and we aren't sure about what

[jira] [Commented] (KAFKA-3817) KTableRepartitionMap should handle null inputs

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

[jira] [Comment Edited] (KAFKA-3791) Broken tools -- need better way to get offsets and other info

2016-06-10 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325481#comment-15325481 ] Vahid Hashemian edited comment on KAFKA-3791 at 6/10/16 11:15 PM: -- I

[jira] [Comment Edited] (KAFKA-3791) Broken tools -- need better way to get offsets and other info

2016-06-10 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325481#comment-15325481 ] Vahid Hashemian edited comment on KAFKA-3791 at 6/10/16 11:15 PM: -- I

[jira] [Commented] (KAFKA-1120) Controller could miss a broker state change

2016-06-10 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325496#comment-15325496 ] Jun Rao commented on KAFKA-1120: A better way is probably for the controller to store the ZK version of

[jira] [Commented] (KAFKA-3693) Race condition between highwatermark-checkpoint thread and handleLeaderAndIsrRequest at broker start-up

2016-06-10 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325510#comment-15325510 ] Jun Rao commented on KAFKA-3693: [~maysamyabandeh], thanks for confirming this. The root of the problem

[jira] [Commented] (KAFKA-3791) Broken tools -- need better way to get offsets and other info

2016-06-10 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325481#comment-15325481 ] Vahid Hashemian commented on KAFKA-3791: I agree that the deprecation message is not as clear as

[jira] [Commented] (KAFKA-3818) Change Mirror Maker default assignment strategy to round robin

2016-06-10 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325499#comment-15325499 ] Vahid Hashemian commented on KAFKA-3818: [~hachikuji] Since _round robin_ provides a more balanced

[GitHub] kafka pull request #1495: Minor: Respect the default value for partition arg...

2016-06-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1495 Minor: Respect the default value for partition argument in SimpleConsumerShell The `partition` argument is not marked as required, and has a default of `0`, according to the tool's help

[jira] [Commented] (KAFKA-3693) Race condition between highwatermark-checkpoint thread and handleLeaderAndIsrRequest at broker start-up

2016-06-10 Thread Maysam Yabandeh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325563#comment-15325563 ] Maysam Yabandeh commented on KAFKA-3693: [~junrao] oh yeah that makes sense. Thanks for your help

[jira] [Commented] (KAFKA-1981) Make log compaction point configurable

2016-06-10 Thread Eric Wasserman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325578#comment-15325578 ] Eric Wasserman commented on KAFKA-1981: --- [~ijuma] A new pull request is available

[GitHub] kafka pull request #1491: remove size and message count constraints

2016-06-10 Thread ewasserman
GitHub user ewasserman opened a pull request: https://github.com/apache/kafka/pull/1491 remove size and message count constraints KAFKA-1981Make log compaction point configurable Reduced scope to controlling only minimum time before compaction. Changed to using message

Jenkins build is back to normal : kafka-0.10.0-jdk7 #123

2016-06-10 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2016-06-10 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325356#comment-15325356 ] Dustin Cote commented on KAFKA-2394: [~ewencp] makes sense, I went ahead and updated the docs in the

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

2016-06-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3817: handle null keys in KTableRepartitionMap -- [...truncated 6269 lines...] kafka.log.LogTest > testParseTopicPartitionNameForNull PASSED kafka.log.LogTest >

[GitHub] kafka pull request #1491: remove size and message count constraints

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

[GitHub] kafka pull request #1491: remove size and message count constraints

2016-06-10 Thread ewasserman
GitHub user ewasserman reopened a pull request: https://github.com/apache/kafka/pull/1491 remove size and message count constraints KAFKA-1981Make log compaction point configurable Reduced scope to controlling only minimum time before compaction. Changed to using message

[jira] [Updated] (KAFKA-3818) Change Mirror Maker default assignment strategy to round robin

2016-06-10 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3818: --- Status: Patch Available (was: Open) > Change Mirror Maker default assignment strategy to

[jira] [Updated] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2016-06-10 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dustin Cote updated KAFKA-2394: --- Fix Version/s: 0.11.0.0 > Use RollingFileAppender by default in log4j.properties >

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Jun Rao
Rajini, The new proposal sounds good to me too. My only question is what happens to those existing quotas on client-id. Do we just treat them as the quota for that client-id under ANONYMOUS user name? Thanks, Jun On Fri, Jun 10, 2016 at 2:43 PM, Rajini Sivaram < rajinisiva...@googlemail.com>

[jira] [Commented] (KAFKA-3818) Change Mirror Maker default assignment strategy to round robin

2016-06-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325436#comment-15325436 ] ASF GitHub Bot commented on KAFKA-3818: --- Github user vahidhashemian closed the pull request at:

[GitHub] kafka pull request #1492: KAFKA-3818: Change Mirror Maker default assignment...

2016-06-10 Thread vahidhashemian
Github user vahidhashemian closed the pull request at: https://github.com/apache/kafka/pull/1492 --- 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

[jira] [Commented] (KAFKA-3801) Provide static serialize() and deserialize() for use as method references

2016-06-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3801?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325454#comment-15325454 ] Guozhang Wang commented on KAFKA-3801: -- Before we introduced Kafka Streams, we did not consider

[GitHub] kafka pull request #1491: remove size and message count constraints

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

[jira] [Resolved] (KAFKA-3801) Provide static serialize() and deserialize() for use as method references

2016-06-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3801?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3801. -- Resolution: Won't Fix > Provide static serialize() and deserialize() for use as method

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-10 Thread Rajini Sivaram
Jay, Thank you for the quick feedback. It shouldn't be too hard since I had a PR earlier along these lines anyway. Jun, are you ok with this approach? If everyone agrees, I will close this vote, update the KIP and give some more time for discussions. On Fri, Jun 10, 2016 at 10:27 PM, Jay Kreps

[GitHub] kafka pull request #1492: Change Mirror Maker default assignment strategy to...

2016-06-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1492 Change Mirror Maker default assignment strategy to round robin Update Mirror Maker to to use round robin assignment by default since it gives a better balance between the instances, in

[GitHub] kafka pull request #1493: KAFKA-3818: Change Mirror Maker default assignment...

2016-06-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1493 KAFKA-3818: Change Mirror Maker default assignment strategy to round robin Update Mirror Maker to use round robin assignment by default since it gives a better balance between the instances,

[jira] [Commented] (KAFKA-3818) Change Mirror Maker default assignment strategy to round robin

2016-06-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325441#comment-15325441 ] ASF GitHub Bot commented on KAFKA-3818: --- GitHub user vahidhashemian opened a pull request:

[jira] [Created] (KAFKA-3824) Docs indicate auto.commit breaks at least once delivery but that is incorrect

2016-06-10 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-3824: Summary: Docs indicate auto.commit breaks at least once delivery but that is incorrect Key: KAFKA-3824 URL: https://issues.apache.org/jira/browse/KAFKA-3824 Project: Kafka

[jira] [Commented] (KAFKA-3797) Improve security of __consumer_offsets topic

2016-06-10 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15323930#comment-15323930 ] Todd Palino commented on KAFKA-3797: Obviously we can't do something like that with a running

[jira] [Commented] (KAFKA-3809) Auto-generate documentation for topic-level configuration

2016-06-10 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15323967#comment-15323967 ] James Cheng commented on KAFKA-3809: [~jjkoshy], would you be open to splitting

[jira] [Commented] (KAFKA-3806) Adjust default values of log.retention.hours and offsets.retention.minutes

2016-06-10 Thread Michal Turek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324020#comment-15324020 ] Michal Turek commented on KAFKA-3806: - Hi, thanks for the advises. We will workaround the issue by

[jira] [Commented] (KAFKA-3594) Kafka new producer retries doesn't work in 0.9.0.1

2016-06-10 Thread Petr Novotnik (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3594?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324104#comment-15324104 ] Petr Novotnik commented on KAFKA-3594: -- Any chance for this to be back ported to a 0.9.0.x release?

[jira] [Assigned] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-06-10 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar reassigned KAFKA-3727: Assignee: Edoardo Comar > Consumer.poll() stuck in loop on non-existent topic manually

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-06-10 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324429#comment-15324429 ] Ismael Juma commented on KAFKA-3727: It's due to auto-topic creation as you mentioned. >

[jira] [Assigned] (KAFKA-3811) Introduce Kafka Streams metrics recording levels

2016-06-10 Thread aarti gupta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] aarti gupta reassigned KAFKA-3811: -- Assignee: aarti gupta (was: Guozhang Wang) > Introduce Kafka Streams metrics recording levels

[jira] [Comment Edited] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-06-10 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324624#comment-15324624 ] Edoardo Comar edited comment on KAFKA-3727 at 6/10/16 3:26 PM: --- would it

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-06-10 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324624#comment-15324624 ] Edoardo Comar commented on KAFKA-3727: -- would it make sense that the missing topic/partition