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

2016-12-15 Thread Manikumar
Hi, > Can you add a sample Jaas configuration using delegation tokens to the KIP? > Will add sample Jaas configuration. > To make sure I have understood correctly, KAFKA-3712 is aimed at enabling a > superuser to impersonate another (single) user, say alice. A producer using > impersonation

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

2016-12-15 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4537: StreamPartitionAssignor incorrectly adds standby partitions [wangguoz] KAFKA-4539: StreamThread is not correctly creating StandbyTasks [wangguoz] MINOR: Fix typo on introduction page

[jira] [Commented] (KAFKA-4549) KafkaLZ4OutputStream output invalid format when it was called close method without flash method

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

[GitHub] kafka pull request #2265: KAFKA-4549: Change to call flush method before wri...

2016-12-15 Thread fossamagna
GitHub user fossamagna opened a pull request: https://github.com/apache/kafka/pull/2265 KAFKA-4549: Change to call flush method before writeEndMark method in close method You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka-site pull request #34: Fix typo on introduction page

2016-12-15 Thread ashishg-qburst
Github user ashishg-qburst closed the pull request at: https://github.com/apache/kafka-site/pull/34 --- 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

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-15 Thread radai
@Matthias - oh. I think over the course of this thread enough use cases have been presented for things that can be done/solved with headers that even if every single potential use case has a better custom implementation (which I dont believe) headers are clearly one of the best possible kafka

[jira] [Commented] (KAFKA-4507) The client should send older versions of requests to the broker if necessary

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

[GitHub] kafka pull request #2264: Kafka 4507: The client should send older versions ...

2016-12-15 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2264 Kafka 4507: The client should send older versions of requests to the broker if necessary KAFKA-4507 The client should send older versions of requests to the broker if necessary.

[jira] [Work started] (KAFKA-4507) The client should send older versions of requests to the broker if necessary

2016-12-15 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4507?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4507 started by Colin P. McCabe. -- > The client should send older versions of requests to the broker if

[VOTE] KIP-92 - Add per partition lag metrics to KafkaConsumer

2016-12-15 Thread Becket Qin
Hi, I want to start a voting thread on KIP-92 which proposes to add per partition lag metrics to KafkaConsumer. The KIP wiki page is below: https://cwiki.apache.org/confluence/display/KAFKA/KIP-92+-+Add+per+partition+lag+metrics+to+KafkaConsumer Thanks, Jiangjie (Becket) Qin

[jira] [Updated] (KAFKA-4550) current trunk unstable

2016-12-15 Thread radai rosenblatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] radai rosenblatt updated KAFKA-4550: Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 > current trunk unstable >

[GitHub] kafka pull request #2259: MINOR: Fix typo on introduction page

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

[GitHub] kafka-site issue #34: Fix typo on introduction page

2016-12-15 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/34 Could you close this PR then? --- 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

[jira] [Updated] (KAFKA-4539) StreamThread is not correctly creating StandbyTasks

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

[jira] [Updated] (KAFKA-4550) current trunk unstable

2016-12-15 Thread radai rosenblatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] radai rosenblatt updated KAFKA-4550: Description: on latest trunk (commit hash 908b6d1148df963d21a70aaa73a7a87571b965a9) when

[jira] [Commented] (KAFKA-4539) StreamThread is not correctly creating StandbyTasks

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

[GitHub] kafka pull request #2255: KAFKA-4539: StreamThread is not correctly creating...

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

[jira] [Created] (KAFKA-4550) current trunk unstable

2016-12-15 Thread radai rosenblatt (JIRA)
radai rosenblatt created KAFKA-4550: --- Summary: current trunk unstable Key: KAFKA-4550 URL: https://issues.apache.org/jira/browse/KAFKA-4550 Project: Kafka Issue Type: Bug Affects

[jira] [Updated] (KAFKA-4550) current trunk unstable

2016-12-15 Thread radai rosenblatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] radai rosenblatt updated KAFKA-4550: Attachment: run5.log run4.log run3.log

[jira] [Closed] (KAFKA-4536) Kafka clients throw NullPointerException on poll when delete the relative topic

2016-12-15 Thread mayi_hetu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mayi_hetu closed KAFKA-4536. > Kafka clients throw NullPointerException on poll when delete the relative > topic >

[jira] [Resolved] (KAFKA-4536) Kafka clients throw NullPointerException on poll when delete the relative topic

