[jira] [Commented] (KAFKA-4126) No relevant log when the topic is non-existent

2016-09-05 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15466496#comment-15466496 ] Manikumar Reddy commented on KAFKA-4126: Currently producer logs below warn message. May be we

Re: WARN log message flooding broker logs for a pretty typical SSL setup

2016-09-05 Thread Jaikiran Pai
Thank you. Just opened one https://github.com/apache/kafka/pull/1825 -Jaikiran On Tuesday 06 September 2016 10:12 AM, Manikumar Reddy wrote: We don't need JIRA for minor PRs. Just prefix "MINOR:" to PR title. On Tue, Sep 6, 2016 at 9:16 AM, Jaikiran Pai wrote:

[GitHub] kafka pull request #1825: MINOR: Reduce the log level when the peer isn't au...

2016-09-05 Thread jaikiran
GitHub user jaikiran opened a pull request: https://github.com/apache/kafka/pull/1825 MINOR: Reduce the log level when the peer isn't authenticated but is using SSL The commit here changes the log level of a log message from WARN to DEBUG. As noted in the mail discussion here

Re: WARN log message flooding broker logs for a pretty typical SSL setup

2016-09-05 Thread Manikumar Reddy
We don't need JIRA for minor PRs. Just prefix "MINOR:" to PR title. On Tue, Sep 6, 2016 at 9:16 AM, Jaikiran Pai wrote: > Thanks Ismael, I'll raise a PR for this. As a process, is there a JIRA > that's expected to be filed for this before I raise a PR or would this be

Re: Plans to improve SSL performance in Kafka, for 0.10.x?

2016-09-05 Thread Jaikiran Pai
On Tuesday 06 September 2016 09:42 AM, Jaikiran Pai wrote: Hi Todd, Note that this relies on the fix in KAFKA-4050 and switching the PRNG to SHA1PRNG. Thanks, I hadn't noticed that JIRA. I'll make sure we use it in our setup. However, right now, (intentionally) we have single broker within

Re: Plans to improve SSL performance in Kafka, for 0.10.x?

2016-09-05 Thread Jaikiran Pai
Hi Tom, On Monday 05 September 2016 06:32 PM, Tom Crayford wrote: We've been using SSL quite successfully on a few thousand production clusters, and have done heavy performance testing, and as far as we can see, it doesn't impact the cluster's capacity. The only case I could see that happening

Re: Plans to improve SSL performance in Kafka, for 0.10.x?

2016-09-05 Thread Jaikiran Pai
Hi Todd, On Sunday 04 September 2016 10:58 PM, Todd Palino wrote: We've been using SSL for produce traffic (mirror makers only right now, but that's a very large percentage of traffic for us), and we're in the process of turning it on for inter broker traffic as well. Our experience is that

Re: WARN log message flooding broker logs for a pretty typical SSL setup

2016-09-05 Thread Jaikiran Pai
Thanks Ismael, I'll raise a PR for this. As a process, is there a JIRA that's expected to be filed for this before I raise a PR or would this be OK without a JIRA? -Jaikiran On Monday 05 September 2016 03:55 PM, Ismael Juma wrote: Hi Jaikiran, I agree that this is a valid configuration and

[jira] [Updated] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work

2016-09-05 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-4116: - Assignee: Yuto Kawamura (was: Gwen Shapira) > Specifying 0.0.0.0 in "listeners" doesn't work >

[jira] [Assigned] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work

2016-09-05 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura reassigned KAFKA-4116: Assignee: Gwen Shapira (was: Yuto Kawamura) > Specifying 0.0.0.0 in "listeners" doesn't

Re: [DISCUSS] KIP-78: Cluster Id

2016-09-05 Thread Ismael Juma
Hi Dong, Thanks for your reply. See inline. On Mon, Sep 5, 2016 at 11:28 PM, Dong Lin wrote: > Yes, user has the option to manually set the cluster.id by directly > setting > the znode. But the KIP doesn't provide script for doing this. That is correct. And

