[jira] [Commented] (KAFKA-3736) Add http metrics reporter

2016-05-23 Thread Adrian Muraru (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3736?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296063#comment-15296063 ] Adrian Muraru commented on KAFKA-3736: -- Ok, just noticed that "Monitoring" is conside

[jira] [Commented] (KAFKA-3736) Add http metrics reporter

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3736?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296075#comment-15296075 ] Ismael Juma commented on KAFKA-3736: [~amuraru], what is your Apache Wiki id? I can gi

[jira] [Commented] (KAFKA-3647) Unable to set a ssl provider

2016-05-23 Thread Johan Abbors (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296098#comment-15296098 ] Johan Abbors commented on KAFKA-3647: - Yeah, in my opinion this is enough. The ssl.pr

[jira] [Commented] (KAFKA-3567) Add --security-protocol option to console consumer and producer

2016-05-23 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296163#comment-15296163 ] Edoardo Comar commented on KAFKA-3567: -- Hi, the console producer has these options:

[jira] [Commented] (KAFKA-3726) Enable cold storage option

2016-05-23 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296178#comment-15296178 ] Ben Stopford commented on KAFKA-3726: - Ah, OK. Yes I quite like this idea, particularl

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Ismael Juma
+1 to Jun's suggestion. Having said that, as a general point, I think we should consider supporting topic patterns in the wire protocol. It requires some thinking for cross-language support, but it seems surmountable and it could make certain operations a lot more efficient (the fact that a basic

Re: KAFKA-3722 : Discussion about custom PrincipalBuilder and Authorizer configs

2016-05-23 Thread Ismael Juma
Hi Mayuresh and Harsha, If we were doing this from scratch, I would prefer option 4 too. However, users have their own custom principal builders now and option 2 with a suitably updated javadoc is the way to go in my opinion. Ismael On Sat, May 21, 2016 at 2:28 AM, Harsha wrote: > Mayuresh, >

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-05-23 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296394#comment-15296394 ] Edoardo Comar commented on KAFKA-3727: -- To explain better, this is a stack trace when

[jira] [Issue Comment Deleted] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-05-23 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar updated KAFKA-3727: - Comment: was deleted (was: To explain better, this is a stack trace when the consumer is stuck in

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-05-23 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296398#comment-15296398 ] Edoardo Comar commented on KAFKA-3727: -- The consumer remains stuck in a loop because

[jira] [Commented] (KAFKA-3726) Enable cold storage option

2016-05-23 Thread Radoslaw Gruchalski (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296495#comment-15296495 ] Radoslaw Gruchalski commented on KAFKA-3726: bq. So my interpretation of your

[jira] [Created] (KAFKA-3746) InvalidReceiveException when connecting to broker over SSL

2016-05-23 Thread Sergey Alaev (JIRA)
Sergey Alaev created KAFKA-3746: --- Summary: InvalidReceiveException when connecting to broker over SSL Key: KAFKA-3746 URL: https://issues.apache.org/jira/browse/KAFKA-3746 Project: Kafka Issue

[jira] [Commented] (KAFKA-3746) InvalidReceiveException when connecting to broker over SSL

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3746?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296514#comment-15296514 ] Ismael Juma commented on KAFKA-3746: In your server configuration you have: l

[jira] [Commented] (KAFKA-3746) InvalidReceiveException when connecting to broker over SSL

2016-05-23 Thread Sergey Alaev (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3746?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296517#comment-15296517 ] Sergey Alaev commented on KAFKA-3746: - Ah. Yes. Sorry for being stupid. Closing it now

[jira] [Resolved] (KAFKA-3746) InvalidReceiveException when connecting to broker over SSL

2016-05-23 Thread Sergey Alaev (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sergey Alaev resolved KAFKA-3746. - Resolution: Not A Problem Configuration issue - attempting to establish SSL connection to plaintex

[jira] [Commented] (KAFKA-3727) Consumer.poll() stuck in loop on non-existent topic manually assigned

2016-05-23 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296538#comment-15296538 ] Edoardo Comar commented on KAFKA-3727: -- My expectation is that with either subscribe(

[jira] [Updated] (KAFKA-3664) When subscription set changes on new consumer, the partitions may be removed without offset being committed.

2016-05-23 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3664?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3664: --- Status: Patch Available (was: Open) > When subscription set changes on new consumer, the part

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Guozhang Wang
The original concern is that regex may not be efficiently supported across-languages, but if there is a neat workaround I would love to learn. Guozhang On Mon, May 23, 2016 at 5:31 AM, Ismael Juma wrote: > +1 to Jun's suggestion. > > Having said that, as a general point, I think we should consi

Kafka KIP meeting May 24 at 11:00am PST

2016-05-23 Thread Jun Rao
Hi, Everyone, Now that Kafka 0.10.0.0 is released, we will have a Kafka KIP meeting tomorrow at 11:00am PST. If you plan to attend but haven't received an invite, please let me know. The following is the agenda. Agenda: KIP-48 - Delegation tokens KIP-58 - Make Log Compaction Point Configurable K

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

2016-05-23 Thread Jun Rao
Hi, Harsha, Just sent out a KIP meeting invite. We can discuss this in the meeting tomorrow. Thanks, Jun On Thu, May 19, 2016 at 8:47 AM, Harsha wrote: > Hi All, >Can we have a KIP meeting around this. The KIP is up for >sometime and if there are any questions lets qui

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296597#comment-15296597 ] Guozhang Wang commented on KAFKA-3745: -- Thanks for reporting this [~gfodor]. > Consi

[jira] [Updated] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3745: - Labels: api newbie (was: api) > Consider adding join key to ValueJoiner interface > -

[jira] [Updated] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3745: - Labels: api (was: ) > Consider adding join key to ValueJoiner interface > ---

[jira] [Updated] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3745: - Assignee: (was: Guozhang Wang) > Consider adding join key to ValueJoiner interface > -

Re: [DISCUSS] KIP-55: Secure quotas for authenticated users

2016-05-23 Thread Jun Rao
Rajini, Thanks for the KIP. When we first added the quota support, the intention was to be able to add a quota per application. Since at that time, we don't have security yet. We essentially simulated users with client-ids. Now that we do have security. It seems that we just need to have a way to

[jira] [Created] (KAFKA-3747) Close `RecordBatch.records` when append to batch fails

2016-05-23 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-3747: -- Summary: Close `RecordBatch.records` when append to batch fails Key: KAFKA-3747 URL: https://issues.apache.org/jira/browse/KAFKA-3747 Project: Kafka Issue Type:

[jira] [Updated] (KAFKA-3704) Improve mechanism for compression stream block size selection in KafkaProducer

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3704?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3704: --- Description: As discovered in https://issues.apache.org/jira/browse/KAFKA-3565, the current default b

[jira] [Updated] (KAFKA-3704) Improve mechanism for compression stream block size selection in KafkaProducer

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3704?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3704: --- Description: As discovered in https://issues.apache.org/jira/browse/KAFKA-3565, the current default b

[jira] [Commented] (KAFKA-3704) Improve mechanism for compression stream block size selection in KafkaProducer

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3704?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296616#comment-15296616 ] Ismael Juma commented on KAFKA-3704: I filed KAFKA-3747 for 3). > Improve mechanism

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Becket Qin
+1 on Jun's idea. Even without the new consumer, currently we may still face this issue of record size too large in offset topic if user commits offsets with a big blob of metadata. Topic pattern would help reduce the group metadata size. However some use cases may not be able to benefit from it.

[jira] [Assigned] (KAFKA-3747) Close `RecordBatch.records` when append to batch fails

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-3747: -- Assignee: Ismael Juma > Close `RecordBatch.records` when append to batch fails > --

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Jay Kreps
I think the concern was just that we didn't want to do java regex for non-java clients, but I do think there are perl regex libraries (which is kind of more the standard) for java. So that might be a solution. -Jay On Mon, May 23, 2016 at 9:17 AM, Guozhang Wang wrote: > The original concern is

[GitHub] kafka pull request: KAFKA-3747; Close `RecordBatch.records` when a...

2016-05-23 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1418 KAFKA-3747; Close `RecordBatch.records` when append to batch fails With this change, `test_producer_throughput` with message_size=1, compression_type=snappy and a snappy buffer size of 32k can be

[jira] [Commented] (KAFKA-3747) Close `RecordBatch.records` when append to batch fails

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

[jira] [Updated] (KAFKA-3747) Close `RecordBatch.records` when append to batch fails

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3747: --- Status: Patch Available (was: Open) > Close `RecordBatch.records` when append to batch fails > --

Re: KAFKA-3722 : Discussion about custom PrincipalBuilder and Authorizer configs

2016-05-23 Thread Mayuresh Gharat
Hi Harsha and Ismael, Option 2 sounds like a good idea if we want to make this quick fix I think. Option 4 might require a KIP as its public interface change. I can resubmit a patch for option 2 or create a KIP if necessary for option 4. >From the previous conversation here, I think Ismael prefer

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

2016-05-23 Thread Jun Rao
Thanks for the KIP. A few comments. 100. This potentially can be useful for Kafka Connect and Kafka rest proxy where a worker agent will need to run a task on behalf of a client. We will likely need to change how those services use Kafka clients (producer/consumer). Instead of a shared client per

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Jason Gustafson
Hey Onur, Thanks for the investigation. I agree with Ismael that pushing regex or some kind of patterns into the protocol would help for communicating subscriptions and for avoiding unnecessary overhead when fetching topic metadata, but it doesn't seem like it would address the main issue here sin

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Ismael Juma
Hi Jason, It would definitely be interesting to try a few of these optimisations on a real world example to quantify the impact. Ismael On Mon, May 23, 2016 at 6:59 PM, Jason Gustafson wrote: > Hey Onur, > > Thanks for the investigation. I agree with Ismael that pushing regex or > some kind of

[jira] [Commented] (KAFKA-3120) Consumer doesn't get messages from some partitions after reassign

2016-05-23 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296845#comment-15296845 ] Vahid Hashemian commented on KAFKA-3120: I can't reproduce this using the current

[jira] [Commented] (KAFKA-3158) ConsumerGroupCommand should tell whether group is actually dead

2016-05-23 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296877#comment-15296877 ] Ishita Mandhan commented on KAFKA-3158: --- [~jasong35] How can I reproduce the error?

[jira] [Assigned] (KAFKA-3264) Mark the old Scala consumer and related classes as deprecated

2016-05-23 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3264?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-3264: -- Assignee: Vahid Hashemian > Mark the old Scala consumer and related classes as deprecat

[jira] [Commented] (KAFKA-3275) Replace 0.9.1.0 references with 0.10.0.0

2016-05-23 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296898#comment-15296898 ] Vahid Hashemian commented on KAFKA-3275: [~ijuma] Can this JIRA be closed since it

[jira] [Resolved] (KAFKA-3275) Replace 0.9.1.0 references with 0.10.0.0

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3275. Resolution: Fixed Yes, thanks. > Replace 0.9.1.0 references with 0.10.0.0 > ---

Build failed in Jenkins: kafka-0.10.0-jdk7 #103

2016-05-23 Thread Apache Jenkins Server
See Changes: [cshapi] Bump version to 0.10.0.0 for a release candidate [cshapi] bumping version of branch from 0.10.0.0-SNAPSHOT to 0.10.0.1-SNAPSHOT, -- Started by an SCM change [EnvInject] -

[jira] [Commented] (KAFKA-3158) ConsumerGroupCommand should tell whether group is actually dead

2016-05-23 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296928#comment-15296928 ] Jason Gustafson commented on KAFKA-3158: [~imandhan] The easiest way to reproduce

[jira] [Assigned] (KAFKA-3685) Auto-generate ZooKeeper data structure wiki

2016-05-23 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3685?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-3685: -- Assignee: Vahid Hashemian > Auto-generate ZooKeeper data structure wiki > -

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Onur Karaman
To get a better sense of the limit and what we should be optimizing for, it helps to look at the message format: private val MEMBER_METADATA_V0 = new Schema(new Field("member_id", STRING), new Field("client_id", STRING), new Field("client_host", STRING), new Field("session_timeout", INT32),

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

2016-05-23 Thread Gwen Shapira
Hi Jun, Few of my answers below (since these are things we discussed, or that I thought about) On Mon, May 23, 2016 at 10:35 AM, Jun Rao wrote: > Thanks for the KIP. A few comments. > > 100. This potentially can be useful for Kafka Connect and Kafka rest proxy > where a worker agent will need t

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296991#comment-15296991 ] Guozhang Wang commented on KAFKA-3745: -- [~gfodor] Actually can you share your join us

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

2016-05-23 Thread parth brahmbhatt
Hi Jun, Thanks for reviewing. * We could add a Cluster action to add acls on who can request delegation tokens. I don't see the use case for that yet but down the line when we start supporting getDelegationTokenAs it will be necessary. * Yes we recommend tokens to be only used/distributed over se

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Guozhang Wang
Discussed with Jason about several optimization proposals, and summarize them here: --- Today the offset topic message value format is: [member subscription assignment] where subscription and assignment are just bytes to the brokers, and consumers know the schema to interpret them; usual

[jira] [Updated] (KAFKA-3393) Update site docs and javadoc based on max.block.ms changes

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3393?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3393: --- Fix Version/s: (was: 0.10.1.0) 0.10.0.1 > Update site docs and javadoc based on

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Jason Gustafson
> > Assignments also can be optimized with some tricks like the ones Jason > mentioned, but I think these end up being specific to the assignment > strategy, making it hard to keep a generic ConsumerProtocol. Leaving the protocol generic would be ideal since tools (such as consumer-groups.sh) dep

[jira] [Updated] (KAFKA-3258) BrokerTopicMetrics of deleted topics are never deleted

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3258: --- Fix Version/s: (was: 0.10.1.0) 0.10.0.1 > BrokerTopicMetrics of deleted topics

Build failed in Jenkins: kafka-0.10.0-jdk7 #104

2016-05-23 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-3393; Updated the docs to reflect the deprecation of [ismael] KAFKA-3258; Delete broker topic metrics of deleted topics -- Started by an SCM change [EnvInject] - Loa

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297250#comment-15297250 ] Greg Fodor commented on KAFKA-3745: --- sure. we are left joining a KTable against a KStrea

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297299#comment-15297299 ] Guozhang Wang commented on KAFKA-3745: -- I may be still getting your motivations wrong

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread James Cheng
> On May 23, 2016, at 10:59 AM, Jason Gustafson wrote: > > 2. Maybe there's a better way to lay out the assignment without needing to > explicitly repeat the topic? For example, the leader could sort the topics > for each member and just use an integer to represent the index of each > topic with

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Onur Karaman
When figuring out these optimizations, it's worth keeping in mind the improvements when the message is uncompressed vs when it's compressed. When uncompressed: Fixing the Assignment serialization to instead be a topic index into the corresponding member's subscription list would usually be a good

[GitHub] kafka-site pull request: KAFKA-3709: Create a project security pag...

2016-05-23 Thread ijuma
Github user ijuma commented on the pull request: https://github.com/apache/kafka-site/pull/12#issuecomment-221129684 Thanks Jun. I emailed active committers informing them about the creation of the mailing list. I sent an email to the list from a different account and verified that me

[jira] [Commented] (KAFKA-3709) Create project security page

2016-05-23 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297378#comment-15297378 ] ASF GitHub Bot commented on KAFKA-3709: --- Github user ijuma commented on the pull req

[jira] [Created] (KAFKA-3748) Add consumer-property to console tools consumer (similar to --producer-property)

2016-05-23 Thread Bharat Viswanadham (JIRA)
Bharat Viswanadham created KAFKA-3748: - Summary: Add consumer-property to console tools consumer (similar to --producer-property) Key: KAFKA-3748 URL: https://issues.apache.org/jira/browse/KAFKA-3748

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Liquan Pei
Would be interesting to see size after with compression on. On Mon, May 23, 2016 at 4:23 PM, Onur Karaman wrote: > When figuring out these optimizations, it's worth keeping in mind the > improvements when the message is uncompressed vs when it's compressed. > > When uncompressed: > Fixing the As

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Jason Gustafson
> > Jason, doesn't gzip (or other compression) basically do this? If the topic > is a string and the topic is repeated throughout, won't compression > basically replace all repeated instances of it with an index reference to > the full string? Hey James, yeah, that's probably true, but keep in mi

[jira] [Updated] (KAFKA-3396) Unauthorized topics are returned to the user

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3396?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3396: --- Fix Version/s: 0.10.0.1 > Unauthorized topics are returned to the user > -

[jira] [Updated] (KAFKA-3396) Unauthorized topics are returned to the user

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3396?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3396: --- Component/s: security > Unauthorized topics are returned to the user > ---

[jira] [Updated] (KAFKA-3396) Unauthorized topics are returned to the user

2016-05-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3396?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3396: --- Affects Version/s: 0.9.0.0 0.10.0.0 > Unauthorized topics are returned to the u

[GitHub] kafka pull request: Allocate 2 attribute bits to signal payload fo...

2016-05-23 Thread davek2
GitHub user davek2 opened a pull request: https://github.com/apache/kafka/pull/1419 Allocate 2 attribute bits to signal payload format This documentation update proposes a mechanism to signal the serialization used for the message payload, resolving issue https://issues.apache.org/

[jira] [Commented] (KAFKA-3744) Message format needs to identify serializer

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

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Becket Qin
It might worth thinking a little further. We have discussed this before that we want to avoid holding all the group metadata in memory. I am thinking about the following end state: 1. Enable compression on the offset topic. 2. Instead of holding the entire group metadata in memory on the brokers,

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297477#comment-15297477 ] Greg Fodor commented on KAFKA-3745: --- Yep, I admit this is definitely not the most common

[jira] [Created] (KAFKA-3749) "BOOSTRAP_SERVERS_DOC" typo in CommonClientConfigs

2016-05-23 Thread Manu Zhang (JIRA)
Manu Zhang created KAFKA-3749: - Summary: "BOOSTRAP_SERVERS_DOC" typo in CommonClientConfigs Key: KAFKA-3749 URL: https://issues.apache.org/jira/browse/KAFKA-3749 Project: Kafka Issue Type: Improv

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

2016-05-23 Thread Jun Rao
Parth, Thanks for the explanation. A couple of more questions. 110. What does getDelegationTokenAs mean? 111. What's the typical rate of getting and renewing delegation tokens? That may have an impact on whether they should be directed to the controller. Jun On Mon, May 23, 2016 at 1:19 PM, pa

[jira] [Created] (KAFKA-3750) "BOOSTRAP_SERVERS_DOC" typo in CommonClientConfigs

2016-05-23 Thread Manu Zhang (JIRA)
Manu Zhang created KAFKA-3750: - Summary: "BOOSTRAP_SERVERS_DOC" typo in CommonClientConfigs Key: KAFKA-3750 URL: https://issues.apache.org/jira/browse/KAFKA-3750 Project: Kafka Issue Type: Improv

[jira] [Resolved] (KAFKA-3750) "BOOSTRAP_SERVERS_DOC" typo in CommonClientConfigs

2016-05-23 Thread Manu Zhang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3750?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manu Zhang resolved KAFKA-3750. --- Resolution: Duplicate sorry, only at-least-once is guaranteed. > "BOOSTRAP_SERVERS_DOC" typo in Comm

[GitHub] kafka pull request: KAFKA-3749, fix "BOOSTRAP_SERVERS_DOC" typo

2016-05-23 Thread manuzhang
GitHub user manuzhang opened a pull request: https://github.com/apache/kafka/pull/1420 KAFKA-3749, fix "BOOSTRAP_SERVERS_DOC" typo You can merge this pull request into a Git repository by running: $ git pull https://github.com/manuzhang/kafka KAFKA-3749 Alternatively you can

[jira] [Commented] (KAFKA-3749) "BOOSTRAP_SERVERS_DOC" typo in CommonClientConfigs

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

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297660#comment-15297660 ] Guozhang Wang commented on KAFKA-3745: -- Just wanted to make sure that we are on the s

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297733#comment-15297733 ] Greg Fodor commented on KAFKA-3745: --- Yes, the join key needs to be added to the final jo

[jira] [Commented] (KAFKA-3745) Consider adding join key to ValueJoiner interface

2016-05-23 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15297745#comment-15297745 ] Greg Fodor commented on KAFKA-3745: --- Actually this seems like a good enough solution, I