[GitHub] kafka pull request #1544: MINOR: Clean `Metrics.defaultRegistry` to avoid tr...

2016-06-23 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1544 MINOR: Clean `Metrics.defaultRegistry` to avoid transient failures in `testSessionExpireListenerMetrics` The failure could manifest itself if the default metrics registry had some entries from other

[jira] [Commented] (KAFKA-3892) Clients retain metadata for non-subscribed topics

2016-06-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15345949#comment-15345949 ] Ismael Juma commented on KAFKA-3892: Yes, we would want to understand the root cause to make sure we

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

2016-06-23 Thread Onur Karaman
>From what I understood, it seems that stickiness is preserved only for the remaining live consumers. Say a consumer owns some partitions and then dies. Those partitions will get redistributed to the rest of the group. Now if the consumer comes back up, based on the algorithm described with the

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

2016-06-23 Thread Andrew Coates
Hi all, I think sticky assignment is immensely important / useful in many situations. Apps that use Kafka are many and varied. Any app that stores any state, either in the form of data from incoming messages, cached results from previous out-of-process calls or expensive operations, (and let's

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

2016-06-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346113#comment-15346113 ] Ismael Juma commented on KAFKA-1981: Sorry for the delay [~ewasserman]. I will try to take a look, but

KTable.filter usage, memory consumption and materialized view semantics

2016-06-23 Thread Philippe Derome
I made a modification of latest Confluent's example UserRegionLambdaExample. See relevant code at end of email. Am I correct in understanding that KTable semantics should be similar to a store-backed cache of a view as (per wikipedia on materialized views) or similar to Oracle's materialized

[jira] [Commented] (KAFKA-3892) Clients retain metadata for non-subscribed topics

2016-06-23 Thread Noah Sloan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346321#comment-15346321 ] Noah Sloan commented on KAFKA-3892: --- I can reproduce but I don't have code I can share (and you would

[jira] [Comment Edited] (KAFKA-3892) Clients retain metadata for non-subscribed topics

2016-06-23 Thread Noah Sloan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346321#comment-15346321 ] Noah Sloan edited comment on KAFKA-3892 at 6/23/16 12:03 PM: - I can reproduce

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

2016-06-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346428#comment-15346428 ] Ismael Juma commented on KAFKA-1981: About the config name, we probably should try to reach consensus

Re: [DISCUSS] KIP-4 Delete Topic Schema

2016-06-23 Thread Grant Henke
Hi Guozhang, Thanks for the review! In the timeout <= 0 case, if the client should always ignore and treat > the timeout > error code as "OK", should we just return no error code in this case? The wiki behavior documentation was there, but a question was never brought up for create or delete.

[jira] [Commented] (KAFKA-3894) Log Cleaner thread crashes and never restarts

2016-06-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346485#comment-15346485 ] Ismael Juma commented on KAFKA-3894: The exception looks like the one in KAFKA-3587, which was fixed

[jira] [Commented] (KAFKA-3894) Log Cleaner thread crashes and never restarts

2016-06-23 Thread Tom Crayford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346463#comment-15346463 ] Tom Crayford commented on KAFKA-3894: - (disclaimer: I work with Tim) It feels like there are a few

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

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

[GitHub] kafka pull request #1545: KAFKA-3727 - ClientListener for UnknownTopicOrPart...

2016-06-23 Thread edoardocomar
GitHub user edoardocomar opened a pull request: https://github.com/apache/kafka/pull/1545 KAFKA-3727 - ClientListener for UnknownTopicOrPartitionException Added a ClientListener to KafkaConsumer, Fetcher and Client Modified Fetcher and NetworkClient to notify listeners for

[jira] [Commented] (KAFKA-2359) New consumer - partitions auto assigned only on poll

2016-06-23 Thread Elias Levy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2359?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346678#comment-15346678 ] Elias Levy commented on KAFKA-2359: --- I wonder if the calling pause() before calling poll() and

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

2016-06-23 Thread Jun Rao
Hi, Rajini, For the following statements, would it be better to allocate the quota to all connections whose client-id is clientX? This way, existing client-id quotas are fully compatible in the new release whether the cluster is in a single user or multi-user environment. 4. If client-id quota

[jira] [Commented] (KAFKA-3894) Log Cleaner thread crashes and never restarts

2016-06-23 Thread The Data Lorax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346583#comment-15346583 ] The Data Lorax commented on KAFKA-3894: --- [~ijuma] Not sure this is related. Our situation seems to

[jira] [Commented] (KAFKA-3894) Log Cleaner thread crashes and never restarts

2016-06-23 Thread The Data Lorax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346580#comment-15346580 ] The Data Lorax commented on KAFKA-3894: --- Yep, we've ran into the same issue. Would be nice if the

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

2016-06-23 Thread Andrew Coates
Hey Jason, Good to know on the round robin assignment. I'll look into that. The issue I have with the current rebalance listener is that it's not intuitive and unnecessarily exposes the inner workings of rebalance logic. When the onPartitionsRevoked method is called it's not really saying the

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

2016-06-23 Thread Vahid S Hashemian
Hi Onur, Your understanding is correct. If a consumer dies and later comes back, with the current proposal, there is no guarantee that it would reclaim its previous assignment. Regards, --Vahid From: Onur Karaman To: dev@kafka.apache.org Date:

[jira] [Created] (KAFKA-3895) Implement a default queue for expired messages

2016-06-23 Thread Krish Iyer (JIRA)
Krish Iyer created KAFKA-3895: - Summary: Implement a default queue for expired messages Key: KAFKA-3895 URL: https://issues.apache.org/jira/browse/KAFKA-3895 Project: Kafka Issue Type: New

[jira] [Commented] (KAFKA-3846) Connect record types should include timestamps

2016-06-23 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3846?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346891#comment-15346891 ] Shikhar Bhushan commented on KAFKA-3846:

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

2016-06-23 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347092#comment-15347092 ] Ashish K Singh commented on KAFKA-3822: --- Turns out KAFKA-3879 is a dup of this, had some discussion

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

2016-06-23 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347169#comment-15347169 ] Edoardo Comar commented on KAFKA-3727: -- Thanks [~hachikuji], sure I could start a small KIP - an

[DISCUSS] KIP-65 Expose timestamps to Connect

2016-06-23 Thread Shikhar Bhushan
Kafkarati, Here is a pretty straightforward proposal, for exposing timestamps that were added in Kafka 0.10 to the connect framework so connectors can make use of them: https://cwiki.apache.org/confluence/display/KAFKA/KIP-65%3A+Expose+timestamps+to+Connect Appreciate your thoughts! Shikhar

[jira] [Created] (KAFKA-3896) Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations

2016-06-23 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-3896: - Summary: Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations Key: KAFKA-3896 URL: https://issues.apache.org/jira/browse/KAFKA-3896

[jira] [Commented] (KAFKA-3896) Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations

2016-06-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347154#comment-15347154 ] Ismael Juma commented on KAFKA-3896: Including the error message as Jenkins jobs are deleted

[jira] [Commented] (KAFKA-3879) KafkaConsumer with auto commit enabled gets stuck when killed after broker is dead

2016-06-23 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347069#comment-15347069 ] Ashish K Singh commented on KAFKA-3879: --- Sure [~hachikuji], linking the JIRAs. > KafkaConsumer with

[jira] [Resolved] (KAFKA-3879) KafkaConsumer with auto commit enabled gets stuck when killed after broker is dead

2016-06-23 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3879?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh resolved KAFKA-3879. --- Resolution: Duplicate > KafkaConsumer with auto commit enabled gets stuck when killed after

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

2016-06-23 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh reassigned KAFKA-3822: - Assignee: Ashish K Singh > Kafka Consumer close() hangs indefinitely if Kafka Broker

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

2016-06-23 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347175#comment-15347175 ] Ishita Mandhan commented on KAFKA-2857: --- So it seems like we need to check if the offsets topic

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

2016-06-23 Thread Rajini Sivaram
Jun, Yes, I agree that it makes sense to retain the existing semantics for client-id quotas for compatibility. Especially if we can provide the option to enable secure client-id quotas for multi-user clusters as well. I have updated the KIP - each of these levels can have defaults as well as

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

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347193#comment-15347193 ] Jason Gustafson commented on KAFKA-2857: [~imandhan] I think maybe it can be even simpler. If the

[GitHub] kafka pull request #1546: HOTFIX: Remove Java verion 1.6 in quick-start docs

2016-06-23 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1546 HOTFIX: Remove Java verion 1.6 in quick-start docs You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Assigned] (KAFKA-3897) Improve Unit Tests for New Consumer's Regex Subscription