Re: [DISCUSS] KIP-78: Cluster Id

2016-09-05 Thread Dong Lin
Hey Ismael, Yes, user has the option to manually set the cluster.id by directly setting the znode. But the KIP doesn't provide script for doing this. Unlike reading cluster.id from broker, this approach doesn't allow you to persist the cluster.id in case of cluster migration and znode deletion.

Re: [DISCUSS] KIP-78: Cluster Id

2016-09-05 Thread Ismael Juma
Hi Dong, A few clarifications below. On Mon, Sep 5, 2016 at 7:21 PM, Dong Lin wrote: > > I think you are saying that we can stop our discussion and follow simply > take a vote where the majority decides. I don't think this is a good way to > find the best design for a KIP

Re: PartitionAssignor / Sort members per subscription time before assigning partitions

2016-09-05 Thread Florian Hussonnois
Hi Jason, The downside to use client-ids is there is no certitude that they will be increasing. So in case there is already as many consumers as partitions, additional consumers can change the partitions assignment. This lead to rebalances which are unnecessary as partitions are already

Re: Queryable state client read guarantees

2016-09-05 Thread Mikael Högqvist
Hi Damian, > > Failed to read key hello, org.mkhq.kafka.Topology$StoreUnavailable > > > Failed to read key hello, org.mkhq.kafka.Topology$KeyNotFound > > > hello -> 10 > > > > > The case where you get KeyNotFound looks like a bug to me. This shouldn't > happen. I can see why it might happen and

[jira] [Created] (KAFKA-4127) Possible data loss

2016-09-05 Thread Ramnatthan Alagappan (JIRA)
Ramnatthan Alagappan created KAFKA-4127: --- Summary: Possible data loss Key: KAFKA-4127 URL: https://issues.apache.org/jira/browse/KAFKA-4127 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3408) consumer rebalance fail

2016-09-05 Thread Matt Navarifar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15465537#comment-15465537 ] Matt Navarifar commented on KAFKA-3408: --- Hi [~ewencp], hate to annoy you further, but could I also

Re: [DISCUSS] KIP-78: Cluster Id

2016-09-05 Thread Dong Lin
Ismael, I think you are saying that we can stop our discussion and follow simply take a vote where the majority decides. I don't think this is a good way to find the best design for a KIP and the discussion seems to be useless. It doesn't seem like anyone else is interested to join this

[jira] [Comment Edited] (KAFKA-4120) byte[] keys in RocksDB state stores do not work as expected

2016-09-05 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15464171#comment-15464171 ] Greg Fodor edited comment on KAFKA-4120 at 9/5/16 5:34 PM: --- We were able to work

Re: SASL session expiry

2016-09-05 Thread Mickael Maison
Indeed On Mon, Sep 5, 2016 at 5:43 PM, Ismael Juma wrote: > On Mon, Sep 5, 2016 at 3:48 PM, Mickael Maison > wrote: > >> Yes in our case, I can see how we would add the functionality, but I >> was wondering if people might be interested to directly

Re: SASL session expiry

2016-09-05 Thread Ismael Juma
On Mon, Sep 5, 2016 at 3:48 PM, Mickael Maison wrote: > Yes in our case, I can see how we would add the functionality, but I > was wondering if people might be interested to directly have such a > feature in Kafka. At the moment, the authorization logic is only >

[jira] [Updated] (KAFKA-4123) Queryable State returning null for key before all stores in instance have been initialized

2016-09-05 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4123: -- Summary: Queryable State returning null for key before all stores in instance have been initialized

[jira] [Updated] (KAFKA-4123) Queryable State returning null for key before all stores in instance have been intialized

2016-09-05 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4123: -- Description: A couple of problems: 1. A RocksDBStore instance is currently marked as open before the

[jira] [Updated] (KAFKA-4123) Queryable State returning null for key before all stores in instance have been intialized

