[jira] [Updated] (KAFKA-5310) reset ControllerContext during resignation

2017-05-22 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5310?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-5310: Status: Patch Available (was: Open) > reset ControllerContext during resignation >

[GitHub] kafka pull request #3122: KAFKA-5310: reset ControllerContext during resigna...

2017-05-22 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/3122 KAFKA-5310: reset ControllerContext during resignation This ticket is all about ControllerContext initialization and teardown. The key points are: 1. we should teardown ControllerContext dur

[jira] [Commented] (KAFKA-5310) reset ControllerContext during resignation

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

[jira] [Updated] (KAFKA-5310) reset ControllerContext during resignation

2017-05-22 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5310?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-5310: Description: This ticket is all about ControllerContext initialization and teardown. The key points

[jira] [Created] (KAFKA-5310) reset ControllerContext during resignation

2017-05-22 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-5310: --- Summary: reset ControllerContext during resignation Key: KAFKA-5310 URL: https://issues.apache.org/jira/browse/KAFKA-5310 Project: Kafka Issue Type: Sub-task

Re: Kafka broker startup issue

2017-05-22 Thread dhiraj prajapati
Thanks for pointing this out. There was a broker instance of version 0.10.1.0 running. On May 23, 2017 11:34 AM, "Ewen Cheslack-Postava" wrote: > Version 2 of UpdateMetadataRequest does not exist in version 0.9.0.1. This > suggests that you have a broker with a newer version of Kafka running > a

Re: Kafka broker startup issue

2017-05-22 Thread Ewen Cheslack-Postava
Version 2 of UpdateMetadataRequest does not exist in version 0.9.0.1. This suggests that you have a broker with a newer version of Kafka running against the same ZK broker. Do you have any other versions running? Or is it possible this is a shared ZK cluster and you're not using a namespace within

[GitHub] kafka pull request #3118: MINOR: Broker should disallow downconversion of tr...

2017-05-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3118 --- 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

Jenkins build is back to normal : kafka-0.10.2-jdk7 #167

2017-05-22 Thread Apache Jenkins Server
See

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

2017-05-22 Thread Apache Jenkins Server
See Changes: [me] HOTFIX: Replace JDK download and fix missing argument in Vagrant -- [...truncated 2.46 MB...] org.apache.kafka.common.security.scram.ScramMessagesTest >

[jira] [Commented] (KAFKA-5296) Unable to write to some partitions of newly created topic in 10.2

2017-05-22 Thread Abhisek Saikia (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020558#comment-16020558 ] Abhisek Saikia commented on KAFKA-5296: --- [~huxi_2b] Currently we have a really large

[jira] [Created] (KAFKA-5309) Stores not queryable after one thread died

2017-05-22 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5309: -- Summary: Stores not queryable after one thread died Key: KAFKA-5309 URL: https://issues.apache.org/jira/browse/KAFKA-5309 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #3121: HOTFIX: Replace JDK download and fix missing argum...

2017-05-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3121 --- 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 #3121: HOTFIX: Replace JDK download and fix missing argum...

2017-05-22 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/3121 HOTFIX: Replace JDK download and fix missing argument in Vagrant provisioning script You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafk

[jira] [Commented] (KAFKA-5265) Move ACLs, Config, NodeVersions classes into org.apache.kafka.common

2017-05-22 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5265?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020479#comment-16020479 ] Colin P. McCabe commented on KAFKA-5265: https://github.com/apache/kafka/pull/3120

[jira] [Commented] (KAFKA-5218) New Short serializer, deserializer, serde

2017-05-22 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5218?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020477#comment-16020477 ] Guozhang Wang commented on KAFKA-5218: -- Done. > New Short serializer, deserializer,

[GitHub] kafka pull request #3120: Kafka 5265

2017-05-22 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/3120 Kafka 5265 You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka KAFKA-5265 Alternatively you can review and apply these changes as the

[jira] [Assigned] (KAFKA-5218) New Short serializer, deserializer, serde

2017-05-22 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5218?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-5218: Assignee: Mario Molina > New Short serializer, deserializer, serde > --

[jira] [Commented] (KAFKA-4896) Offset loading can use more threads

2017-05-22 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020447#comment-16020447 ] Onur Karaman commented on KAFKA-4896: - handleGroupImmigration only gets called from ha

[jira] [Assigned] (KAFKA-5282) Transactions integration test: Use factory methods to keep track of open producers and consumers and close them all on tearDown

2017-05-22 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-5282: -- Assignee: Vahid Hashemian > Transactions integration test: Use factory methods to keep

Jenkins build is back to normal : kafka-0.11.0-jdk7 #14

2017-05-22 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-5273) KafkaConsumer.committed() should get latest committed offsets from the server