2016-06-23 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3897?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ishita Mandhan reassigned KAFKA-3897: - Assignee: Ishita Mandhan > Improve Unit Tests for New Consumer's Regex Subscription >

Re: KTable.filter usage, memory consumption and materialized view semantics

2016-06-23 Thread Guozhang Wang
Hello Philippe, I think your question is really in two-folds: 1. What is the semantic difference between a KTable and a KStream, and more specifically how should we interpret (key, null) in KTable? You can find some explanations in this documentation:

[GitHub] kafka pull request #1529: Kafka-3880: Disallow Join Window with size zero

2016-06-23 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1529 --- 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 #1546: HOTFIX: Remove Java verion 1.6 in quick-start docs

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

[jira] [Created] (KAFKA-3897) Improve Unit Tests for New Consumer's Regex Subscription

2016-06-23 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-3897: -- Summary: Improve Unit Tests for New Consumer's Regex Subscription Key: KAFKA-3897 URL: https://issues.apache.org/jira/browse/KAFKA-3897 Project: Kafka

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

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347269#comment-15347269 ] Jason Gustafson commented on KAFKA-3822: [~alex-cook4] I'm hoping we can fix the behavior of

[DISCUSS] KAFKA-3761: Controller has RunningAsBroker instead of RunningAsController state