2016-09-05 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4123: -- Summary: Queryable State returning null for key before all stores in instance have been intialized

[jira] [Created] (KAFKA-4126) No relevant log when the topic is non-existent

2016-09-05 Thread JIRA
Balázs Barnabás created KAFKA-4126: -- Summary: No relevant log when the topic is non-existent Key: KAFKA-4126 URL: https://issues.apache.org/jira/browse/KAFKA-4126 Project: Kafka Issue Type:

WARN log message flooding broker logs for a pretty typical SSL setup

2016-09-05 Thread Jaikiran Pai
We just started enabling SSL for our Kafka brokers and (Java) clients and among some of the issues we are running into, one of them is the flooding of the server/broker Kafka logs where we are seeing these messages: [2016-09-02 08:07:13,773] WARN SSL peer is not authenticated, returning

Re: SASL session expiry

2016-09-05 Thread Mickael Maison
Hi Rajini, Yes in our case, I can see how we would add the functionality, but I was wondering if people might be interested to directly have such a feature in Kafka. At the moment, the authorization logic is only invoked for new SASL (and apparently Kerberos) connections. I feel like having the

[jira] [Commented] (KAFKA-4123) RocksDBStore can be marked as open before it has been initialized

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

[GitHub] kafka pull request #1824: KAFKA-4123: RocksDBStore can be marked as open bef...

2016-09-05 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/1824 KAFKA-4123: RocksDBStore can be marked as open before it has been initialized Mark the store as open after the DB has been restored from the changelog. Only add the store to the map in

Re: Plans to improve SSL performance in Kafka, for 0.10.x?

2016-09-05 Thread Tom Crayford
We've been using SSL quite successfully on a few thousand production clusters, and have done heavy performance testing, and as far as we can see, it doesn't impact the cluster's capacity. The only case I could see that happening is if you have a *lot* of connection churn, but that isn't very

[jira] [Created] (KAFKA-4125) Provide low-level Processor API meta data in DSL layer

2016-09-05 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4125: -- Summary: Provide low-level Processor API meta data in DSL layer Key: KAFKA-4125 URL: https://issues.apache.org/jira/browse/KAFKA-4125 Project: Kafka

Re: SASL session expiry

2016-09-05 Thread Ismael Juma
Hi Rajini, It's a good question and it depends on a number of details. For example, for short-lived certificates with long-lived connections, it would seem that one would have to duplicate some logic performed by the TLS stack on the Authorizer, which is not ideal. For the case where the

Re: SASL session expiry

2016-09-05 Thread Rajini Sivaram
Mickael, I imagine it is fairly easy in MessageHub to deal with expired SASL/PLAIN credentials since checks can be added to the interceptor in the broker. Ismael, Is it really feasible in general to deal with expired credentials in Authorizers? It sort of expects tight coupling between

[jira] [Commented] (KAFKA-3779) Add the LRU cache for KTable.to() operator

2016-09-05 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15464816#comment-15464816 ] Eno Thereska commented on KAFKA-3779: - This might not make sense, since a KTable will already contain

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-09-05 Thread Eno Thereska
Hi Matthias, The motivation for KIP-63 was primarily aggregates and reducing the load on "both" state stores and downstream. I think there is agreement that for the DSL the motivation and design make sense. For the Processor API: caching is a major component in any system, and it is difficult

[jira] [Created] (KAFKA-4124) Handle disk failures gracefully

2016-09-05 Thread Gokul (JIRA)
Gokul created KAFKA-4124: Summary: Handle disk failures gracefully Key: KAFKA-4124 URL: https://issues.apache.org/jira/browse/KAFKA-4124 Project: Kafka Issue Type: Improvement Reporter:

[jira] [Created] (KAFKA-4123) RocksDBStore can be marked as open before it has been initialized

2016-09-05 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4123: - Summary: RocksDBStore can be marked as open before it has been initialized Key: KAFKA-4123 URL: https://issues.apache.org/jira/browse/KAFKA-4123 Project: Kafka