2017-05-22 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5273: Status: Patch Available (was: In Progress) > KafkaConsumer.committed() should get latest committed

[jira] [Commented] (KAFKA-5273) KafkaConsumer.committed() should get latest committed offsets from the server

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

[GitHub] kafka pull request #3119: KAFKA-5273: Make KafkaConsumer.committed query the...

2017-05-22 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/3119 KAFKA-5273: Make KafkaConsumer.committed query the server for all partitions Before this patch the consumer would return the cached offsets for partitions in its current assignment. This worked when

[jira] [Updated] (KAFKA-5308) TC should handle UNSUPPORTED_FOR_MESSAGE_FORMAT in WriteTxnMarker response

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5308?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5308: --- Labels: exactly-once (was: ) > TC should handle UNSUPPORTED_FOR_MESSAGE_FORMAT in WriteTxnMar

[jira] [Updated] (KAFKA-5307) Concurrent offset commit on same partition may result in inconsistent cache

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5307: --- Fix Version/s: (was: 0.11.0.0) > Concurrent offset commit on same partition may result in

[jira] [Updated] (KAFKA-5307) Concurrent offset commit on same partition may result in inconsistent cache

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5307: --- Fix Version/s: 0.11.0.0 > Concurrent offset commit on same partition may result in inconsisten

[jira] [Commented] (KAFKA-5308) TC should handle UNSUPPORTED_FOR_MESSAGE_FORMAT in WriteTxnMarker response

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5308?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020384#comment-16020384 ] Jason Gustafson commented on KAFKA-5308: cc [~guozhang] [~damianguy] > TC should

[jira] [Created] (KAFKA-5308) TC should handle UNSUPPORTED_FOR_MESSAGE_FORMAT in WriteTxnMarker response

2017-05-22 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5308: -- Summary: TC should handle UNSUPPORTED_FOR_MESSAGE_FORMAT in WriteTxnMarker response Key: KAFKA-5308 URL: https://issues.apache.org/jira/browse/KAFKA-5308 Project:

[GitHub] kafka pull request #3118: MINOR: Broker should disallow downconversion of tr...

2017-05-22 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3118 MINOR: Broker should disallow downconversion of transactional/idempotent records You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/ka

[jira] [Updated] (KAFKA-5186) Avoid expensive initialization of producer state when upgrading

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5186: --- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull requ

[jira] [Commented] (KAFKA-5186) Avoid expensive initialization of producer state when upgrading

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

[GitHub] kafka pull request #3113: KAFKA-5186: Avoid expensive log scan to build prod...

2017-05-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3113 --- 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] [Comment Edited] (KAFKA-3821) Allow Kafka Connect source tasks to produce offset without writing to topics

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020279#comment-16020279 ] Randall Hauch edited comment on KAFKA-3821 at 5/22/17 10:12 PM:

[jira] [Commented] (KAFKA-3821) Allow Kafka Connect source tasks to produce offset without writing to topics

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020279#comment-16020279 ] Randall Hauch commented on KAFKA-3821: -- [~ewencp], the more I think about this issue,

Re: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-05-22 Thread BigData dev
Hi Matthias, For the AdminClient, client configuration is needed. And for zookeeper, no properties are required. So, in other tools like consumerGroupCommand, they used the command config option. I think consumer-config and consumer-property are not required here. We will use the configurations pas

