[jira] [Commented] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234514#comment-15234514 ] ASF GitHub Bot commented on KAFKA-3160: --- GitHub user dpkp opened a pull request:

[GitHub] kafka pull request: KAFKA-3160: Fix LZ4 Framing

2016-04-10 Thread dpkp
GitHub user dpkp opened a pull request: https://github.com/apache/kafka/pull/1212 KAFKA-3160: Fix LZ4 Framing This contribution is my original work and I license the work under Apache 2.0. You can merge this pull request into a Git repository by running: $ git pull

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-10 Thread Jun Rao
Hi, Jiangjie, Thanks for the update. Looks good to me overall. Just a few minor comments below. 10. On broker startup, it's not clear to me why we need to scan the log segment to retrieve the largest timestamp since the time index always has an entry for the largest timestamp. Is that only for

"nag" for Pull Request

2016-04-10 Thread Eric Wasserman
Following the advice on http://kafka.apache.org/contributing.html > this email is a "nag" for the PR https://github.com/apache/kafka/pull/1168

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

2016-04-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3338: Add print and writeAsText to KStream/KTable in Kafka Streams -- [...truncated 1590 lines...] kafka.log.LogTest > testTruncateTo PASSED kafka.log.LogTest >

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

2016-04-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3338: Add print and writeAsText to KStream/KTable in Kafka Streams -- [...truncated 1628 lines...] kafka.log.LogTest > testTruncateTo PASSED kafka.log.LogTest >

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-10 Thread Jun Rao
Thinking about ApiVersionRequest a bit more. There are quite a few things special about it. In the ideal case, (1) its version should never change; (2) it needs to be done before authentication (either SSL/SASL); (3) it is required to be issued at the beginning of each connection but never needs

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jun Rao
Ismael, My responses are inlined below. On Sun, Apr 10, 2016 at 12:25 PM, Ismael Juma wrote: > Hi Jun, > > A couple of points below. > > On Sat, Apr 9, 2016 at 12:19 AM, Jun Rao wrote: > > > 5. Your main request is how can a client know that the broker is

[jira] [Resolved] (KAFKA-3338) Add print and writeAsText functions to the Streams DSL

2016-04-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3338. -- Resolution: Fixed Fix Version/s: (was: 0.10.1.0) 0.10.0.0 Issue

[GitHub] kafka pull request: KAFKA-3338 [Kafka Streams] : Add print and wri...

2016-04-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1209 --- 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-3338) Add print and writeAsText functions to the Streams DSL

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

[jira] [Commented] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234370#comment-15234370 ] Guozhang Wang commented on KAFKA-3511: -- I like [~jkreps] suggestion about giving the aggregators

Re: KStream Close Processor

2016-04-10 Thread Guozhang Wang
Re 1), Kafka Streams intentionally close all underlying clients before closing processors since some of closing the processors require shutting down its processor state managers, for example we need to make sure producer's message sends // have all been acked before the state manager records //

[jira] [Commented] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234312#comment-15234312 ] Guozhang Wang commented on KAFKA-3160: -- One thing with serving old format consume request is that now

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Ismael Juma
Hi Jun, A couple of points below. On Sat, Apr 9, 2016 at 12:19 AM, Jun Rao wrote: > 5. Your main request is how can a client know that the broker is now > supporting new SASL mechanisms. One way to support that is to adjust KIP-43 > slightly. We can model the

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-10 Thread Jun Rao
Ashish, A 3rd option is to in 0.10.0, just sanity check the principal type in the implementation of addAcls/removeAcls of Authorizer, but don't change the Authorizer api to add the getDescription() method. This fixes the immediate issue that an acl rule with the wrong principal type is silently

[jira] [Commented] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-10 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234245#comment-15234245 ] Magnus Edenhill commented on KAFKA-3160: [~dana.powers] My broker patch adds a new Attribute bit

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jun Rao
Magnus proposal versions each SASL token. Perhaps you had something different in mind? I agree that we want to make the part that Kafka controls in SASL extensible. For now, the mechanism part is the only thing that we had in mind. Is there anything else you think would be useful? We could rename

[jira] [Commented] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-10 Thread Dana Powers (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234221#comment-15234221 ] Dana Powers commented on KAFKA-3160: Magnus: have you made any progress on this? The more I think

Re: [VOTE] KIP-4 Metadata Schema

2016-04-10 Thread Jun Rao
Grant, The limitation with the current MetadataResponse is that if a broker is down, all replicas on that broker will be missing in the assigned replica list in the response. Now, imagine that you want to use MetadataRequest to do a describe of a topic, it's weird that you don't see the full

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jay Kreps
I don't think we are versioning their bytes but providing a framework for any additional data needed (in this case the mechanism at the least is needed but it could be more), right? I think this is what the authenticate api in zookeeper does, right? -Jay On Sun, Apr 10, 2016 at 10:08 AM, Jun Rao

[jira] [Updated] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-10 Thread Dana Powers (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dana Powers updated KAFKA-3160: --- Description: KAFKA-1493 partially implements the LZ4 framing specification, but it incorrectly

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jun Rao
The way that we want to extend SASL mechanism is to first add the mechanism exchange part. Once the mechanism is selected, we will exchange the SASL tokens specific to the selected mechanism. Magnus proposes to model both the mechanism exchange and token exchange as independent Kafka