Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-12 Thread Damian Guy
+1 On Sun, 11 Dec 2016 at 08:07 Ewen Cheslack-Postava wrote: > +1 (binding) > > My only concern was around compatibility. It seems like the one case it is > incompatible would be, at worst, an extremely unusual edge case (and I > *think* can be restricted further to "not source compatible for an

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Rajini Sivaram
Thank you for the really well thought out KIP. This is a very neat approach to solve a rather complex problem. I haven't yet digested the doc. But I have a few comments and questions. 11 a). It feels like memory usage in the consumer is unbounded. If an application crashed mid-transaction, lots o

[GitHub] kafka pull request #2244: KAFKA-4490: Add Global Table support to Kafka Stre...

2016-12-12 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2244 KAFKA-4490: Add Global Table support to Kafka Streams Add Global Tables to KafkaStreams. Global Tables are fully replicated once-per instance of KafkaStreams. A single thread is used to update them. Th

[jira] [Commented] (KAFKA-4490) Add Global Table support to Kafka Streams

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

[VOTE] KIP-99: Add Global Tables to Kafka Streams

2016-12-12 Thread Damian Guy
Hi all, I'd like to start the vote for KIP-99: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67633649 There is a PR for it here: https://github.com/apache/kafka/pull/2244 Thanks, Damian

[jira] [Updated] (KAFKA-4490) Add Global Table support to Kafka Streams

2016-12-12 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4490: -- Status: Patch Available (was: Open) > Add Global Table support to Kafka Streams > -

[jira] [Assigned] (KAFKA-4473) RecordCollector should handle retriable exceptions more strictly

2016-12-12 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-4473: - Assignee: Damian Guy > RecordCollector should handle retriable exceptions more strictly > ---

[jira] [Work started] (KAFKA-4473) RecordCollector should handle retriable exceptions more strictly

2016-12-12 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4473 started by Damian Guy. - > RecordCollector should handle retriable exceptions more strictly > --

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Rajini Sivaram
A few more comments: 16) There are a couple of statements in the doc which don't quite sound right: Motivation Section: In particular, if a set of consumed messages c are transformed to result in set of produced messages p, then all the messages in c will be considered consumed if and only if all

[jira] [Commented] (KAFKA-4514) Add Codec for ZStandard Compression