[jira] [Commented] (KAFKA-5307) Concurrent offset commit on same partition may result in inconsistent cache

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5307?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020225#comment-16020225 ] Jason Gustafson commented on KAFKA-5307: cc [~guozhang] [~apurva] > Concurrent of

[jira] [Created] (KAFKA-5307) Concurrent offset commit on same partition may result in inconsistent cache

2017-05-22 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5307: -- Summary: Concurrent offset commit on same partition may result in inconsistent cache Key: KAFKA-5307 URL: https://issues.apache.org/jira/browse/KAFKA-5307 Project

[jira] [Commented] (KAFKA-5225) StreamsResetter doesn't allow custom Consumer properties

2017-05-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5225?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020221#comment-16020221 ] ASF GitHub Bot commented on KAFKA-5225: --- GitHub user bharatviswa504 opened a pull re

[GitHub] kafka pull request #3117: KAFKA-5225:StreamsResetter doesn't allow custom Co...

2017-05-22 Thread bharatviswa504
GitHub user bharatviswa504 opened a pull request: https://github.com/apache/kafka/pull/3117 KAFKA-5225:StreamsResetter doesn't allow custom Consumer properties Added command-config option, as the client configuration is required for AdminClient and Embedded Consumer. @mjsax

[jira] [Updated] (KAFKA-5225) StreamsResetter doesn't allow custom Consumer properties

2017-05-22 Thread Bharat Viswanadham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5225?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bharat Viswanadham updated KAFKA-5225: -- Status: In Progress (was: Patch Available) > StreamsResetter doesn't allow custom Consu

[jira] [Commented] (KAFKA-5225) StreamsResetter doesn't allow custom Consumer properties

2017-05-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5225?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020216#comment-16020216 ] ASF GitHub Bot commented on KAFKA-5225: --- Github user bharatviswa504 closed the pull

[GitHub] kafka pull request #3062: KAFKA-5225: StreamsResetter tool to allow custom c...

2017-05-22 Thread bharatviswa504
Github user bharatviswa504 closed the pull request at: https://github.com/apache/kafka/pull/3062 --- 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

[jira] [Updated] (KAFKA-5236) Regression in on-disk log size when using Snappy compression with 0.8.2 log message format

2017-05-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5236: --- Labels: regression (was: ) > Regression in on-disk log size when using Snappy compression with 0.8.2

[jira] [Updated] (KAFKA-5236) Regression in on-disk log size when using Snappy compression with 0.8.2 log message format

2017-05-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5236: --- Fix Version/s: 0.11.0.0 > Regression in on-disk log size when using Snappy compression with 0.8.2 log

[jira] [Comment Edited] (KAFKA-5275) Review and potentially tweak AdminClient API for the initial release (KIP-117)

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16017701#comment-16017701 ] Randall Hauch edited comment on KAFKA-5275 at 5/22/17 8:55 PM: -

Re: Kafka Connect: To much restarting with a SourceConnector with dynamic set of tasks

2017-05-22 Thread Randall Hauch
You're not doing anything wrong, but I suspect you're requesting task reconfiguration more frequently than was originally envisioned, which means that the current implementation is not as optimal for your case. I'm not sure how much effort is required to implement this new behavior. The logic for

[jira] [Commented] (KAFKA-5275) Review and potentially tweak AdminClient API for the initial release (KIP-117)

2017-05-22 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020137#comment-16020137 ] Colin P. McCabe commented on KAFKA-5275: I can imagine clients handling some types

[jira] [Comment Edited] (KAFKA-3821) Allow Kafka Connect source tasks to produce offset without writing to topics

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020100#comment-16020100 ] Randall Hauch edited comment on KAFKA-3821 at 5/22/17 8:13 PM: -

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

2017-05-22 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-3821) Allow Kafka Connect source tasks to produce offset without writing to topics

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020100#comment-16020100 ] Randall Hauch commented on KAFKA-3821: -- The problem with the connector directly using

[jira] [Comment Edited] (KAFKA-5275) Review and potentially tweak AdminClient API for the initial release (KIP-117)

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020058#comment-16020058 ] Randall Hauch edited comment on KAFKA-5275 at 5/22/17 7:48 PM: -

