[jira] [Commented] (KAFKA-5269) TransactionBounceTest occasionally fails due to partition errors

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

[jira] [Updated] (KAFKA-5269) TransactionBounceTest occasionally fails due to partition errors

2017-05-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5269: --- Resolution: Fixed Fix Version/s: 0.11.0.0 Status: Resolved (was: Patch

[GitHub] kafka pull request #3094: KAFKA-5269: Correct handling of UNKNOWN_TOPIC_OR_P...

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

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

2017-05-19 Thread Apache Jenkins Server
See

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

2017-05-19 Thread Abhisek Saikia (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5296?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Abhisek Saikia updated KAFKA-5296: -- Description: We are using kafka 10.2 and the cluster was running fine for a month with 50

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

2017-05-19 Thread Abhisek Saikia (JIRA)
Abhisek Saikia created KAFKA-5296: - Summary: Unable to write to some partitions of newly created topic in 10.2 Key: KAFKA-5296 URL: https://issues.apache.org/jira/browse/KAFKA-5296 Project: Kafka

[jira] [Updated] (KAFKA-5247) Consumer GroupCoordinator should continue to materialize committed offsets in offset order even for transactional offset commits

2017-05-19 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5247: Status: Patch Available (was: Open) > Consumer GroupCoordinator should continue to materialize

[jira] [Commented] (KAFKA-5247) Consumer GroupCoordinator should continue to materialize committed offsets in offset order even for transactional offset commits

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

[GitHub] kafka pull request #3108: KAFKA-5247: Materialize committed offsets in offse...

2017-05-19 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/3108 KAFKA-5247: Materialize committed offsets in offset order With this patch, offset commits are always materialized according to the order of the commit records in the offsets topic. Before

[GitHub] kafka pull request #3107: MINOR: improve descriptions of Streams reset tool ...

2017-05-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3107 --- 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: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-05-19 Thread Matthias J. Sax
Couple of follow ups. The reset tool uses AdminClient, ZkUtils, and a KafkaConsumer internally. Thus, I am wondering if we need the possibility to specify configs for all of them? The original JIRA reported, that the reset tool does not work for a secured cluster, and thus, I doubt that consumer

[jira] [Commented] (KAFKA-5063) Flaky ResetIntegrationTest.testReprocessingFromScratchAfterResetWithIntermediateUserTopic

2017-05-19 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5063?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16018135#comment-16018135 ] Matthias J. Sax commented on KAFKA-5063: [~damianguy] The error you report is a different stack

[jira] [Updated] (KAFKA-5269) TransactionBounceTest occasionally fails due to partition errors

2017-05-19 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5269: Status: Patch Available (was: Open) > TransactionBounceTest occasionally fails due to partition

[jira] [Commented] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16018106#comment-16018106 ] Jason Gustafson commented on KAFKA-5266: [~jeqo] Maybe it could be "--to-current," which sort of

[jira] [Comment Edited] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16018106#comment-16018106 ] Jason Gustafson edited comment on KAFKA-5266 at 5/19/17 10:14 PM: --

[jira] [Updated] (KAFKA-5259) TransactionalId authorization should imply ProducerId authorization

2017-05-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5259: --- Status: Patch Available (was: In Progress) > TransactionalId authorization should imply

[jira] [Work started] (KAFKA-5259) TransactionalId authorization should imply ProducerId authorization

2017-05-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5259 started by Jason Gustafson. -- > TransactionalId authorization should imply ProducerId authorization >

[GitHub] kafka pull request #3107: MINOR: improve descriptions of Streams reset tool ...

2017-05-19 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3107 MINOR: improve descriptions of Streams reset tool options You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka minor-reset-tool-options

[jira] [Commented] (KAFKA-4785) Records from internal repartitioning topics should always use RecordMetadataTimestampExtractor

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

[GitHub] kafka pull request #3106: KAFKA-4785: Records from internal repartitioning t...

2017-05-19 Thread jeyhunkarimov
GitHub user jeyhunkarimov opened a pull request: https://github.com/apache/kafka/pull/3106 KAFKA-4785: Records from internal repartitioning topics should always use RecordMetadataTimestampExtractor You can merge this pull request into a Git repository by running: $ git pull

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

2017-05-19 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16018069#comment-16018069 ] James Cheng commented on KAFKA-5191: Pinging [~ijuma] [~hachikuji] [~benstopford] for a code review.

Re: KTable aggregate corruption on poorly timed exception

2017-05-19 Thread Matthias J. Sax
Hi Mathieu, Thanks for reporting this! This is definitely a bug and it must get fixed for at-last-once processing, too. Exactly-once is not required to avoid the bug. Can you open a Jira? I think, I know already how to fix it. -Matthias On 5/19/17 8:53 AM, Mathieu Fenniak wrote: > Whoops, I

[jira] [Work started] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jorge Quilcate (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5266 started by Jorge Quilcate. - > Follow-up improvements for consumer offset reset tool (KIP-122) >

[jira] [Comment Edited] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jorge Quilcate (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16018004#comment-16018004 ] Jorge Quilcate edited comment on KAFKA-5266 at 5/19/17 9:04 PM:

[jira] [Commented] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jorge Quilcate (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16018004#comment-16018004 ] Jorge Quilcate commented on KAFKA-5266: --- [~hachikuji] Thanks! For case 3, maybe `--now` could be

Re: [DISCUSS]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-05-19 Thread Matthias J. Sax
With the current KIP, using ValueMapper and ValueMapperWithKey interfaces, RichFunction seems to be an independent add-on. To fix the original issue to allow key access, RichFunctions are not required IMHO. I initially put the RichFunction idea on the table, because I was hoping to get a uniform

[jira] [Commented] (KAFKA-5294) PlainSaslServerFactory should allow a null Map in getMechanismNames

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

[GitHub] kafka pull request #3105: KAFKA-5294: PlainSaslServerFactory should allow a ...

2017-05-19 Thread mimaison
GitHub user mimaison opened a pull request: https://github.com/apache/kafka/pull/3105 KAFKA-5294: PlainSaslServerFactory should allow a null Map in getMech… …anismNames If props is null, use POLICY_NOPLAINTEXT default value: false As far as I can tell, none of

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

2017-05-19 Thread Apache Jenkins Server
See

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

2017-05-19 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4714; TimestampConverter transformation (KIP-66) -- [...truncated 883.70 KB...] kafka.security.auth.SimpleAclAuthorizerTest > testTopicAcl

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

2017-05-19 Thread kyle k (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017884#comment-16017884 ] kyle k commented on KAFKA-3821: --- As a potential solution why can't the {{OffsetStorageWriter}}'s {{void

[jira] [Commented] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017870#comment-16017870 ] Jason Gustafson commented on KAFKA-5266: [~jeqo] Thanks for explaining the case for 3. I guess

[GitHub] kafka pull request #3104: MINOR: improve EmbeddedKafkaCluster test utility f...

2017-05-19 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3104 MINOR: improve EmbeddedKafkaCluster test utility for deleting topics You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka

[jira] [Commented] (KAFKA-4714) Implement remaining KIP-66 SMTs

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

[GitHub] kafka pull request #3065: KAFKA-4714: TimestampConverter transformation (KIP...

2017-05-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3065 --- 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: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-05-19 Thread BigData dev
Thanks for the info, Matthias. Regards, Bharat On Fri, May 19, 2017 at 10:25 AM, Matthias J. Sax wrote: > KIP-157 cannot be included in 0.11.0.0 anymore. KIP freeze date deadline > is strict. > > -Matthias > > On 5/19/17 10:15 AM, BigData dev wrote: > > Hi Matthias, > >

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

2017-05-19 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017701#comment-16017701 ] Randall Hauch commented on KAFKA-5275: -- [~colinmccabe] wrote: {quote} We could add an option to

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

2017-05-19 Thread Matthias J. Sax
KIP-157 cannot be included in 0.11.0.0 anymore. KIP freeze date deadline is strict. -Matthias On 5/19/17 10:15 AM, BigData dev wrote: > Hi Matthias, > I will start a new KIP for Kafka tools options to be a standard across all > tools shortly. But I think the KIP 157 for Kafka Streams, should be

[jira] [Commented] (KAFKA-5117) Kafka Connect REST endpoints reveal Password typed values

2017-05-19 Thread Thomas Holmes (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017689#comment-16017689 ] Thomas Holmes commented on KAFKA-5117: -- I had some code that seems to work but I'm not very happy

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

2017-05-19 Thread BigData dev
Hi Matthias, I will start a new KIP for Kafka tools options to be a standard across all tools shortly. But I think the KIP 157 for Kafka Streams, should be needed for 0.11.0.0 release, (KIP freeze date is already over, but I think this is minor code change in tools to add option to streams reset

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

2017-05-19 Thread Matthias J. Sax
I double checked with Matthew Warhaftig (the original author of KIP-14) and he has not interest to continue the KIP atm. Thus, Bharat can continue the work on KIP-14. I think it would be best, to start a new DISCUSS thread after you update KIP-14. Thanks for your contributions! -Matthias On

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

2017-05-19 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017659#comment-16017659 ] Xavier Léauté commented on KAFKA-5275: -- To echo Randall's point about having a topic config builder,

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

2017-05-19 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017646#comment-16017646 ] Xavier Léauté commented on KAFKA-5275: -- Speaking of {{TopicExistsException}} it would also be nice to

[jira] [Commented] (KAFKA-5291) AdminClient should not trigger auto creation of topics

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5291?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017637#comment-16017637 ] Ismael Juma commented on KAFKA-5291: cc [~cmccabe] > AdminClient should not trigger auto creation of

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

2017-05-19 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017629#comment-16017629 ] Colin P. McCabe edited comment on KAFKA-5275 at 5/19/17 4:37 PM: - bq. It'd

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

2017-05-19 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017629#comment-16017629 ] Colin P. McCabe commented on KAFKA-5275: bq. It'd be great to simplify the creation of the

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

2017-05-19 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017626#comment-16017626 ] Colin P. McCabe commented on KAFKA-5275: bq. [~rhauch]: Also, a common use case for creating

KTable aggregate corruption on poorly timed exception

2017-05-19 Thread Mathieu Fenniak
Hi Kafka devs, This morning I observed a specific Kafka Streams aggregation that ended up with an incorrect computed output after a Kafka Streams thread crashed with an unhandled exception. The topology is pretty simple -- a single KTable source, group by a field in the value, aggregate that

Re: KTable aggregate corruption on poorly timed exception

2017-05-19 Thread Mathieu Fenniak
Whoops, I said I'd put the specific exception at the bottom of the e-mail. It probably isn't the important part of this thread, but might suggest when this situation can occur. Also of note, this is occurring on Kafka Streams 0.10.2.1. 20:56:07.061 [StreamThread-3] ERROR

[jira] [Commented] (KAFKA-4785) Records from internal repartitioning topics should always use RecordMetadataTimestampExtractor

2017-05-19 Thread Jeyhun Karimov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4785?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017575#comment-16017575 ] Jeyhun Karimov commented on KAFKA-4785: --- [~mjsax] Thanks for reminding. I am on it > Records from

Re: [Vote] KIP-150 - Kafka-Streams Cogroup

2017-05-19 Thread Xavier Léauté
Hi Kyle, I left a few more comments in the discussion thread, if you wouldn't mind taking a look On Fri, May 19, 2017 at 5:31 AM Kyle Winkelman wrote: > Hello all, > > I would like to start the vote on KIP-150. > >

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2017-05-19 Thread Xavier Léauté
Sorry to jump on this thread so late. I agree this is a very useful addition and wanted to provide an additional use-case and some more comments. This is actually a very common analytics use-case in the ad-tech industry. The typical setup will have an auction stream, an impression stream, and a

[GitHub] kafka pull request #3103: KAFKA-5128: check inter broker and message format ...

2017-05-19 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3103 KAFKA-5128: check inter broker and message format versions in transactional methods [WIP] Add check in `KafkaApis` that the inter broker protocol and message format versions are at least

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

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

[jira] [Commented] (KAFKA-5278) kafka-console-consumer: `--value-deserializer` is not working but `--property value.deserializer` does

2017-05-19 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017556#comment-16017556 ] Vahid Hashemian commented on KAFKA-5278: Looks like the issue reported here is a subset of

[jira] [Comment Edited] (KAFKA-5278) kafka-console-consumer: `--value-deserializer` is not working but `--property value.deserializer` does

2017-05-19 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017556#comment-16017556 ] Vahid Hashemian edited comment on KAFKA-5278 at 5/19/17 3:38 PM: - Looks

[jira] [Updated] (KAFKA-5294) PlainSaslServerFactory should allow a null Map in getMechanismNames

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5294?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5294: --- Fix Version/s: 0.11.0.0 > PlainSaslServerFactory should allow a null Map in getMechanismNames >

[jira] [Commented] (KAFKA-5294) PlainSaslServerFactory should allow a null Map in getMechanismNames

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5294?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017505#comment-16017505 ] Ismael Juma commented on KAFKA-5294: Thanks for the report. This is a simple fix, so we should do it.

[jira] [Updated] (KAFKA-5294) PlainSaslServerFactory should allow a null Map in getMechanismNames

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5294?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5294: --- Priority: Major (was: Minor) > PlainSaslServerFactory should allow a null Map in getMechanismNames >

[jira] [Created] (KAFKA-5294) PlainSaslServerFactory should allow a null Map in getMechanismNames

2017-05-19 Thread Bryan Bende (JIRA)
Bryan Bende created KAFKA-5294: -- Summary: PlainSaslServerFactory should allow a null Map in getMechanismNames Key: KAFKA-5294 URL: https://issues.apache.org/jira/browse/KAFKA-5294 Project: Kafka

[jira] [Created] (KAFKA-5295) Allow Kafka Connect source connectors to specify topic-specific settings for new topics

2017-05-19 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5295: Summary: Allow Kafka Connect source connectors to specify topic-specific settings for new topics Key: KAFKA-5295 URL: https://issues.apache.org/jira/browse/KAFKA-5295

[jira] [Commented] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

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

[GitHub] kafka pull request #3102: Implement Improvements

2017-05-19 Thread jeqo
GitHub user jeqo opened a pull request: https://github.com/apache/kafka/pull/3102 Implement Improvements Implement improvements defined here: https://issues.apache.org/jira/browse/KAFKA-5266 You can merge this pull request into a Git repository by running: $ git pull

Re: Bug Submit - Critical

2017-05-19 Thread J Pai
Fixing the subject line in my previous reply. Not sure what happened there. -Jaikiran On 19-May-2017, at 8:06 PM, J Pai wrote: This sounds like https://issues.apache.org/jira/browse/KAFKA-5232. It’s been fixed a few days back in upstream and will be available in next

[jira] [Commented] (KAFKA-5266) Follow-up improvements for consumer offset reset tool (KIP-122)

2017-05-19 Thread Jorge Quilcate (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017475#comment-16017475 ] Jorge Quilcate commented on KAFKA-5266: --- [~hachikuji] some comments about the improvements: 1.

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

2017-05-19 Thread J Pai
This sounds like https://issues.apache.org/jira/browse/KAFKA-5232. It’s been fixed a few days back in upstream and will be available in next releases. Having said that, you note something about split() method which wasn’t applicable here. Which version of Kafka are you on? Can you paste the

Re: Bug Submit - Critical

2017-05-19 Thread Ismael Juma
Hi Fabien, This has already been fixed, see: https://issues.apache.org/jira/browse/KAFKA-5232 https://github.com/apache/kafka/commit/f56bbb6510df8c12db3ad075e2f6c78dd0092d60 https://github.com/apache/kafka/commit/1ca2b1aacc2994e441c1a949310129d3e78532b5 To file issues in the future, please go

Bug Submit - Critical

2017-05-19 Thread Fabien Kerbouci
Hello, I have no idea how to submit a bug : amongst all the help given to participate to the project, submitting a bug is the hardest and least documented, neither is easy to do. So I hope this email will reach the proper hands on your side :-) The bug is easy to reproduce: 1 - Create a topic

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

2017-05-19 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-5279) TransactionCoordinator must expire transactionalIds

2017-05-19 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5279?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-5279: -- Status: Patch Available (was: Open) > TransactionCoordinator must expire transactionalIds >

[jira] [Commented] (KAFKA-5279) TransactionCoordinator must expire transactionalIds

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

[GitHub] kafka pull request #3101: KAFKA-5279: TransactionCoordinator must expire tra...

2017-05-19 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3101 KAFKA-5279: TransactionCoordinator must expire transactionalIds remove transactions that have not been updated for at least `transactional.id.expiration.ms` You can merge this pull request into a Git

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

2017-05-19 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-3878; Support exponential backoff policy via reconnect.backoff.max -- [...truncated 886.37 KB...] kafka.controller.ControllerIntegrationTest

[jira] [Assigned] (KAFKA-5245) KStream builder should capture serdes

2017-05-19 Thread anugrah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5245?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] anugrah reassigned KAFKA-5245: -- Assignee: anugrah > KStream builder should capture serdes > -- >

[jira] [Commented] (KAFKA-5245) KStream builder should capture serdes

2017-05-19 Thread anugrah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017431#comment-16017431 ] anugrah commented on KAFKA-5245: sure. I will start on this. > KStream builder should capture serdes >

[jira] [Updated] (KAFKA-5293) Do not apply exponential backoff if users have overridden reconnect.backoff.ms

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5293: --- Description: The PR for KAFKA-3878 implemented KIP-144 with one exception: it automatically enables

[jira] [Updated] (KAFKA-5293) Do not apply exponential backoff if users have overridden reconnect.backoff.ms

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5293: --- Issue Type: Bug (was: Improvement) > Do not apply exponential backoff if users have overridden

[jira] [Created] (KAFKA-5293) Do not apply exponential backoff if users have overridden reconnect.backoff.ms

2017-05-19 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5293: -- Summary: Do not apply exponential backoff if users have overridden reconnect.backoff.ms Key: KAFKA-5293 URL: https://issues.apache.org/jira/browse/KAFKA-5293 Project:

[jira] [Updated] (KAFKA-3878) Exponential backoff for broker reconnect attempts (KIP-144)

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3878: --- Summary: Exponential backoff for broker reconnect attempts (KIP-144) (was: Exponential backoff for

[jira] [Commented] (KAFKA-3878) Exponential backoff for broker reconnect attempts

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

[GitHub] kafka pull request #1523: KAFKA-3878: Support exponential backoff policy via...

2017-05-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1523 --- 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-3878) Exponential backoff for broker reconnect attempts

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

[jira] [Assigned] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-19 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-5154: - Assignee: Damian Guy (was: Matthias J. Sax) > Kafka Streams throws NPE during rebalance >

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

2017-05-19 Thread Ismael Juma
Hi all, Feedback from people who tried the AdminClient is that auto topic creation during describe is unexpected and confusing. This is consistent with the reaction of most people when they learn that MetadataRequest can cause topics to be created. We had assumed that we'd tackle this issue for

[jira] [Created] (KAFKA-5292) Authorization tests for AdminClient

2017-05-19 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5292: -- Summary: Authorization tests for AdminClient Key: KAFKA-5292 URL: https://issues.apache.org/jira/browse/KAFKA-5292 Project: Kafka Issue Type: Sub-task

[Vote] KIP-150 - Kafka-Streams Cogroup

2017-05-19 Thread Kyle Winkelman
Hello all, I would like to start the vote on KIP-150. https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup Thanks, Kyle

[jira] [Commented] (KAFKA-5278) kafka-console-consumer: `--value-deserializer` is not working but `--property value.deserializer` does

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

[GitHub] kafka pull request #3100: KAFKA-5278: Have console-consumer support `--value...

2017-05-19 Thread amethystic
GitHub user amethystic opened a pull request: https://github.com/apache/kafka/pull/3100 KAFKA-5278: Have console-consumer support `--value-deserializer` In the original implementation, console-consumer fails to honor `--value-deserializer` config. You can merge this pull request

[jira] [Assigned] (KAFKA-5278) kafka-console-consumer: `--value-deserializer` is not working but `--property value.deserializer` does

2017-05-19 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] huxi reassigned KAFKA-5278: --- Assignee: huxi > kafka-console-consumer: `--value-deserializer` is not working but `--property >

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

2017-05-19 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4660) Improve test coverage KafkaStreams

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

[GitHub] kafka pull request #3099: KAFKA-4660 : Improve test coverage KafkaStreams

2017-05-19 Thread umesh9794
GitHub user umesh9794 opened a pull request: https://github.com/apache/kafka/pull/3099 KAFKA-4660 : Improve test coverage KafkaStreams @dguy , @mjsax Please review the PR and let me know your comments. You can merge this pull request into a Git repository by running: $ git

[jira] [Updated] (KAFKA-5291) AdminClient should not trigger auto creation of topics

2017-05-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5291?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5291: --- Status: Patch Available (was: Open) > AdminClient should not trigger auto creation of topics >

[jira] [Commented] (KAFKA-5291) AdminClient should not trigger auto creation of topics

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

[GitHub] kafka pull request #3098: KAFKA-5291: AdminClient should not trigger auto cr...

2017-05-19 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3098 KAFKA-5291: AdminClient should not trigger auto creation of topics - Added a boolean allow_auto_topic_creation to MetadataRequest. I didn’t bump the version a second time since we did it once

[jira] [Updated] (KAFKA-5289) One StopReplicaRequest will caused two Responses

2017-05-19 Thread Ma Tianchi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ma Tianchi updated KAFKA-5289: -- Attachment: KAFKA-5289.patch just catch it > One StopReplicaRequest will caused two Responses >

[jira] [Commented] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-05-19 Thread james chien (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16017189#comment-16017189 ] james chien commented on KAFKA-4830: [~mjsax] Okay, I will study about how to write it :) > Augment

[jira] [Commented] (KAFKA-5134) Replace zkClient.getChildren method with zkUtils.getChildren

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

[GitHub] kafka pull request #3097: KAFKA-5134 Replace zkClient.getChildren method wit...

2017-05-19 Thread baluchicken
GitHub user baluchicken opened a pull request: https://github.com/apache/kafka/pull/3097 KAFKA-5134 Replace zkClient.getChildren method with zkUtils.getChildren @ijuma plz review. I also created a version where I didn't refactor the getPartitionAssignmentForTopics and the

[jira] [Created] (KAFKA-5291) AdminClient should not trigger auto creation of topics

2017-05-19 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5291: -- Summary: AdminClient should not trigger auto creation of topics Key: KAFKA-5291 URL: https://issues.apache.org/jira/browse/KAFKA-5291 Project: Kafka Issue Type:

  1   2   >