2016-12-15 Thread mayi_hetu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mayi_hetu resolved KAFKA-4536. -- Resolution: Fixed Fix Version/s: 0.10.1.0 > Kafka clients throw NullPointerException on poll

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-15 Thread Jun Rao
Hi, Michael, Thanks for the response. 100. Is there any other metadata associated with the uuid that APM sends to the central coordinator? What kind of things could you do once the tracing is embedded in each message? 103. How do you preserve the per key ordering when switching to a different

[jira] [Created] (KAFKA-4549) KafkaLZ4OutputStream output invalid format when it was called close method without flash method

2016-12-15 Thread MURAKAMI Masahiko (JIRA)
MURAKAMI Masahiko created KAFKA-4549: Summary: KafkaLZ4OutputStream output invalid format when it was called close method without flash method Key: KAFKA-4549 URL:

[jira] [Updated] (KAFKA-4521) MirrorMaker should flush all messages before releasing partition ownership during rebalance

2016-12-15 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-4521: Affects Version/s: 0.10.1.0 Fix Version/s: 0.10.2.0 > MirrorMaker should flush all messages

[jira] [Resolved] (KAFKA-4521) MirrorMaker should flush all messages before releasing partition ownership during rebalance

2016-12-15 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin resolved KAFKA-4521. - Resolution: Fixed > MirrorMaker should flush all messages before releasing partition ownership >

Re: [DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-15 Thread Becket Qin
+1 on the idea. We have a ticket about making all the blocking call have a timeout in KafkaConsumer. The implementation could be a little tricky as Ewen mentioned. But for close it is probably a simpler case because in the worst case the consumer will just stop polling and heartbeating and

[jira] [Updated] (KAFKA-4539) StreamThread is not correctly creating StandbyTasks

2016-12-15 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4539: - Description: Fails because {{createStandbyTask(..)}} can return null if the topology for the

[jira] [Resolved] (KAFKA-4537) StreamPartitionAssignor incorrectly adds standby partitions to the partitionsByHostState map

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

[jira] [Commented] (KAFKA-4537) StreamPartitionAssignor incorrectly adds standby partitions to the partitionsByHostState map

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

[GitHub] kafka pull request #2254: KAFKA-4537: StreamPartitionAssignor incorrectly ad...

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

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Shikhar Bhushan
There is no decision being proposed on the final list of transformations that will ever be in Kafka :-) Just the initial set we should roll with. On Thu, Dec 15, 2016 at 3:34 PM Gwen Shapira wrote: You are absolutely right that the vast majority of NiFi's processors are not

[jira] [Commented] (KAFKA-4541) Add capability to create delegation token

2016-12-15 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4541?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15752982#comment-15752982 ] Ashish K Singh commented on KAFKA-4541: --- Yes, it will. > Add capability to create delegation token

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

2016-12-15 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2258: MINOR: update KStream JavaDocs

2016-12-15 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2258 --- 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-4540) Suspended tasks that are not assigned to the StreamThread need to be closed before new active and standby tasks are created

2016-12-15 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15752887#comment-15752887 ] Guozhang Wang commented on KAFKA-4540: -- Thanks for the explanation. Makes sense. > Suspended tasks

[jira] [Commented] (KAFKA-4451) Recovering empty replica yields negative offsets in index of compact partitions

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

[GitHub] kafka pull request #2210: KAFKA-4451: Fix OffsetIndex overflow when replicat...

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

[jira] [Resolved] (KAFKA-4451) Recovering empty replica yields negative offsets in index of compact partitions

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

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-12-15 Thread Matthias J. Sax
What is you wiki ID? We can grant you permission. -Matthias On 12/15/16 3:27 PM, Avi Flax wrote: > >> On Dec 13, 2016, at 21:02, Matthias J. Sax wrote: >> >> thanks for your feedback. > > My pleasure! > >> We want to enlarge the scope for Streams >> application and

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Gwen Shapira
You are absolutely right that the vast majority of NiFi's processors are not what we would consider SMT. I went over the list and I think the still contain just short of 50 legit SMTs: https://cwiki.apache.org/confluence/display/KAFKA/Analyzing+NiFi+Transformations You are right that ExtractHL7

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-12-15 Thread Avi Flax
> On Dec 13, 2016, at 21:02, Matthias J. Sax wrote: > > thanks for your feedback. My pleasure! > We want to enlarge the scope for Streams > application and started to collect use cases in the Wiki: > >

[jira] [Work started] (KAFKA-4508) Create system tests that run newer versions of the client against older versions of the broker

2016-12-15 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4508 started by Colin P. McCabe. -- > Create system tests that run newer versions of the client against older >