Re: Queryable state client read guarantees

2016-09-05 Thread Damian Guy
Hi Mikael, > > Failed to read key hello, org.mkhq.kafka.Topology$StoreUnavailable > > Failed to read key hello, org.mkhq.kafka.Topology$KeyNotFound > > hello -> 10 > > The case where you get KeyNotFound looks like a bug to me. This shouldn't happen. I can see why it might happen and we will

Re: WARN log message flooding broker logs for a pretty typical SSL setup

2016-09-05 Thread Ismael Juma
Hi Jaikiran, I agree that this is a valid configuration and the log level seems too high given that. The original motivation is explained in the PR: https://github.com/apache/kafka/pull/155/files#diff-fce430ae21a0c98d82da6d4aa551f824L603 That is, help people figure out if client authentication

Re: SASL session expiry

2016-09-05 Thread Ismael Juma
Hi Mickael, The Kerberos ticket refresh mechanism is there for new connections, not existing connections. Currently, the suggested approach is to rely on the authorizer to deal with expired credentials. Would this work for you? Ismael On Mon, Sep 5, 2016 at 11:13 AM, Mickael Maison

SASL session expiry

2016-09-05 Thread Mickael Maison
Hi, While Kerberos has a mechanism to refresh its tickets, SASL PLAIN has no such feature. This means if a client is connected, as far as I can tell, we have currently no way of disconnecting him, revoking his credentials won't help. I think it would be useful to have a way to force clients to

Re: [DISCUSS] KIP-78: Cluster Id

2016-09-05 Thread Ismael Juma
Dong, Sumit responded to a number of points already, so I will try to be brief. See inline. Also, it may just be possible that we won't reach agreement. In that case, a vote may be a way to figure out if people feel that this proposal adds value in its current form or not. On Mon, Sep 5, 2016

Re: [DISCUSS] KIP-78: Cluster Id

2016-09-05 Thread Dong Lin
Hey Sumit, Thanks for your detailed response. Please see my comment inline. On Sun, Sep 4, 2016 at 10:56 PM, sumit arrawatia wrote: > Hi Dong, > > Please find my answers inline. > > Hopefully they address your concerns this time ! > > Sumit > > On Sun, Sep 4, 2016 at

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

2016-09-05 Thread Apache Jenkins Server
See

Re: Question regarding Producer and Duplicates

2016-09-05 Thread Florian Hussonnois
Thanks Guozhang, So increasing batch.size can lead to more duplicates in case of failure. Also when you said : "The broker will accept a batch of records as a whole or reject them ". For example, if a producer request contains two batches, a first one for Topic A / Partition 0 and a second one

[GitHub] kafka pull request #1823: Fixes javadoc of Windows, fixes typo in parameter ...

2016-09-05 Thread miguno
GitHub user miguno opened a pull request: https://github.com/apache/kafka/pull/1823 Fixes javadoc of Windows, fixes typo in parameter name of KGroupedTable You can merge this pull request into a Git repository by running: $ git pull https://github.com/miguno/kafka

[GitHub] kafka pull request #1822: Fix javadocs of Windowed

2016-09-05 Thread miguno
Github user miguno closed the pull request at: https://github.com/apache/kafka/pull/1822 --- 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 pull request #1822: Fix javadocs of Windowed

2016-09-05 Thread miguno
GitHub user miguno opened a pull request: https://github.com/apache/kafka/pull/1822 Fix javadocs of Windowed Previous Javadoc was referring to a 0.10.0.x method that was since removed from trunk. You can merge this pull request into a Git repository by running: $ git pull

[jira] [Commented] (KAFKA-4119) Get topic offset with Kafka SSL

2016-09-05 Thread zhang shuai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15464184#comment-15464184 ] zhang shuai commented on KAFKA-4119: Nobody get this problem ? > Get topic offset with Kafka SSL >