2016-06-23 Thread Roger Hoover
Hi all, Does anyone have an issue with removing the broker state called "RunningAsController"? The reasons to remove it are: 1. It's currently broken. The purpose of the JIRA was to report that the RunningAsController state gets overwritten

Re: [DISCUSS] KAFKA-3761: Controller has RunningAsBroker instead of RunningAsController state

2016-06-23 Thread Ismael Juma
+1 from me. Ismael On Thu, Jun 23, 2016 at 11:57 PM, Roger Hoover wrote: > Hi all, > > Does anyone have an issue with removing the broker state called > "RunningAsController"? > > The reasons to remove it are: > 1. It's currently broken. The purpose of the JIRA >

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

2016-06-23 Thread Guozhang Wang
Just adding some related reference here: Henry Cai is contributing some advanced feature in Kafka Streams regarding static assignment: https://github.com/apache/kafka/pull/1543 The main motivation is that when you do rolling bounce for upgrading your Kafka Streams code, for example, you would

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

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347286#comment-15347286 ] Jason Gustafson commented on KAFKA-3822: [~singhashish] Yeah, that might make the most sense. On

[GitHub] kafka pull request #1544: MINOR: Clean `Metrics.defaultRegistry` to avoid tr...

2016-06-23 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1544 --- 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] [Updated] (KAFKA-3880) Disallow Join Windows with size zero

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

[GitHub] kafka pull request #1481: MINOR: Pass absolute directory path to RocksDB.ope...

2016-06-23 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1481 --- 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 Delete Topic Schema

2016-06-23 Thread Guozhang Wang
I think I'm convinced your way for exposing the exception out of the client, let's leave it as is. Guozhang On Thu, Jun 23, 2016 at 8:31 AM, Grant Henke wrote: > Hi Guozhang, > > Thanks for the review! > > In the timeout <= 0 case, if the client should always ignore and

[jira] [Commented] (KAFKA-3897) Improve Unit Tests for New Consumer's Regex Subscription

2016-06-23 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3897?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347273#comment-15347273 ] Vahid Hashemian commented on KAFKA-3897: The issue reported is KAFKA-3854 should be covered in the

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

2016-06-23 Thread Vahid S Hashemian
Hi Jason, I appreciate your feedback. Please see my comments below, and advise if you have further suggestions. Thanks. Regards, --Vahid From: Jason Gustafson To: dev@kafka.apache.org Date: 06/22/2016 04:41 PM Subject:Re: [DISCUSS] KIP-54 Sticky

[GitHub] kafka pull request #1547: KAFKA-3594; After calling MemoryRecords.close() me...

2016-06-23 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1547 KAFKA-3594; After calling MemoryRecords.close() method, hasRoomFor() method should return false This exception is occurring when producer is trying to append a record to a Re-enqueued record batch

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

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

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

2016-06-23 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #1548: KAFKA-2857 ConsumerGroupCommand throws GroupCoordi...

2016-06-23 Thread imandhan
GitHub user imandhan opened a pull request: https://github.com/apache/kafka/pull/1548 KAFKA-2857 ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created Added a check to make sure different cases

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

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

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

2016-06-23 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3809 started by James Cheng. -- > Auto-generate documentation for topic-level configuration >

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

2016-06-23 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-3809: --- Reviewer: Gwen Shapira Status: Patch Available (was: In Progress) > Auto-generate

