[jira] [Commented] (KAFKA-4522) Using Disruptor instead of Array Blocking queue in Kafka Producer

2016-12-14 Thread Pratik kumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750643#comment-15750643 ] Pratik kumar commented on KAFKA-4522: - Actually I am using kafka v0.8.x , wherein the async

[jira] [Updated] (KAFKA-4535) http://kafka.apache.org/quickstart Step 8 missing argument

2016-12-14 Thread Xin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4535?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xin updated KAFKA-4535: --- Fix Version/s: (was: 0.10.1.1) > http://kafka.apache.org/quickstart Step 8 missing argument >

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

2016-12-14 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/2260 KAFKA-4529; LogCleaner should not delete the tombstone too early. cc @junrao You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka

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

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

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

2016-12-14 Thread ashishg-qburst
GitHub user ashishg-qburst opened a pull request: https://github.com/apache/kafka/pull/2259 MINOR: Fix typo on introduction page You can merge this pull request into a Git repository by running: $ git pull https://github.com/ashishg-qburst/kafka intro Alternatively you can

Re: Brokers cashing with OOME Map failed

2016-12-14 Thread Gwen Shapira
Did you recently add topics / partitions? Each partitions takes a memory buffer for replication, so you sometimes get OOME by adding partitions without sizing memory. You basically need the Java heapsize to be larger than # partitions on the broker X replica.fetch.size Gwen On Wed, Dec 14, 2016

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

2016-12-14 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4546: -- Summary: a consumer could miss tombstone when leader changes during the reads Key: KAFKA-4546 URL: https://issues.apache.org/jira/browse/KAFKA-4546 Project: Kafka

[jira] [Commented] (KAFKA-4545) tombstone needs to be removed after delete.retention.ms has passed after it has been cleaned

2016-12-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4545?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750154#comment-15750154 ] Jun Rao commented on KAFKA-4545: One potential way to fix this is when cleaning a segment after the dirty

[jira] [Created] (KAFKA-4545) tombstone needs to be removed after delete.retention.ms has passed after it has been cleaned

2016-12-14 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4545: -- Summary: tombstone needs to be removed after delete.retention.ms has passed after it has been cleaned Key: KAFKA-4545 URL: https://issues.apache.org/jira/browse/KAFKA-4545

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