[jira] [Commented] (KAFKA-4508) Create system tests that run newer versions of the client against older versions of the broker

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

[GitHub] kafka pull request #2263: KAFKA-4508. Create system tests that run newer ver...

2016-12-15 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2263 KAFKA-4508. Create system tests that run newer versions of the client… KAFKA-4508. Create system tests that run newer versions of the client against older versions of the broker You can merge

Re: [DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-15 Thread Rajini Sivaram
Ewen, Thank you, I will try to prototype a solution early next week to get a better understanding of how invasive the changes are. On Thu, Dec 15, 2016 at 9:35 PM, Ewen Cheslack-Postava wrote: > Rajini, > > Thanks for this KIP, I'd definitely like to see this. Connect has

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

2016-12-15 Thread radai
some clarifications on my alternative proposal: TX msgs are written "sideways" to a transaction (ad-hoc) partition. this partition can be replicated to followers, or can be an in-mem buffer - depends on the resilience guarantees you want to provide for TXs in case of broker crash. on "commit" the

[jira] [Commented] (KAFKA-3540) KafkaConsumer.close() may block indefinitely

2016-12-15 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15752780#comment-15752780 ] Rajini Sivaram commented on KAFKA-3540: --- I have opened

Re: [DISCUSS] KIP 88: OffsetFetch Protocol Update

2016-12-15 Thread Vahid S Hashemian
Hi all, Even though KIP-88 was recently approved, due to a limitation that comes with the proposed protocol change in KIP-88 I'll have to re-open it to address the problem. I'd like to thank Jason Gustafson for catching this issue. I'll explain this in the KIP as well, but to summarize, KIP-88

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Ewen Cheslack-Postava
I think there are a couple of factors that make transformations and connectors different. First, NiFi's 150 processors is a bit misleading. In NiFi, processors cover data sources, data sinks, serialization/deserialization, *and* transformations. I haven't filtered the list to see how many fall

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Shikhar Bhushan
I think the tradeoffs for including connectors are different. Connectors are comparatively larger in scope, they tend to come with their own set of dependencies for the systems they need to talk to. Transformations as I imagine them - at least the ones on the table in the wiki currently - should

[jira] [Work started] (KAFKA-4548) Add CompatibilityTest to verify that individual features are supported or not by the broker we're connecting to

2016-12-15 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4548?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4548 started by Colin P. McCabe. -- > Add CompatibilityTest to verify that individual features are supported or

[jira] [Commented] (KAFKA-4548) Add CompatibilityTest to verify that individual features are supported or not by the broker we're connecting to

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

[GitHub] kafka pull request #2262: KAFKA-4548: Add CompatibilityTest to verify that i...

2016-12-15 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2262 KAFKA-4548: Add CompatibilityTest to verify that individual features … Add CompatibilityTest to verify that individual features are supported or not by the broker we're connecting to You can

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

2016-12-15 Thread radai
I can see several issues with the current proposal. messages, even if sent under a TX, are delivered directly to their destination partitions, downstream consumers need to be TX-aware. they can either: 1. be oblivious to TXs. that means they will deliver "garbage" - msgs sent during

Re: [VOTE] 0.10.1.1 RC0

2016-12-15 Thread Becket Qin
Yes, that sounds good. Thanks. Jiangjie (Becket) Qin On Thu, Dec 15, 2016 at 1:46 PM, Guozhang Wang wrote: > Hey Becket, > > I just cut the release this morning and the RC1 is out a few minutes ago so > that we can possibly have the release out before the break. I looked >

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

2016-12-15 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4529; LogCleaner should not delete the tombstone too early. -- [...truncated 26063 lines...] org.apache.kafka.common.record.MemoryRecordsBuilderTest >

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Gwen Shapira
I agree about the ease of use in adding a small-subset of built-in transformations. But the same thing is true for connectors - there are maybe 5 super popular OSS connectors and the rest is a very long tail. We drew the line at not adding any, because thats the easiest and because we did not

Re: [VOTE] 0.10.1.1 RC0

2016-12-15 Thread Guozhang Wang
Hey Becket, I just cut the release this morning and the RC1 is out a few minutes ago so that we can possibly have the release out before the break. I looked through https://issues.apache.org/jira/browse/KAFKA-4521 and feel it is OK to have it in the next minor release happening in a month or so.

[jira] [Created] (KAFKA-4548) Add CompatibilityTest to verify that individual features are supported or not by the broker we're connecting to

2016-12-15 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-4548: -- Summary: Add CompatibilityTest to verify that individual features are supported or not by the broker we're connecting to Key: KAFKA-4548 URL:

Re: [DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-15 Thread Ewen Cheslack-Postava
Rajini, Thanks for this KIP, I'd definitely like to see this. Connect has had a long-standing TODO around stopping sink tasks where we can't properly manage the rebalance process (which involves stopping consumers) because we lack a timeout here. Not a huge problem in practice, but would be nice

Re: [VOTE] 0.10.1.1 RC0

2016-12-15 Thread Becket Qin
Hey Guozhang, Thanks for running the release. KAFKA-4521 is just checked in. It fixes a bug in mirror maker that may result in message loss. Can we include that in 0.10.1.1 as well? Thanks, Jiangjie (Becket) Qin On Thu, Dec 15, 2016 at 9:46 AM, Guozhang Wang wrote: >

[VOTE] 0.10.1.1 RC1

2016-12-15 Thread Guozhang Wang
Hello Kafka users, developers and client-developers, This is the second, and hopefully the last candidate for the release of Apache Kafka 0.10.1.1 before the break. This is a bug fix release and it includes fixes and improvements from 30 JIRAs. See the release notes for more details:

[GitHub] kafka pull request #2241: KAFKA-4521; MirrorMaker should flush all messages ...

2016-12-15 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2241 --- 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-4521) MirrorMaker should flush all messages before releasing partition ownership during rebalance

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

[jira] [Assigned] (KAFKA-4547) Consumer.position returns incorrect results for Kafka 0.10.1.0 client

2016-12-15 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-4547: -- Assignee: Vahid Hashemian > Consumer.position returns incorrect results for Kafka

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

2016-12-15 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Shikhar Bhushan
I have updated KIP-66 https://cwiki.apache.org/confluence/display/KAFKA/KIP-66%3A+Single+Message+Transforms+for+Kafka+Connect with the changes I proposed in the design. Gwen, I think the main downside to not including some transformations with Kafka Connect is that it seems less user friendly if

[jira] [Updated] (KAFKA-4546) a consumer could miss tombstone when leader changes during the reads

2016-12-15 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-4546: --- Affects Version/s: 0.10.1.0 This is affecting all versions of Kafka. Not sure what's the best way to address

[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-15 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15752252#comment-15752252 ] Jun Rao commented on KAFKA-4477: [~michael.andre.pearce], the deadlock seems to be the same as in

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

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

[GitHub] kafka pull request #2260: KAFKA-4529; LogCleaner should not delete the tombs...

2016-12-15 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2260 --- 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-3539) KafkaProducer.send() may block even though it returns the Future

2016-12-15 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-3539: -- Assignee: Manikumar Reddy > KafkaProducer.send() may block even though it

[jira] [Updated] (KAFKA-3540) KafkaConsumer.close() may block indefinitely

2016-12-15 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-3540: -- Assignee: Manikumar Reddy > KafkaConsumer.close() may block indefinitely >

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

2016-12-15 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4529. -- Resolution: Fixed Marked as fixed for the 0.10.1.1 release process. > tombstone may be removed

[GitHub] kafka pull request #2252: HOTFIX: fix state transition stuck on rebalance

2016-12-15 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2252 --- 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-1696) Kafka should be able to generate Hadoop delegation tokens