[jira] [Commented] (KAFKA-3834) Consumer should not block in poll on coordinator discovery

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347486#comment-15347486 ] Jason Gustafson commented on KAFKA-3834: [~ewencp] Depends on the nature of the problem, I guess.

Re: KTable.filter usage, memory consumption and materialized view semantics

2016-06-23 Thread Philippe Derome
Thanks a lot for the detailed feedback, its clarity and the reference to KIP-63, which however is for the most part above my head for now. Having said that, I still hold the view that the behaviour I presented is undesirable and hardly defensible and we may have no choice but to agree to disagree

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

2016-06-23 Thread Jason Gustafson
Hey Andy, Thanks for jumping in. A couple comments: In addition, I think it is important that during a rebalance consumers do > not first have all partitions revoked, only to have a very similar, (or the > same!), set reassigned. This is less than initiative and complicates client > code

[jira] [Work started] (KAFKA-3846) Connect record types should include timestamps

2016-06-23 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3846?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3846 started by Shikhar Bhushan. -- > Connect record types should include timestamps >

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

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347679#comment-15347679 ] Jason Gustafson commented on KAFKA-3480: [~wushujames] This is looking good to me. Left a comment

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

2016-06-23 Thread Apache Jenkins Server
See

Re: [DISCUSS] KAFKA-3761: Controller has RunningAsBroker instead of RunningAsController state

2016-06-23 Thread Gwen Shapira
+1 from me too On Thu, Jun 23, 2016 at 3:06 PM, Ismael Juma wrote: > +1 from me. > > Ismael > > On Thu, Jun 23, 2016 at 11:57 PM, Roger Hoover > wrote: > >> Hi all, >> >> Does anyone have an issue with removing the broker state called >>

[jira] [Assigned] (KAFKA-3522) Consider adding version information into rocksDB storage format

2016-06-23 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3522?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ishita Mandhan reassigned KAFKA-3522: - Assignee: Ishita Mandhan > Consider adding version information into rocksDB storage

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

2016-06-23 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15347448#comment-15347448 ] Ishita Mandhan commented on KAFKA-2857: --- Thanks [~hachikuji]! I just created a PR for this jira. >

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

2016-06-23 Thread Vahid S Hashemian
Thank you Andy for your feedback on the KIP. I agree with Jason on the responses he provided below. If we give precedence to fairness over stickiness there is no assumption that can be made about which assignment would remain and which would be revoked. If we give precedence to stickiness over

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

2016-06-23 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: --- Reviewer: Jason Gustafson Status: Patch Available (was: In Progress) > Autogenerate metrics

[jira] [Work started] (KAFKA-3480) Autogenerate metrics documentation

2016-06-23 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3480 started by James Cheng. -- > Autogenerate metrics documentation > -- > >

[jira] [Commented] (KAFKA-3879) KafkaConsumer with auto commit enabled gets stuck when killed after broker is dead

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346819#comment-15346819 ] Jason Gustafson commented on KAFKA-3879: [~singhashish] I was going to submit a KIP for this

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

2016-06-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15346902#comment-15346902 ] Jason Gustafson commented on KAFKA-3727: [~ecomar] Super late in chiming in (finally caught up to

Re: [DISCUSS] KAFKA-3761: Controller has RunningAsBroker instead of RunningAsController state

2016-06-23 Thread Grant Henke
+1 The Metadata response now has the controller broker in it too leveraging KafkaController.isActive. Clients can use that to identify the controller. On Thu, Jun 23, 2016 at 7:17 PM, Gwen Shapira wrote: > +1 from me too > > On Thu, Jun 23, 2016 at 3:06 PM, Ismael Juma

[jira] [Work started] (KAFKA-3794) Add Stream / Table prefix in print functions

2016-06-23 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3794?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3794 started by Bill Bejeck. -- > Add Stream / Table prefix in print functions >

Re: [VOTE] KIP-4 Create Topics Schema

2016-06-23 Thread Grant Henke
Thanks to all who voted. The KIP-4 Create Topics changes passed with +4 (binding), and +4 (non-binding). There is a patch available for review here: https://github.com/apache/kafka/pull/1489 On Tue, Jun 21, 2016 at 1:11 AM, Manikumar Reddy wrote: > +1 (non-binding) >

[VOTE] KIP-4 Delete Topics Schema

2016-06-23 Thread Grant Henke
I would like to initiate the voting process for the "KIP-4 Delete Topics Schema changes". This is not a vote for all of KIP-4, but specifically for the delete topics changes. I have included the exact changes below for clarity: > > Delete Topics Request (KAFKA-2946 >