2016-12-12 Thread Thomas Graves (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15741991#comment-15741991 ] Thomas Graves commented on KAFKA-4514: -- [~dongjin] That is fine, go ahead. > Add Co

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-12 Thread Bill Bejeck
+1 On Mon, Dec 12, 2016 at 3:44 AM, Damian Guy wrote: > +1 > > On Sun, 11 Dec 2016 at 08:07 Ewen Cheslack-Postava > wrote: > > > +1 (binding) > > > > My only concern was around compatibility. It seems like the one case it > is > > incompatible would be, at worst, an extremely unusual edge case

Re: [VOTE] 0.10.1.1 RC0

2016-12-12 Thread Moczarski, Swen
-0 (non-binding) Would it make sense to include https://issues.apache.org/jira/browse/KAFKA-4497 ? The issue sounds quite critical and a patch for 0.10.1.1 seems to be available. On 2016-12-07 23:46 (+0100), Guozhang Wang mailto:w...@gmail.com>> wrote: > Hello Kafka users, developers and clien

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-12 Thread Eno Thereska
+1 (non-binding) > On 12 Dec 2016, at 15:35, Bill Bejeck wrote: > > +1 > > On Mon, Dec 12, 2016 at 3:44 AM, Damian Guy wrote: > >> +1 >> >> On Sun, 11 Dec 2016 at 08:07 Ewen Cheslack-Postava >> wrote: >> >>> +1 (binding) >>> >>> My only concern was around compatibility. It seems like the

[jira] [Created] (KAFKA-4523) Controlled shutdown fails if consumer group restabilizes during shutdown

2016-12-12 Thread Steve Niemitz (JIRA)
Steve Niemitz created KAFKA-4523: Summary: Controlled shutdown fails if consumer group restabilizes during shutdown Key: KAFKA-4523 URL: https://issues.apache.org/jira/browse/KAFKA-4523 Project: Kafka

[GitHub] kafka-site pull request #37: Add new meetup link on events page

2016-12-12 Thread kokumutyoukan
GitHub user kokumutyoukan opened a pull request: https://github.com/apache/kafka-site/pull/37 Add new meetup link on events page Added Japan meetup link. If the event page should be meetup.com, I will consider migration to meetup.com. You can merge this pull request into a Git

[jira] [Updated] (KAFKA-4523) Controlled shutdown fails if consumer group restabilizes during shutdown

2016-12-12 Thread Steve Niemitz (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4523?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Steve Niemitz updated KAFKA-4523: - Description: If I begin a controlled shutdown of a broker that is a coordinator for a consumer gr

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

2016-12-12 Thread Manikumar
Hi, I would like to reinitiate the discussion on Delegation token support for Kafka. Brief summary of the past discussion: 1) Broker stores delegation tokens in zookeeper. All brokers will have a cache backed by zookeeper so they will all get notified whenever a new token is generated and th

[jira] [Commented] (KAFKA-1696) Kafka should be able to generate Hadoop delegation tokens

2016-12-12 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15742486#comment-15742486 ] Manikumar Reddy commented on KAFKA-1696: .[~singhashish] Sorry for the late reply.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Andrew Schofield
Guozhang, Exactly. This is the crux of the matter. Because it's async, the log is basically slightly out of date wrt to the run-time state and a failure of all replicas might take the data slightly back in time. Given this, do you think that KIP-98 gives an all-or-nothing, no-matter-what guaran

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2016-12-12 Thread Matthias J. Sax
+1 On 12/12/16 3:45 AM, Damian Guy wrote: > Hi all, > > I'd like to start the vote for KIP-99: > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67633649 > > There is a PR for it here: https://github.com/apache/kafka/pull/2244 > > Thanks, > Damian > signature.asc Descriptio

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2016-12-12 Thread Bill Bejeck
+1 On Mon, Dec 12, 2016 at 12:29 PM, Matthias J. Sax wrote: > +1 > > On 12/12/16 3:45 AM, Damian Guy wrote: > > Hi all, > > > > I'd like to start the vote for KIP-99: > > https://cwiki.apache.org/confluence/pages/viewpage. > action?pageId=67633649 > > > > There is a PR for it here: https://githu

Re: [VOTE]: KIP-97: The client compatibility KIP

2016-12-12 Thread Colin McCabe
With binding +1 votes from Gwen Shapira, Ismael Juma, Sriram Subramanian, Guozhang Wang, Neha Narkhede, Jay Kreps, Ewen Cheslack-Postava, and non-binding votes from Edoardo Comar and Rajini Sivaram, vote passes. There were no +0 or -1 votes. cheers, Colin McCabe On Sat, Dec 10, 2016, at 21:46, E

[jira] [Resolved] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-12-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4405. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2193 [https:

[GitHub] kafka pull request #2193: KAFKA-4405: Check max.poll.records before prefetch...

2016-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2193 --- 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] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

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

Re: [VOTE] KIP-88: OffsetFetch Protocol Update

2016-12-12 Thread Vahid S Hashemian
Happy Monday. Thanks to everyone who provided feedback or voted on this KIP. It passed with 3 binding and 4 non-binding votes. Ewen, that sounds good. We can revisit the AdminClient part when there is consensus on KIP-4 about AdminClient. Regards, --Vahid From: Ewen Cheslack-Postava To:

[GitHub] kafka pull request #2239: Docs templates

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

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15742677#comment-15742677 ] Jun Rao commented on KAFKA-4497: [~becket_qin], thanks for providing the fix. The fix seem

[GitHub] kafka pull request #2245: get docs setup with handlebars and seperate stream...

2016-12-12 Thread derrickdoo
GitHub user derrickdoo opened a pull request: https://github.com/apache/kafka/pull/2245 get docs setup with handlebars and seperate streams out to its own page - Seperate Streams documentation out to a standalone page. - Setup templates to use handlebars.js - Create template

[jira] [Resolved] (KAFKA-4518) Kafka-connect Not starting after DB restarts

2016-12-12 Thread Akshath Patkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4518?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Akshath Patkar resolved KAFKA-4518. --- Resolution: Not A Problem > Kafka-connect Not starting after DB restarts > ---

[jira] [Commented] (KAFKA-4518) Kafka-connect Not starting after DB restarts

2016-12-12 Thread Akshath Patkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4518?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15742761#comment-15742761 ] Akshath Patkar commented on KAFKA-4518: --- This looks like a issue with server.id conf

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-12 Thread Guozhang Wang
+1 (binding) Thanks Xavier for the proposal ! I think the source compatibility should be very rare as of now. On Mon, Dec 12, 2016 at 8:20 AM, Eno Thereska wrote: > +1 (non-binding) > > > On 12 Dec 2016, at 15:35, Bill Bejeck wrote: > > > > +1 > > > > On Mon, Dec 12, 2016 at 3:44 AM, Damian Gu

[jira] [Created] (KAFKA-4524) ConfigDef.Type.LIST does not handle escaping

2016-12-12 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4524: -- Summary: ConfigDef.Type.LIST does not handle escaping Key: KAFKA-4524 URL: https://issues.apache.org/jira/browse/KAFKA-4524 Project: Kafka Issue Type: Bu

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

2016-12-12 Thread Ashish Singh
Manikumar, Thanks for the update. As its been a while since any progress was made here, I started working on POC for this. Will be happy to share the progress I have made so far, let's discuss that on parent JIRA. However, more importantly while working on POC, I had following questions that I thi

[jira] [Created] (KAFKA-4525) Kafka should not require SSL trust store password

2016-12-12 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-4525: -- Summary: Kafka should not require SSL trust store password Key: KAFKA-4525 URL: https://issues.apache.org/jira/browse/KAFKA-4525 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #2246: KAFKA-4525: Kafka should not require SSL trust sto...

2016-12-12 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/2246 KAFKA-4525: Kafka should not require SSL trust store password You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka truststore-pas

[jira] [Commented] (KAFKA-4525) Kafka should not require SSL trust store password

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

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

2016-12-12 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743122#comment-15743122 ] Jiangjie Qin commented on KAFKA-4497: - [~junrao] I think 0x is actually -1, ri

[jira] [Updated] (KAFKA-4460) Consumer stops getting messages when partition leader dies

2016-12-12 Thread Bernhard Bonigl (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4460?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bernhard Bonigl updated KAFKA-4460: --- Description: I have a setup consisting of 2 Kafka broker (0 and 1) using a zookeeper, a sprin

[jira] [Commented] (KAFKA-2500) Expose fetch response high watermark in ConsumerRecords

2016-12-12 Thread Robert Metzger (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743186#comment-15743186 ] Robert Metzger commented on KAFKA-2500: --- +1 to get this feature into the Java Kafka

[jira] [Commented] (KAFKA-4460) Consumer stops getting messages when partition leader dies

2016-12-12 Thread Bernhard Bonigl (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743183#comment-15743183 ] Bernhard Bonigl commented on KAFKA-4460: Updated with missing links: Log: https://

[jira] [Created] (KAFKA-4526) Transient failure in ThrottlingTest.test_throttled_reassignment

2016-12-12 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-4526: Summary: Transient failure in ThrottlingTest.test_throttled_reassignment Key: KAFKA-4526 URL: https://issues.apache.org/jira/browse/KAFKA-4526 Project

[jira] [Commented] (KAFKA-4526) Transient failure in ThrottlingTest.test_throttled_reassignment

2016-12-12 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4526?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743246#comment-15743246 ] Ewen Cheslack-Postava commented on KAFKA-4526: -- re: related test failures, we

[jira] [Created] (KAFKA-4527) Transient failure of ConnectDistributedTest.test_pause_and_resume_sink where paused connector produces messages

2016-12-12 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-4527: Summary: Transient failure of ConnectDistributedTest.test_pause_and_resume_sink where paused connector produces messages Key: KAFKA-4527 URL: https://issues.apach

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-12-12 Thread Nisarg Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743278#comment-15743278 ] Nisarg Shah commented on KAFKA-3209: [~shikhar] the proposal looks details and flexibl

[jira] [Commented] (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.

2016-12-12 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743283#comment-15743283 ] Apurva Mehta commented on KAFKA-4477: - Hi [~tdevoe]: Thanks for sharing those logs. I

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-12 Thread Neha Narkhede
+1 (binding) On Mon, Dec 12, 2016 at 11:41 AM Guozhang Wang wrote: > +1 (binding) Thanks Xavier for the proposal ! > > I think the source compatibility should be very rare as of now. > > On Mon, Dec 12, 2016 at 8:20 AM, Eno Thereska > wrote: > > > +1 (non-binding) > > > > > On 12 Dec 2016, at 1

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743297#comment-15743297 ] Jun Rao commented on KAFKA-4497: [~becket_qin], it could be. Where do we lose the negative

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-12-12 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743300#comment-15743300 ] Shikhar Bhushan commented on KAFKA-3209: Thanks [~snisarg]. I self-assigned it as

[GitHub] kafka pull request #2247: MINOR: Fix Streams examples in documentation

2016-12-12 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2247 MINOR: Fix Streams examples in documentation Performed minor cleanup and escaped `<` and `>` so code examples are shown correctly in the browser. You can merge this pull request into a Git r

[GitHub] kafka pull request #2248: add instrumentation to follower local time

2016-12-12 Thread rnpridgeon
GitHub user rnpridgeon opened a pull request: https://github.com/apache/kafka/pull/2248 add instrumentation to follower local time You can merge this pull request into a Git repository by running: $ git pull https://github.com/junrao/kafka instrument-0.9.0 Alternatively you c

[jira] [Created] (KAFKA-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

2016-12-12 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4528: Summary: Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout Key: KAFKA-4528 URL: https://issues.apache.org/jira/browse/KAFKA-4528 Project

[jira] [Updated] (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.

2016-12-12 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tom DeVoe updated KAFKA-4477: - Attachment: issue_node_1003_ext.log issue_node_1002_ext.log issue_node_1001

[jira] [Commented] (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.

2016-12-12 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743405#comment-15743405 ] Tom DeVoe commented on KAFKA-4477: -- It didn't seem like there was anything interesting th

[GitHub] kafka pull request #2248: add instrumentation to follower local time

2016-12-12 Thread rnpridgeon
Github user rnpridgeon closed the pull request at: https://github.com/apache/kafka/pull/2248 --- 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: [VOTE] 0.10.1.1 RC0

2016-12-12 Thread Ismael Juma
Yes, it would be good to include that (that's why I set the fix version to 0.10.1.1 a couple of days ago). Ismael On Mon, Dec 12, 2016 at 3:47 PM, Moczarski, Swen < smoczar...@ebay-kleinanzeigen.de> wrote: > -0 (non-binding) > > Would it make sense to include https://issues.apache.org/ > jira/br

[jira] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-12-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743437#comment-15743437 ] Ismael Juma commented on KAFKA-4405: Thanks for the PR [~enothereska]. Can you please

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-12 Thread Jason Gustafson
Yeah, that's a good point. Perhaps in retrospect, it would have been better to define `buffer.memory` first and let `fetch.max.bytes` be based off of it. I like `buffer.memory` since it gives the consumer nice symmetry with the producer and its generic naming gives us some flexibility internally wi

[jira] [Commented] (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.

2016-12-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743503#comment-15743503 ] Jason Gustafson commented on KAFKA-4477: A deadlock is what we were hoping to see.

[jira] [Commented] (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.

2016-12-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743504#comment-15743504 ] Jason Gustafson commented on KAFKA-4477: A deadlock is what we were hoping to see.

[jira] [Issue Comment Deleted] (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.

2016-12-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4477: --- Comment: was deleted (was: A deadlock is what we were hoping to see. We have fixed a couple of

[jira] [Commented] (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.

2016-12-12 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743529#comment-15743529 ] Apurva Mehta commented on KAFKA-4477: - Thanks [~tdevoe], These seem to be the broker l

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-12-12 Thread Nisarg Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743641#comment-15743641 ] Nisarg Shah commented on KAFKA-3209: Yeah sorry about that, I got caught up with schoo

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743650#comment-15743650 ] Jiangjie Qin commented on KAFKA-4497: - The negative bit is the MSB which is 1 in 0XFFF

Re: [VOTE] 0.10.1.1 RC0

2016-12-12 Thread Guozhang Wang
@Bernard You should be able to find the artifacts in the staging RC repository I listed above, named as "kafka_2.12-0.10.1.1.tgz ". @Everyone Since KAFKA-4497 is a critical issue, that anyone using compaction with comp

[jira] [Comment Edited] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743650#comment-15743650 ] Jiangjie Qin edited comment on KAFKA-4497 at 12/13/16 12:38 AM:

[jira] [Comment Edited] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743650#comment-15743650 ] Jiangjie Qin edited comment on KAFKA-4497 at 12/13/16 12:40 AM:

Re: [VOTE] 0.10.1.1 RC0

2016-12-12 Thread Bernard Leach
I found those ones but was hoping to see them in https://repository.apache.org/content/groups/staging/org/apache/kafka/ so we can just point the maven build there for testing. > On 13 Dec 2016, at 11:38, Guozhang Wang wro

Re: [VOTE] 0.10.1.1 RC0

2016-12-12 Thread Guozhang Wang
I see. Currently the upload command has not included the 2.12 version yet, I will manually do that in the next RC. Guozhang On Mon, Dec 12, 2016 at 4:47 PM, Bernard Leach wrote: > I found those ones but was hoping to see them in > https://repository.apache.org/content/groups/staging/org/apache/

[jira] [Commented] (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.

2016-12-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743728#comment-15743728 ] Jun Rao commented on KAFKA-4477: [~tdevoe], thanks for sharing the log. On node 1001 and

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Apurva Mehta
@Becket and @Rajini, Thanks for those comments. You raise some very astute points. I will address a subset of them here. One common thread across your emails has to do with the notion of a 'batch' of messages from the consumer's point of view. In particular, Rajini's points 12 and 16, and Becket'

[jira] [Commented] (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.

2016-12-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743748#comment-15743748 ] Jun Rao commented on KAFKA-4477: [~michael.andre.pearce], the issue that you reported is a

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743756#comment-15743756 ] Jun Rao commented on KAFKA-4497: Got it. I thought Robert was showing the DumpLogSegment o

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Guozhang Wang
Andrew, As I mentioned above, in Kafka durability is supported via data replication instead of sync-flushing to disks. KIP-98 does not try to change that part of the Kafka: if all your replicas are gone at the same time before the data was ever flushed to disks, then your data is lost today, and i

[jira] [Commented] (KAFKA-4516) When a CachingStateStore is closed it should clear its associated NamedCache. Subsequent queries should throw InvalidStateStoreException

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

[jira] [Updated] (KAFKA-4516) When a CachingStateStore is closed it should clear its associated NamedCache. Subsequent queries should throw InvalidStateStoreException

2016-12-12 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4516: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[GitHub] kafka pull request #2235: KAFKA-4516: When a CachingStateStore is closed it ...

2016-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2235 --- 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] [Commented] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743832#comment-15743832 ] Michał Woś commented on KAFKA-1407: --- [~dmitrybugaychenko] +1 The same issue in 0.8.2.0.

[jira] [Issue Comment Deleted] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michał Woś updated KAFKA-1407: -- Comment: was deleted (was: [~dmitrybugaychenko] +1 The same issue in 0.8.2.0. Scenario 1. Broker looses

[jira] [Comment Edited] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743832#comment-15743832 ] Michał Woś edited comment on KAFKA-1407 at 12/13/16 2:03 AM: -

[jira] [Commented] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743840#comment-15743840 ] Michał Woś commented on KAFKA-1407: --- [~dmitrybugaychenko] +1 The same issue in 0.8.2.0.

[jira] [Comment Edited] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743840#comment-15743840 ] Michał Woś edited comment on KAFKA-1407 at 12/13/16 2:04 AM: -

[jira] [Comment Edited] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743840#comment-15743840 ] Michał Woś edited comment on KAFKA-1407 at 12/13/16 2:05 AM: -

[jira] [Comment Edited] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743840#comment-15743840 ] Michał Woś edited comment on KAFKA-1407 at 12/13/16 2:25 AM: -

[jira] [Comment Edited] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743840#comment-15743840 ] Michał Woś edited comment on KAFKA-1407 at 12/13/16 2:26 AM: -

[jira] [Comment Edited] (KAFKA-1407) Broker can not return to ISR because of BadVersionException

2016-12-12 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743840#comment-15743840 ] Michał Woś edited comment on KAFKA-1407 at 12/13/16 2:27 AM: -

[jira] [Created] (KAFKA-4529) tombstone may be removed earlier than it should

2016-12-12 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4529: -- Summary: tombstone may be removed earlier than it should Key: KAFKA-4529 URL: https://issues.apache.org/jira/browse/KAFKA-4529 Project: Kafka Issue Type: Bug Affects

[jira] [Commented] (KAFKA-4529) tombstone may be removed earlier than it should

2016-12-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743987#comment-15743987 ] Jun Rao commented on KAFKA-4529: [~becket_qin], do you think you have time to fix this iss

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Jason Gustafson
@Becket It has been a pain in many cases that we do not know the number of >messages in a message set, not sure if the OffsetDelta field in the > wrapper >message will address this. Interestingly, we had this in one of the design iterations, but we found in the prototype that we weren't

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Jason Gustafson
@Neha 1. I think we should consider renaming initTransactions to just init() and > moving the metadata initialization there. Let's make sure we don't add APIs > that are relevant to this proposal only. Instead, try to think what we'd > propose if we were writing the producer from scratch today. I

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Jay Kreps
Hey Jason/Neha, Yeah, clearly having a mandatory, generic init() method that initializes both transactions and topic metadata would be the ideal solution. This would solve the occasional complaint about blocking behavior during initialization of metadata (or at least shift it to a new complaint ab

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-12 Thread Jay Kreps
I think the question is whether we have a truly optimal strategy for deriving the partition- and fetch-level configs from the global setting. If we do then we should just get rid of them. If not, then if we can at least derive usually good and never terrible settings from the global limit at initia

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

2016-12-12 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4516: When a CachingStateStore is closed it should clear its -- [...truncated 12570 lines...] :streams:classes :streams:checkstyleMain :streams:compileTestJavaNote:

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Ismael Juma
Hi Jay, I like the idea of having a single `init`, but I am not sure about the specifics of the metadata initialisation (as Jason alluded to). More inline. On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps wrote: >1. Add a generic init() call which initializes both transactions and >metadata >

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-12 Thread Ismael Juma
Hi Jay, About `max.partition.fetch.bytes`, yes it was an oversight not to lower its priority as part of KIP-74 given the existence of `fetch.max.bytes` and the fact that we can now make progress in the presence of oversized messages independently of either of those settings. I agree that we shoul