2016-12-14 Thread Gwen Shapira
I'm a bit concerned about adding transformations in Kafka. NiFi has 150 processors, presumably they are all useful for someone. I don't know if I'd want all of that in Apache Kafka. What's the downside of keeping it out? Or at least keeping the built-in set super minimal (Flume has like 3 built-in

[jira] [Commented] (KAFKA-3135) Unexpected delay before fetch response transmission

2016-12-14 Thread Jeff Widman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750082#comment-15750082 ] Jeff Widman commented on KAFKA-3135: Can the tags on this issue be updated to note that it applies to

[jira] [Commented] (KAFKA-3856) Move inner classes accessible only functions in TopologyBuilder out of public APIs

2016-12-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3856?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750030#comment-15750030 ] Matthias J. Sax commented on KAFKA-3856: [~jeyhunkarimov] Are you still working on this? > Move

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

2016-12-14 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2258 MINOR: update KStream JavaDocs You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka minorKStreamJavaDoc Alternatively you can review and

[jira] [Created] (KAFKA-4542) Add authentication based on delegation token.

2016-12-14 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-4542: - Summary: Add authentication based on delegation token. Key: KAFKA-4542 URL: https://issues.apache.org/jira/browse/KAFKA-4542 Project: Kafka Issue Type:

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

2016-12-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749769#comment-15749769 ] Ashish K Singh commented on KAFKA-1696: --- [~omkreddy] I had to convert this JIRA from sub-task to a

[jira] [Updated] (KAFKA-4544) Add system tests for delegation token based authentication

2016-12-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4544?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-4544: -- Description: Add system tests for delegation token based authentication. (was: Add capability

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

2016-12-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4541?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh reassigned KAFKA-4541: - Assignee: Ashish K Singh > Add capability to create delegation token >

[jira] [Updated] (KAFKA-4544) Add system tests for delegation token based authentication

2016-12-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4544?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-4544: -- Summary: Add system tests for delegation token based authentication (was: Add capability to

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

2016-12-14 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-4543: - Summary: Add capability to renew/expire delegation tokens. Key: KAFKA-4543 URL: https://issues.apache.org/jira/browse/KAFKA-4543 Project: Kafka Issue

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

2016-12-14 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-4544: - Summary: Add capability to renew/expire delegation tokens. Key: KAFKA-4544 URL: https://issues.apache.org/jira/browse/KAFKA-4544 Project: Kafka Issue

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

2016-12-14 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Reenable streams smoke test -- [...truncated 17361 lines...] org.apache.kafka.streams.KafkaStreamsTest > shouldNotGetAllTasksWithStoreWhenNotRunning PASSED

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

2016-12-14 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-4541: - Summary: Add capability to create delegation token Key: KAFKA-4541 URL: https://issues.apache.org/jira/browse/KAFKA-4541 Project: Kafka Issue Type:

[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-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749750#comment-15749750 ] Ismael Juma commented on KAFKA-4477: One clarification: if you use the `log4j.properties` file that is

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

2016-12-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-1696: -- Issue Type: New Feature (was: Sub-task) Parent: (was: KAFKA-1682) > Kafka should

[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-14 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749722#comment-15749722 ] Apurva Mehta commented on KAFKA-4477: - Hi [~tdevoe] (and other as well), Would it be possible to

[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-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749636#comment-15749636 ] Matthias J. Sax commented on KAFKA-4540: This makes sense. The problem is that

[jira] [Commented] (KAFKA-1521) Producer Graceful Shutdown issue in Container (Kafka version 0.8.x.x)

2016-12-14 Thread Anish Khanzode (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749639#comment-15749639 ] Anish Khanzode commented on KAFKA-1521: --- This is really a problem when kafka consumer is used in a

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

2016-12-14 Thread Shikhar Bhushan
With regard to a), just using `ConnectRecord` with `newRecord` as a new abstract method would be a fine choice. In prototyping, both options end up looking pretty similar (in terms of how transformations are implemented and the runtime initializes and uses them) and I'm starting to lean towards

[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-14 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749549#comment-15749549 ] Damian Guy commented on KAFKA-4540: --- This issue isn't resolved. The problem here is that say i

[jira] [Work stopped] (KAFKA-4437) Incremental Batch Processing for Kafka Streams

2016-12-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4437?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4437 stopped by Matthias J. Sax. -- > Incremental Batch Processing for Kafka Streams >

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

2016-12-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4528?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749479#comment-15749479 ] Guozhang Wang commented on KAFKA-4528: -- Another observed failure:

[GitHub] kafka pull request #2257: MINOR: Reenable streams smoke test

2016-12-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2257 --- 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-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749465#comment-15749465 ] Guozhang Wang commented on KAFKA-4540: -- Is this already resolved as part of

[GitHub] kafka pull request #2257: MINOR: Reenable streams smoke test

2016-12-14 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2257 MINOR: Reenable streams smoke test I ran it 3 times and it works again. You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka

Brokers cashing with OOME Map failed

2016-12-14 Thread Zakee
Recently, we have seen our brokers crash with below errors, any idea what might be wrong here? The brokers have been running for long with the same hosts/configs without this issue before. Is this something to do with new version 0.10.0.1 (which we upgraded recently) or could it be a h/w

Re: kafka_2.11-0.9.0.1 crash with java coredump

2016-12-14 Thread Schumann,Robert
I would suggest trying a recent java version first, if I read about this one: http://mail.openjdk.java.net/pipermail/hotspot-gc-dev/2014-December/011534.html Cheers Robert -- Robert Schumann | Lead DevOps Engineer | mobile.de GmbH T: + 49. 30. 8109. 7219 M: +49.151. 5504. 8246 F: +49. 30. 8109.

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

2016-12-14 Thread Apurva Mehta
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 is not enough for only the control messages to be flushed before being acknowledged -- all the messages (and offset commits) which

[DISCUSS] Control Messages - [Was: KIP-82 - Add Record Headers]

2016-12-14 Thread Ignacio Solis
I'm renaming this thread in case we start deep diving. I'm in favor of so called "control messages", at least the notion of those. However, I'm not sure about the design. What I understood from the original mail: A. Provide a message that does not get returned by poll() B. Provide a way for

Jenkins build is back to normal : kafka-0.10.1-jdk7 #108

2016-12-14 Thread Apache Jenkins Server
See

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

2016-12-14 Thread Apurva Mehta
Hi Ben, You are are right on both counts: Writing apps to do consume-process-produce batching will be tricky to program using this API directly. The expectation is that 99% of the users would use the streams API to leverage this functionality, and that API will take care of the details. This

[jira] [Commented] (KAFKA-4431) HeartbeatThread should be a daemon thread

2016-12-14 Thread David Judd (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749174#comment-15749174 ] David Judd commented on KAFKA-4431: --- Awesome, thanks! > HeartbeatThread should be a daemon thread >

Re: kafka_2.11-0.9.0.1 crash with java coredump

2016-12-14 Thread Ismael Juma
Hi Mazhar, That looks like a G1 GC bug. Lots of bug in G1 have been fixed since JDK 7 update 51. I highly recommend that you upgrade to a more recent JDK (JDK 8 recommended). Alternatively you could try the CMS GC. Ismael On Wed, Dec 14, 2016 at 3:28 AM, Mazhar Shaikh

Re: kafka_2.11-0.9.0.1 crash with java coredump

2016-12-14 Thread Apurva Mehta
I would suggest creating a JIRA and describing in detail what was going on in the cluster when this happened, and posting the associated broker / state change / controller logs. Thanks, Apurva On Wed, Dec 14, 2016 at 3:28 AM, Mazhar Shaikh wrote: > Hi All, > > I am

[jira] [Commented] (KAFKA-4531) Rationalise client configuration validation

2016-12-14 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749120#comment-15749120 ] Vahid Hashemian commented on KAFKA-4531: [~ecomar] It would be nice to centralize all config

[jira] [Assigned] (KAFKA-4531) Rationalise client configuration validation

2016-12-14 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4531?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-4531: -- Assignee: Vahid Hashemian > Rationalise client configuration validation >

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Jun Rao
Hi, Onur, The reason for keeping track of the CZXID of the broker registration path is the following. There is one corner case bug (KAFKA-1120) that Ben mentioned where the controller could miss a ZK watcher event if the broker deregisters and registers quickly. Always triggering a leader

[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-14 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15749077#comment-15749077 ] Tom DeVoe commented on KAFKA-4477: -- Regarding 2 - I restarted the host which shrank all of it's ISRs at

[jira] [Assigned] (KAFKA-3714) Allow users greater access to register custom streams metrics

2016-12-14 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3714?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-3714: --- Assignee: Eno Thereska (was: Guozhang Wang) > Allow users greater access to register custom

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

2016-12-14 Thread Matthias J. Sax
Yes and no. I did overload the term "control message". EOS control messages are for client-broker communication and thus never exposed to any application. And I think this is a good design because broker needs to understand those control messages. Thus, this should be a protocol change. The type

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

2016-12-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748966#comment-15748966 ] Jun Rao commented on KAFKA-4529: Committed [~becket_qin]'s patch to 0.10.1 branch. Leaving the jira open

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

2016-12-14 Thread radai
arent control messages getting pushed as their own top level protocol change (and a fairly massive one) for the transactions KIP ? On Tue, Dec 13, 2016 at 5:54 PM, Matthias J. Sax wrote: > Hi, > > I want to add a completely new angle to this discussion. For this, I > want

Build failed in Jenkins: kafka-0.10.1-jdk7 #107

2016-12-14 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4431; Make consumer heartbeat thread a daemon thread -- [...truncated 14289 lines...] org.apache.kafka.streams.state.internals.CompositeReadOnlyKeyValueStoreTest >

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

2016-12-14 Thread Harsha Chintalapani
@Gwen @Mani Not sure why we want to authenticate at every request. Even if the token exchange is cheap it still a few calls that need to go through round trip. Impersonation doesn't require authentication for every request. "So a centralized app can create few producers, do the metadata request

[jira] [Created] (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-14 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4540: - Summary: Suspended tasks that are not assigned to the StreamThread need to be closed before new active and standby tasks are created Key: KAFKA-4540 URL:

Re: Kafka on Ubuntu Core and Snap store

2016-12-14 Thread Colin McCabe
Hi Michael, "snapcraft" sounds like an interesting idea. I'm curious what you see as the advantage over Docker images. Does the snap file contain the Java runtime? Kafka, like most JDK projects, already bundles its library dependencies internally, so I don't think snapcraft is adding quite as

[jira] [Commented] (KAFKA-4487) Tests should be run in Jenkins with INFO or DEBUG level

2016-12-14 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748788#comment-15748788 ] Colin P. McCabe commented on KAFKA-4487: Good idea. +1 for Jenkins running tests at a higher log

Re: [DISCUSS] KIP-83 - Allow multiple SASL PLAIN authenticated Java clients in a single JVM process

2016-12-14 Thread Edoardo Comar
Hi Ismael, I have retired the KIP and updated the PR. Can't wait for https://issues.apache.org/jira/browse/KAFKA-4259 to be merged :-) -- Edoardo Comar IBM MessageHub eco...@uk.ibm.com IBM UK Ltd, Hursley Park, SO21 2JN IBM United Kingdom Limited

[jira] [Commented] (KAFKA-4180) Shared authentication with multiple active Kafka producers/consumers

2016-12-14 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748758#comment-15748758 ] Edoardo Comar commented on KAFKA-4180: -- I have updated the PR based on the discussion in the mailing

[jira] [Commented] (KAFKA-4538) Version

2016-12-14 Thread Samuel Durand (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748743#comment-15748743 ] Samuel Durand commented on KAFKA-4538: -- thanks for the info [~ijuma] will follow that closely >

[jira] [Commented] (KAFKA-4474) Poor kafka-streams throughput

2016-12-14 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748741#comment-15748741 ] Eno Thereska commented on KAFKA-4474: - [~jjchorrobe] can I confirm again that this is all running in

New question / request on kafka consumer

2016-12-14 Thread Costache, Vlad
Hello, We are trying to make a consumer for kafka, (client code alone and camel integrated) and we ended in a blocking point. Can you please give us an advice, or any other idea? Our problem: - We create a kafka consumer that connects to a wrong server (wrong ip/port), and the

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

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

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

2016-12-14 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4539: -- Summary: StreamThread is not correctly creating StandbyTasks (was: NPE in StreamThread when creating

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

2016-12-14 Thread Ismael Juma
Thanks Rajini, that helps. A few comments: 1. The `AuthCallbackHandler` interface already exists and we are making breaking changes (removing a parameter from `configure` and adding additional methods). Is the reasoning that it was not a public interface before? It would be good to clearly

log.retention attribute not working

2016-12-14 Thread Ghosh, Achintya (Contractor)
Hi there, Any idea why log.retention attribute is not working? We kept log.retention.hours=6 in server.properties but we see old data are not getting deleted. We see Dec 9th data/log files are still there. We are running this in production boxes and if it does not delete the old files our

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

2016-12-14 Thread Rajini Sivaram
Have added sample callback handlers for PLAIN and SCRAM. On Tue, Dec 13, 2016 at 4:10 PM, Rajini Sivaram < rajinisiva...@googlemail.com> wrote: > Ismael, > > Thank you for the review. I will add an example. > > On Tue, Dec 13, 2016 at 1:07 PM, Ismael Juma wrote: > >> Hi

[jira] [Comment Edited] (KAFKA-4474) Poor kafka-streams throughput

2016-12-14 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748575#comment-15748575 ] Juan Chorro edited comment on KAFKA-4474 at 12/14/16 3:17 PM: -- Hi again! We

[jira] [Commented] (KAFKA-4538) Version

2016-12-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748574#comment-15748574 ] Ismael Juma commented on KAFKA-4538: I cherry-picked KAFKA-4431 into the 0.10.1 branch so it will be

[jira] [Commented] (KAFKA-4474) Poor kafka-streams throughput

2016-12-14 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748575#comment-15748575 ] Juan Chorro commented on KAFKA-4474: Hi again! We have been doing more performance tests and we have

[jira] [Commented] (KAFKA-4431) HeartbeatThread should be a daemon thread

2016-12-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748569#comment-15748569 ] Ismael Juma commented on KAFKA-4431: I cherry-picked this to the 0.10.1 branch as it's a trivial fix

[jira] [Updated] (KAFKA-4431) HeartbeatThread should be a daemon thread

2016-12-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4431: --- Fix Version/s: 0.10.1.1 > HeartbeatThread should be a daemon thread >

[jira] [Resolved] (KAFKA-4538) Version

2016-12-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4538. Resolution: Duplicate Duplicate of KAFKA-4431, please reopen if not. > Version > > >

[jira] [Commented] (KAFKA-4538) Version

2016-12-14 Thread Samuel Durand (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748422#comment-15748422 ] Samuel Durand commented on KAFKA-4538: -- [~rsivaram] it might , if I don't find the time to

[jira] [Comment Edited] (KAFKA-4538) Version

2016-12-14 Thread Samuel Durand (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748422#comment-15748422 ] Samuel Durand edited comment on KAFKA-4538 at 12/14/16 2:13 PM:

Re: [DISCUSS] KIP-83 - Allow multiple SASL PLAIN authenticated Java clients in a single JVM process

2016-12-14 Thread Ismael Juma
Hi Edoardo, Yes, you're right, this can be just a JIRA now as there are no publicly facing changes. Thanks, Ismael On Tue, Dec 13, 2016 at 9:07 AM, Edoardo Comar wrote: > Thanks for your review, Ismael. > > First, I am no longer sure KIP-83 is worth keeping as KIP, I

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

2016-12-14 Thread Ismael Juma
Hi Rajini, I think it would definitely be valuable to have a KIP for impersonation. Ismael On Wed, Dec 14, 2016 at 4:03 AM, Rajini Sivaram wrote: > It would clearly be very useful to enable clients to send requests on > behalf of multiple users. A separate KIP makes

[GitHub] kafka pull request #2256: KAFKA-4534: StreamPartitionAssignor only ever upda...

2016-12-14 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2256 KAFKA-4534: StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics on first assignment partitionsByHostState and metadataWithInternalTopics need to be

[jira] [Commented] (KAFKA-4534) StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics once.

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

[jira] [Commented] (KAFKA-4534) StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics once.

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

[GitHub] kafka pull request #2253: KAFKA-4534: StreamPartitionAssignor only ever upda...

2016-12-14 Thread dguy
Github user dguy closed the pull request at: https://github.com/apache/kafka/pull/2253 --- 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-4536) Kafka clients throw NullPointerException on poll when delete the relative topic

2016-12-14 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748359#comment-15748359 ] Rajini Sivaram commented on KAFKA-4536: --- This is fixed in 0.10.1.0. > Kafka clients throw

[jira] [Commented] (KAFKA-4538) Version

2016-12-14 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748324#comment-15748324 ] Rajini Sivaram commented on KAFKA-4538: --- If you are using KafkaConsumer, this could be due to

[jira] [Updated] (KAFKA-4539) NPE in StreamThread when creating StandbyTasks

2016-12-14 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4539: -- Status: Patch Available (was: In Progress) > NPE in StreamThread when creating StandbyTasks >

[jira] [Commented] (KAFKA-4539) NPE in StreamThread when creating StandbyTasks

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

[GitHub] kafka pull request #2255: KAFKA-4539: NPE in StreamThread when creating Stan...

2016-12-14 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2255 KAFKA-4539: NPE in StreamThread when creating StandbyTasks Tasks that don't have any `StateStore`s wont have a `StandbyTask`, so `createStandbyTask` can return `null`. We need to check for this in

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

2016-12-14 Thread Rajini Sivaram
Edo, I wouldn't introduce a new config entry, especially since you don't need it after KAFKA-4137. As a temporary measure that would work for consumers. But you probably don't want to do the same for brokers - will be worth checking with Radai since the implementation will be based on KIP-72. To

[jira] [Work started] (KAFKA-4539) NPE in StreamThread when creating StandbyTasks

2016-12-14 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4539 started by Damian Guy. - > NPE in StreamThread when creating StandbyTasks >

[jira] [Created] (KAFKA-4539) NPE in StreamThread when creating StandbyTasks

2016-12-14 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4539: - Summary: NPE in StreamThread when creating StandbyTasks Key: KAFKA-4539 URL: https://issues.apache.org/jira/browse/KAFKA-4539 Project: Kafka Issue Type: Bug

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

2016-12-14 Thread Rajini Sivaram
It would clearly be very useful to enable clients to send requests on behalf of multiple users. A separate KIP makes sense, but it may be worth thinking through some of the implications now, especially if the main interest in delegation tokens comes from its potential to enable impersonation. I

[jira] [Commented] (KAFKA-4522) Using Disruptor instead of Array Blocking queue in Kafka Producer

2016-12-14 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748169#comment-15748169 ] Ben Stopford commented on KAFKA-4522: - It'd be good to have some empirical evidence that this was

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

2016-12-14 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2254 KAFKA-4537: StreamPartitionAssignor incorrectly adds standby partitions to the partitionsByHostState map If a KafkaStreams app is using Standby Tasks then `StreamPartitionAssignor` will add the

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

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

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Ben Stopford
To clarify slightly, the case described in the KIP doesn’t necessitate an extra mapping to the CZXID. But there is an issue filed against the controller, which would also affect the LeaderGeneration correctness. The suggested fix for this includes such a mapping, according to Jun’s reasoning in

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Ben Stopford
Thanks Onur. That’s correct, we no longer nee that extra mapping. I’ll update the KIP. B Ben Stopford Confluent, http://www.confluent.io > On 11 Dec 2016, at 23:54, Onur Karaman wrote: > > Pretty happy to see a KIP tackling this

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

2016-12-14 Thread Edoardo Comar
Thanks Rajini, Before Kafka-4137, we could avoid coordinator starvation without making a special case for a special connection, but rather simply, in applying the buffer.memory check only to 'large' responses (e.g. size > 1k, possibly introducing a new config entry) in

[jira] [Created] (KAFKA-4538) Version

2016-12-14 Thread Samuel Durand (JIRA)
Samuel Durand created KAFKA-4538: Summary: Version Key: KAFKA-4538 URL: https://issues.apache.org/jira/browse/KAFKA-4538 Project: Kafka Issue Type: Bug Components: clients

kafka_2.11-0.9.0.1 crash with java coredump

2016-12-14 Thread Mazhar Shaikh
Hi All, I am using kafka_2.11-0.9.0.1 with java version "1.7.0_51". On random days kafka process stops (crashes) with a java coredump file as below. (gdb) bt #0 0x7f33059f70d5 in raise () from /lib/x86_64-linux-gnu/libc.so.6 #1 0x7f33059fa83b in abort () from

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-14 Thread Ben Stopford
Thanks Neha. It’s a fair point. We don’t need two anymore. There isn’t much to be gained from changing the existing nomenclature, unless people feel strongly about it the word epoch. LeaderEpoch is currently fairly ingrained (spans Zookeeper, the controller, the metadata requests etc). I

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

2016-12-14 Thread mayi_hetu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15748056#comment-15748056 ] mayi_hetu commented on KAFKA-4536: -- As I followed the code, I found there is a MetadataListener in the

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

2016-12-14 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4537: - Summary: StreamPartitionAssignor incorrectly adds standby partitions to the partitionsByHostState map Key: KAFKA-4537 URL: https://issues.apache.org/jira/browse/KAFKA-4537

  1   2   >