[jira] [Comment Edited] (KAFKA-5275) Review and potentially tweak AdminClient API for the initial release (KIP-117)

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020058#comment-16020058 ] Randall Hauch edited comment on KAFKA-5275 at 5/22/17 7:48 PM: -

[DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2017-05-22 Thread Randall Hauch
Hi, all. We recently added the ability for Kafka Connect to create *internal* topics using the new AdminClient, but it still would be great if Kafka Connect could do this for new topics that result from source connector records. I've outlined an approach to do this in "KIP-158 Kafka Connect should

[jira] [Commented] (KAFKA-5275) Review and potentially tweak AdminClient API for the initial release (KIP-117)

2017-05-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020058#comment-16020058 ] Randall Hauch commented on KAFKA-5275: -- [~cmccabe] wrote: {quote} Hmm. Methods like t

[jira] [Work started] (KAFKA-5273) KafkaConsumer.committed() should get latest committed offsets from the server

2017-05-22 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5273 started by Apurva Mehta. --- > KafkaConsumer.committed() should get latest committed offsets from the server > -

[jira] [Updated] (KAFKA-5128) TransactionCoordinator - Check inter broker protocol and message format and raise errors if incompatible

2017-05-22 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5128?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5128: Status: Patch Available (was: Open) > TransactionCoordinator - Check inter broker protocol and mess

[jira] [Updated] (KAFKA-5128) TransactionCoordinator - Check inter broker protocol and message format and raise errors if incompatible

2017-05-22 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5128?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5128: Labels: exactly-once (was: ) > TransactionCoordinator - Check inter broker protocol and message for

[jira] [Updated] (KAFKA-5280) Protect concurrent access to the cached transaction status

2017-05-22 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5280: Status: Patch Available (was: Open) > Protect concurrent access to the cached transaction status >

[jira] [Commented] (KAFKA-5275) Review and potentially tweak AdminClient API for the initial release (KIP-117)

2017-05-22 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020039#comment-16020039 ] Colin P. McCabe commented on KAFKA-5275: bq. [~xvl] wrote: To echo Randall's point

Build failed in Jenkins: kafka-0.11.0-jdk7 #13

2017-05-22 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5277; Sticky Assignor should not cache previous assignment (KIP-54 -- [...truncated 882.08 KB...] kafka.admin.TopicCommandTest > testCreateIfNotE

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-22 Thread Andrew Coates
Thanks Mike, I was just about to do the same! I think the benefit of the builder pattern, or the `with` pattern, is greatly improved if you take interceptor implementations into account. On Fri, 19 May 2017 at 08:29 Michael Pearce wrote: > And I just failed at a basic copy/paste exercise (bangs

Re: [VOTE] KIP-117: Add a public AdminClient API for Kafka admin operations

2017-05-22 Thread Colin McCabe
Oops, I just realized if we do a call with topics=*, we don't need to make a follow-up call. :) The question still holds, though: is it worth sacrificing some scalability when talking to older brokers, to get saner semantics? cheers, Colin On Mon, May 22, 2017, at 11:41, Colin McCabe wrote: >

Re: [VOTE] KIP-117: Add a public AdminClient API for Kafka admin operations

2017-05-22 Thread Colin McCabe
I definitely agree that auto topic creation is unexpected and confusing (even with the JavaDoc note in the API). The proposed solution of adding a flag to MetadataRequest seems pretty simple and reasonable. +1. As you noted, though, we don't have a way to do this for the 0.10.x releases. It see

[jira] [Resolved] (KAFKA-5277) Sticky Assignor should not cache the calculated assignment (KIP-54 follow-up)

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5277?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-5277. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 3092 [https:

[jira] [Commented] (KAFKA-5277) Sticky Assignor should not cache the calculated assignment (KIP-54 follow-up)

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

[GitHub] kafka pull request #3092: KAFKA-5277: Sticky Assignor should not cache previ...

2017-05-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3092 --- 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] [Work started] (KAFKA-4935) Consider disabling record level CRC checks for message format V2

2017-05-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4935?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4935 started by Jason Gustafson. -- > Consider disabling record level CRC checks for message format V2 >

[jira] [Updated] (KAFKA-5191) Autogenerate Consumer Fetcher metrics

2017-05-22 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-5191: --- Reviewer: Jason Gustafson (was: Ismael Juma) Changing reviewer to [~hachikuji], since he was the orig

[jira] [Updated] (KAFKA-5297) Broker can take a long time to shut down if there are many active log segments

2017-05-22 Thread Kyle Ambroff (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5297?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kyle Ambroff updated KAFKA-5297: Attachment: flame graph of broker during shut down.png > Broker can take a long time to shut down if

Re: Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread Simon Souter
The following tickets are probably relevant to this KIP: https://issues.apache.org/jira/browse/KAFKA-3457 https://issues.apache.org/jira/browse/KAFKA-1894 https://issues.apache.org/jira/browse/KAFKA-3834 On 22 May 2017 at 16:30, Rajini Sivaram wrote: > Ismael, > > Yes, agree. My concern was tha

[jira] [Commented] (KAFKA-5297) Broker can take a long time to shut down if there are many active log segments

2017-05-22 Thread Kyle Ambroff (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019848#comment-16019848 ] Kyle Ambroff commented on KAFKA-5297: - I'm still unsure about this approach, but I wan

[jira] [Updated] (KAFKA-5218) New Short serializer, deserializer, serde

2017-05-22 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5218?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5218: --- Component/s: streams > New Short serializer, deserializer, serde > ---

[jira] [Commented] (KAFKA-5218) New Short serializer, deserializer, serde

2017-05-22 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5218?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019846#comment-16019846 ] Matthias J. Sax commented on KAFKA-5218: [~guozhang] Can you add [~mmolimar] to th

[jira] [Updated] (KAFKA-5218) New Short serializer, deserializer, serde

2017-05-22 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5218?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5218: --- Status: Patch Available (was: Open) > New Short serializer, deserializer, serde > ---

[jira] [Commented] (KAFKA-5081) two versions of jackson-annotations-xxx.jar in distribution tgz

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

[GitHub] kafka pull request #3116: KAFKA-5081: force version for 'jackson-annotations...

2017-05-22 Thread dejan2609
GitHub user dejan2609 opened a pull request: https://github.com/apache/kafka/pull/3116 KAFKA-5081: force version for 'jackson-annotations' in order to prevent redundant jars from being bundled into kafka distribution (and to align with other jackson libraries) **JIRA ticket:** [KAF

[jira] [Commented] (KAFKA-5081) two versions of jackson-annotations-xxx.jar in distribution tgz

2017-05-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019814#comment-16019814 ] ASF GitHub Bot commented on KAFKA-5081: --- Github user dejan2609 closed the pull reque

[GitHub] kafka pull request #2900: KAFKA-5081: force version for 'jackson-annotations...

2017-05-22 Thread dejan2609
Github user dejan2609 closed the pull request at: https://github.com/apache/kafka/pull/2900 --- 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 e

Build failed in Jenkins: kafka-0.11.0-jdk7 #12

2017-05-22 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-5303, KAFKA-5305: Improve logging when fetches fail in -- [...truncated 881.82 KB...] kafka.admin.TopicCommandTest > testCreateIfNotExist

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

2017-05-22 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-5303, KAFKA-5305: Improve logging when fetches fail in -- [...truncated 1.39 MB...] org.apache.kafka.common.security.scram.ScramMessages

Re: Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread Guozhang Wang
Could you refresh me what's the issue of consumer that caused us to set the default values of connect.timeout to 5 min? Guozhang On Mon, May 22, 2017 at 8:35 AM, Ismael Juma wrote: > Yes, I agree that one config would be better. Just a bit more work to > achieve the desired effect for the Cons

[jira] [Commented] (KAFKA-1955) Explore disk-based buffering in new Kafka Producer

2017-05-22 Thread Brandon Bradley (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1955?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019726#comment-16019726 ] Brandon Bradley commented on KAFKA-1955: Ok, I am wrong. There are two instances w

Re: Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread Ismael Juma
Yes, I agree that one config would be better. Just a bit more work to achieve the desired effect for the Consumer. Ismael On Mon, May 22, 2017 at 4:30 PM, Rajini Sivaram wrote: > Ismael, > > Yes, agree. My concern was that a connection can be shutdown uncleanly at > any time. If a client is in

Re: Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread Rajini Sivaram
Ismael, Yes, agree. My concern was that a connection can be shutdown uncleanly at any time. If a client is in the middle of a request, then it times out after min(request.timeout.ms, tcp-timeout). If we add another config option connect.timeout.ms, then we will sometimes wait for min(connect.timeo

[jira] [Commented] (KAFKA-1955) Explore disk-based buffering in new Kafka Producer

2017-05-22 Thread Brandon Bradley (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1955?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019697#comment-16019697 ] Brandon Bradley commented on KAFKA-1955: The current `BufferPool` implementation d

Re: Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread Ismael Juma
Rajini, For this to have the desired effect, we'd probably need to lower the default request.timeout.ms for the consumer and fix the underlying reason why it is a little over 5 minutes at the moment. Ismael On Mon, May 22, 2017 at 4:15 PM, Rajini Sivaram wrote: > Hi David, > > Sorry, what I me

Re: Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread Rajini Sivaram
Hi David, Sorry, what I meant was: Can you reuse the existing configuration option request.timeout,ms , instead of adding a new config and add the behaviour that you have proposed in the KIP for the connection phase using this timeout? I think the timeout for connection is useful. I am not sure we

[jira] [Assigned] (KAFKA-5305) Missing logging information in ReplicaFetcher

2017-05-22 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5305?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-5305: - Assignee: Ismael Juma > Missing logging information in ReplicaFetcher > -

[jira] [Resolved] (KAFKA-5305) Missing logging information in ReplicaFetcher

2017-05-22 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5305?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-5305. --- Resolution: Fixed Fix Version/s: 0.11.1.0 0.11.0.0 Issue resolved by

[jira] [Assigned] (KAFKA-5303) FetchRequest doesn't implement toString

2017-05-22 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-5303: - Assignee: Ismael Juma > FetchRequest doesn't implement toString > ---

[jira] [Commented] (KAFKA-5303) FetchRequest doesn't implement toString

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

[jira] [Resolved] (KAFKA-5303) FetchRequest doesn't implement toString

2017-05-22 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-5303. --- Resolution: Fixed Fix Version/s: 0.11.1.0 0.11.0.0 Issue resolved by

[GitHub] kafka pull request #3115: KAFKA-5303, KAFKA-5305: Improve logging when fetch...

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

Re:Re: [DISCUSS] KIP-148: Add a connect timeout for client

2017-05-22 Thread 东方甲乙
Hi Rajini. When kafka node' machine is shutdown or network is closed, the connecting phase could not use the request.timeout.ms, because the client haven't send a req yet. And no response for the nio, the selector will not close the connect, so it will not choose other good node to get the m

[jira] [Commented] (KAFKA-5298) MirrorMaker deadlocks with missing topics

2017-05-22 Thread Raymond Conn (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5298?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019596#comment-16019596 ] Raymond Conn commented on KAFKA-5298: - Thanks, this definitely fixes part of it since

[jira] [Comment Edited] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2017-05-22 Thread dhiraj prajapati (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16019536#comment-16019536 ] dhiraj prajapati edited comment on KAFKA-4477 at 5/22/17 1:45 PM: --

Build failed in Jenkins: kafka-0.11.0-jdk7 #11

2017-05-22 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-5289: handleStopReplica should not send a second response -- [...truncated 884.14 KB...] kafka.admin.TopicCommandTest > testCreateIfNotEx

[jira] [Created] (KAFKA-5306) Official init.d scripts

2017-05-22 Thread Shahar (JIRA)
Shahar created KAFKA-5306: - Summary: Official init.d scripts Key: KAFKA-5306 URL: https://issues.apache.org/jira/browse/KAFKA-5306 Project: Kafka Issue Type: Improvement Affects Versions: 0.10.2.

[jira] [Commented] (KAFKA-5303) FetchRequest doesn't implement toString

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

  1   2   >