2016-12-15 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15751936#comment-15751936 ] Manikumar Reddy commented on KAFKA-1696: Thanks for creating sub-tasks. About authentication, we

[GitHub] kafka pull request #2261: MINOR: Replace deepIterator/shallowIterator with d...

2016-12-15 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2261 MINOR: Replace deepIterator/shallowIterator with deepEntries/shallowEntries The latter return `Iterable` instead of `Iterator` so that enhanced foreach can be used in Java. You can merge this

[jira] [Commented] (KAFKA-4541) Add capability to create delegation token

2016-12-15 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4541?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15751744#comment-15751744 ] Manikumar Reddy commented on KAFKA-4541: [~singhashish] Is this JIRA work includes token storage

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-12-15 Thread Rajini Sivaram
Ismael, The reason for choosing CallbackHandler interface as the configurable interface is flexibility. As you say, we could instead define a simpler PlainCredentialProvider and ScramCredentialProvider. But that would tie users to Kafka's SaslServer implementation for PLAIN and SCRAM.

Re: Brokers cashing with OOME Map failed

2016-12-15 Thread Ismael Juma
Hi, This is probably not a Kafka bug, but we should improve the information we report in this case. Something along the lines of what Lucene did here: https://issues.apache.org/jira/browse/LUCENE-5673 This error may be caused by lack of enough unfragmented virtual address space or too

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-12-15 Thread Ismael Juma
Thanks Rajini, your answers make sense to me. One more general point: we are following the JAAS callback architecture and exposing that to the user where the user has to write code like: @Override public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {

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

2016-12-15 Thread Ismael Juma
Hi Rajini, The use case you outline is indeed the one I was thinking of. The concern is indeed what you pointed out, it could be a large change. Ismael On Thu, Dec 15, 2016 at 3:34 AM, Rajini Sivaram wrote: > @Mani > > Can you add a sample Jaas configuration using

[DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-15 Thread Rajini Sivaram
Hi all, I have just created KIP-102 to add a new close method for consumers with a timeout parameter, making Consumer consistent with Producer: https://cwiki.apache.org/confluence/display/KAFKA/KIP-102+-+Add+close+with+timeout+for+consumers Comments and suggestions are welcome. Thank you...

Re: Improve default Kafka logger settings to prevent extremely high disk space usage issue?

2016-12-15 Thread Jaikiran Pai
On Tuesday 13 December 2016 03:29 PM, Ismael Juma wrote: The log config settings for the controller and state change logger have been that way since they were introduced. They're generally useful when investigating issues with the controller. Looks like this is too noisy in some scenarios

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

2016-12-15 Thread Rajini Sivaram
@Mani Can you add a sample Jaas configuration using delegation tokens to the KIP? Since delegation tokens will be handled differently from other SCRAM credentials, it should work anyway, but it will be good to see an example of the configuration the user provides. It sounds like users provide

Re: [VOTE] 0.10.1.1 RC0

2016-12-15 Thread Michael Pearce
Is there any update on this, when do we expect and RC1 for vote? We see KAFKA-4497 is marked Resolved. Cheers Mike On 13/12/2016, 00:59, "Guozhang Wang" wrote: I see. Currently the upload command has not included the 2.12 version yet, I will manually do that in the

[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-15 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Andre Pearce (IG) updated KAFKA-4477: - Comment: was deleted (was: IG ISR issue of 2016-12-15 04:27 (this time we

[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-15 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Andre Pearce (IG) updated KAFKA-4477: - Attachment: 2016_12_15.zip IG ISR issue of 2016-12-15 04:27 (this time we

[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-15 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15751019#comment-15751019 ] Michael Andre Pearce (IG) commented on KAFKA-4477: -- Hi [~junrao] We had similar issue

[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-15 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750994#comment-15750994 ] Michael Andre Pearce (IG) commented on KAFKA-4477: -- Have we a timeline on RC1? It would

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

2016-12-15 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750982#comment-15750982 ] Ben Stopford commented on KAFKA-1120: - [~junrao] added this comment on a mail thread on this topic

[jira] [Updated] (KAFKA-4547) Consumer.position returns incorrect results for Kafka 0.10.1.0 client

2016-12-15 Thread Pranav Nakhe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pranav Nakhe updated KAFKA-4547: Description: Consider the following code - KafkaConsumer consumer

[jira] [Created] (KAFKA-4547) Consumer.position returns incorrect results for Kafka 0.10.1.0 client

2016-12-15 Thread Pranav Nakhe (JIRA)
Pranav Nakhe created KAFKA-4547: --- Summary: Consumer.position returns incorrect results for Kafka 0.10.1.0 client Key: KAFKA-4547 URL: https://issues.apache.org/jira/browse/KAFKA-4547 Project: Kafka

[jira] [Assigned] (KAFKA-4543) Add capability to renew/expire delegation tokens.

2016-12-15 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy reassigned KAFKA-4543: -- Assignee: Manikumar Reddy > Add capability to renew/expire delegation tokens. >

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

2016-12-15 Thread Rajini Sivaram
Hi Apurva, Thank you, makes sense. Rajini On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta wrote: > Hi Rajini, > > I think my original response to your point 15 was not accurate. The regular > definition of durability is that data once committed would never be lost. > So it

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-12-15 Thread Rajini Sivaram
Ismael, 1. At the moment AuthCallbackHandler is not a public interface, so I am assuming that it can be modified. Yes, agree that we should keep non-public methods separate. Will do that as part of the implementation of this KIP. 2. Callback handlers do tend to depend on ordering, including

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

2016-12-15 Thread Manikumar
@Gwen, @Rajini, As mentioned in the KIP, main motivation for this KIP is to reduce load on Kerberos server on large kafka deployments with large number of clients. Also it looks like we are combining two overlapping concepts 1. Single client sending requests with multiple users/authentications