Re: [DISCUSS] scalability limits in the coordinator

2016-05-25 Thread Onur Karaman
I gave the topic index assignment trick a try against the same environment. The implementation just changed the assignment serialization and deserialization logic. It didn't change SyncGroupResponse, meaning it continues to exclude the subscription from the SyncGroupResponse and assumes the member

Re: Kafka server contains messages but consumer cannot receive any(Producer used compression.type = snappy)

2016-05-25 Thread Jason Gustafson
Hi Nicole, It would help to see some log files. I cannot view the image you've tried to include above. I took a quick glance at your code and noticed that your producer only writes 50 messages before it stops. Since you have not overridden auto.offset.reset in the consumer config, the default

Re: Kafka server contains messages but consumer cannot receive any(Producer used compression.type = snappy)

2016-05-25 Thread Shaolu Xu
Added the project. On Thu, May 26, 2016 at 9:25 AM, Shaolu Xu wrote: > Hi Tom, > > The following is my producer and consumer configuration: > >- Producer: > >props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, " > 127.0.0.1:9090"); >

Re: Kafka server contains messages but consumer cannot receive any(Producer used compression.type = snappy)

2016-05-25 Thread Shaolu Xu
Hi Tom, The following is my producer and consumer configuration: - Producer: props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, " 127.0.0.1:9090"); props.put("key.serializer", "org.apache.kafka.common.serialization.IntegerSerializer");

Re: Reporting security issues

2016-05-25 Thread Mayuresh Gharat
Excellent :) Thanks, Mayuresh On Tue, May 24, 2016 at 2:55 AM, Ismael Juma wrote: > Hi all, > > Since Kafka implements a number of security features, we need a procedure > for reporting potential security vulnerabilities privately (as per > http://www.apache.org/security/).

Re: subscription request

2016-05-25 Thread Sriram Subramanian
Sorry, wrong thread. On Wed, May 25, 2016 at 5:12 PM, Sriram Subramanian wrote: > >

subscription request

2016-05-25 Thread Sriram Subramanian

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

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15301135#comment-15301135 ] Ismael Juma commented on KAFKA-3727: [~ecomar], yes, those issues should probably be explained better.

[DISCUSS] KIP-62: Allow consumer to send heartbeats from a background thread

2016-05-25 Thread Jason Gustafson
Hi All, One of the persistent problems we see with the new consumer is the use of the session timeout in order to ensure progress. Whenever there is a delay in message processing which exceeds the session timeout, no heartbeats can be sent and the consumer is removed from the group. We seem to

[GitHub] kafka pull request: [KAFKA-3756] [Javadoc] Improving Javadoc

2016-05-25 Thread rekhajoshm
GitHub user rekhajoshm opened a pull request: https://github.com/apache/kafka/pull/1432 [KAFKA-3756] [Javadoc] Improving Javadoc [KAFKA-3756] [Javadoc] Improving Javadoc You can merge this pull request into a Git repository by running: $ git pull

[jira] [Commented] (KAFKA-3756) javadoc has issues with incorrect @param, @throws, @return

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

[jira] [Commented] (KAFKA-3480) Autogenerate metrics documentation

2016-05-25 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15301077#comment-15301077 ] James Cheng commented on KAFKA-3480: Now that 0.10 is out (Congrats everyone!) I wanted to get some

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

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15301035#comment-15301035 ] Ismael Juma commented on KAFKA-3264: Yeah, we only intend to deprecate this once the new consumer is

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

2016-05-25 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15301027#comment-15301027 ] Edoardo Comar commented on KAFKA-3727: -- Thanks [~ijuma] those issues are related. However

[jira] [Commented] (KAFKA-3682) ArrayIndexOutOfBoundsException thrown by SkimpyOffsetMap.get() when full

2016-05-25 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3682?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15301010#comment-15301010 ] Edoardo Comar commented on KAFKA-3682: -- [~vahid] you can just run my unit test without the fix in

[jira] [Updated] (KAFKA-2720) Periodic purging groups in the coordinator

2016-05-25 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2720: - Description: Currently the coordinator removes the group (i.e. both removing it from the cache

[jira] [Assigned] (KAFKA-3755) tightening the offset check in ReplicaFetcherThread

2016-05-25 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ishita Mandhan reassigned KAFKA-3755: - Assignee: Ishita Mandhan > tightening the offset check in ReplicaFetcherThread >

[GitHub] kafka pull request: KAFKA-3754 Add GC log retention policy to limi...

2016-05-25 Thread rnpridgeon
GitHub user rnpridgeon opened a pull request: https://github.com/apache/kafka/pull/1431 KAFKA-3754 Add GC log retention policy to limit size of log Add a default log retention policy to keep GC logs from growing too large You can merge this pull request into a Git repository by

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

2016-05-25 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300940#comment-15300940 ] Vahid Hashemian commented on KAFKA-3264: Is it fair to assume this is blocked by

[jira] [Commented] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

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

[jira] [Created] (KAFKA-3756) javadoc has issues with incorrect @param, @throws, @return

2016-05-25 Thread Rekha Joshi (JIRA)
Rekha Joshi created KAFKA-3756: -- Summary: javadoc has issues with incorrect @param, @throws, @return Key: KAFKA-3756 URL: https://issues.apache.org/jira/browse/KAFKA-3756 Project: Kafka Issue

[jira] [Commented] (KAFKA-3682) ArrayIndexOutOfBoundsException thrown by SkimpyOffsetMap.get() when full

2016-05-25 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3682?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300854#comment-15300854 ] Vahid Hashemian commented on KAFKA-3682: [~ecomar] Can you provide the steps to reproduce this

[jira] [Updated] (KAFKA-3728) EndToEndAuthorizationTest offsets_topic misconfigured

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3728: --- Assignee: Edoardo Comar > EndToEndAuthorizationTest offsets_topic misconfigured >

[jira] [Updated] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

2016-05-25 Thread Ryan P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan P updated KAFKA-3754: -- Description: By default kafka-run-class.sh defines it's GC settings like so:

[jira] [Updated] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

2016-05-25 Thread Ryan P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan P updated KAFKA-3754: -- Description: By default kafka-run-class.sh defines it's GC settings like so:

[GitHub] kafka pull request: MINOR: Use `--force` instead of `--yes` in `Ac...

2016-05-25 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1430 MINOR: Use `--force` instead of `--yes` in `AclCommand` To be consistent with `ConfigCommand` and `TopicCommand`. No release includes this option yet, so we can simply change it. You can

[jira] [Created] (KAFKA-3755) tightening the offset check in ReplicaFetcherThread

2016-05-25 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-3755: -- Summary: tightening the offset check in ReplicaFetcherThread Key: KAFKA-3755 URL: https://issues.apache.org/jira/browse/KAFKA-3755 Project: Kafka Issue Type:

[jira] [Assigned] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

2016-05-25 Thread Ryan P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan P reassigned KAFKA-3754: - Assignee: Ryan P > Kafka default -Xloggc settings should include GC log rotation flags >

[jira] [Created] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

2016-05-25 Thread Ryan P (JIRA)
Ryan P created KAFKA-3754: - Summary: Kafka default -Xloggc settings should include GC log rotation flags Key: KAFKA-3754 URL: https://issues.apache.org/jira/browse/KAFKA-3754 Project: Kafka Issue

Re: [DISCUSS] scalability limits in the coordinator

2016-05-25 Thread Jason Gustafson
Gwen, Joel: That's correct. The protocol does allow us to give an assignor its own assignment schema, but I think this will require a couple internal changes to the consumer to make use of the full generality. One thing I'm a little uncertain about is whether we should use a different protocol

[jira] [Resolved] (KAFKA-2971) KAFKA - Not obeying log4j settings, DailyRollingFileAppender not rolling files

2016-05-25 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dustin Cote resolved KAFKA-2971. Resolution: Resolved Resolving as superceded by KAFKA-2394. Thanks [~gwenshap]. > KAFKA - Not

[jira] [Commented] (KAFKA-3503) Throw exception on missing/non-existent partition

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300784#comment-15300784 ] Ismael Juma commented on KAFKA-3503: I think this is a duplicate of

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

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300781#comment-15300781 ] Ismael Juma commented on KAFKA-3727: [~ecomar], please see KAFKA-2391 and KAFKA-3177. >

[jira] [Commented] (KAFKA-3682) ArrayIndexOutOfBoundsException thrown by SkimpyOffsetMap.get() when full

2016-05-25 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3682?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300756#comment-15300756 ] Edoardo Comar commented on KAFKA-3682: -- Hi this is a pretty simple fix, backed by a unit test. May

[jira] [Commented] (KAFKA-2971) KAFKA - Not obeying log4j settings, DailyRollingFileAppender not rolling files

2016-05-25 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300723#comment-15300723 ] Gwen Shapira commented on KAFKA-2971: - I don't think [~jkreps] is working on this at the moment. Feel

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

2016-05-25 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300716#comment-15300716 ] Ishita Mandhan commented on KAFKA-3158: --- Thanks [~hachikuji]! I just submitted a PR for it >

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

2016-05-25 Thread Rajini Sivaram
Hi Aditya, Thank you for the review. When *quota.type=user*, quotas are based on the user principal which may be an authenticated or unauthenticated user. For PLAINTEXT, the principal would be "*anonymous*" by default, but can be overridden by supplying a principal builder. Quotas can be applied

[jira] [Commented] (KAFKA-2971) KAFKA - Not obeying log4j settings, DailyRollingFileAppender not rolling files

2016-05-25 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300699#comment-15300699 ] Dustin Cote commented on KAFKA-2971: [~jkreps] and [~dban] I believe this issue is best addressed by

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

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

[GitHub] kafka pull request: KAFKA-3158: ConsumerGroupCommand should tell w...

2016-05-25 Thread imandhan
GitHub user imandhan opened a pull request: https://github.com/apache/kafka/pull/1429 KAFKA-3158: ConsumerGroupCommand should tell whether group is actually dead This patch fix differentiates between when a consumer group is rebalancing or dead and reports the appropriate error

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

2016-05-25 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-3732: Add an auto accept option to kafka-acls.sh -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu3 (Ubuntu

[jira] [Commented] (KAFKA-3370) Add options to auto.offset.reset to reset offsets upon initialization only

2016-05-25 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3370?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300643#comment-15300643 ] Vahid Hashemian commented on KAFKA-3370: Thanks [~gwenshap]. And it seems from the description of

[jira] [Updated] (KAFKA-3732) Add an auto accept option to kafka-acls.sh

2016-05-25 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3732?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3732: Assignee: Mickael Maison > Add an auto accept option to kafka-acls.sh >

[jira] [Commented] (KAFKA-3732) Add an auto accept option to kafka-acls.sh

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

[GitHub] kafka pull request: KAFKA-3732: Add an auto accept option to kafka...

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

[jira] [Resolved] (KAFKA-3732) Add an auto accept option to kafka-acls.sh

2016-05-25 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3732?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-3732. - Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1406

Re: [DISCUSS] scalability limits in the coordinator

2016-05-25 Thread Gwen Shapira
ah, right - we can add as many strategies as we want. On Wed, May 25, 2016 at 10:54 AM, Joel Koshy wrote: > > Yes it would be a protocol bump. > > > > Sorry - I'm officially confused. I think it may not be required - since the > more compact format would be associated with

Re: [DISCUSS] scalability limits in the coordinator

2016-05-25 Thread Joel Koshy
> Yes it would be a protocol bump. > Sorry - I'm officially confused. I think it may not be required - since the more compact format would be associated with a new assignment strategy - right? > smaller than the plaintext PAL, but the post-compressed binary PAL is just > 25% smaller than the

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

2016-05-25 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: - Priority: Critical (was: Major) > Consumer.poll() stuck in loop on non-existent topic manually

Re: [DISCUSS] scalability limits in the coordinator

2016-05-25 Thread Joel Koshy
Yes it would be a protocol bump. @Jason - on reducing the size of the assignment field, I would be interested to see what savings we can get - but my hunch is that we would end up picking one of either: a compact assignment field format or turn on compression. We actually did a similar

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

2016-05-25 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300482#comment-15300482 ] Edoardo Comar commented on KAFKA-3727: -- Hi can please anyone comment on this (IMHO) buggy behavior ?

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

2016-05-25 Thread Aditya Auradkar
Hey Rajini - If the quota.type is set to 'user', what happens to unauthenticated clients? They don't supply a principal, so are they essentially unthrottled? This may be a nit, but I prefer 'quota.type' options to be 'authenticated-user' and 'client-id' as opposed to 'client' and 'user'. For a

[jira] [Commented] (KAFKA-3679) Allow reuse of implementation of RFC 4616 in PlainSaslServer

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

[GitHub] kafka pull request: KAFKA-3679 Allow reuse of implementation of RF...

2016-05-25 Thread edoardocomar
Github user edoardocomar closed the pull request at: https://github.com/apache/kafka/pull/1350 --- 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

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Ismael Juma
Sounds good. On 25 May 2016 17:26, "Gwen Shapira" wrote: > All topic level names are inconsistent. We can have a separate discussion / > KIP on getting out of that mess. > > Gwen > > On Wed, May 25, 2016 at 6:07 AM, Ismael Juma wrote: > > > +1 (binding) > >

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

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

[GitHub] kafka pull request: KAFKA-3396 : Unauthorized topics are returned ...

2016-05-25 Thread edoardocomar
GitHub user edoardocomar opened a pull request: https://github.com/apache/kafka/pull/1428 KAFKA-3396 : Unauthorized topics are returned to the user Modified KafkaApis to return Errors.UNKNOWN_TOPIC_OR_PARTITION if principal has no Describe access to topic Unit tests

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Joel Koshy
+1 on the proposal Re: inconsistent names: KAFKA-3234 has a patch and discussion in the PR that should help address the inconsistencies and various other issues but we decided it would need a small KIP. (If someone else wishes to take over that

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Gwen Shapira
All topic level names are inconsistent. We can have a separate discussion / KIP on getting out of that mess. Gwen On Wed, May 25, 2016 at 6:07 AM, Ismael Juma wrote: > +1 (binding) > > I also think `log.cleaner.compaction.delay.ms` is clearer. As an aside, I > did notice

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

2016-05-25 Thread Apache Jenkins Server
See

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

2016-05-25 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Removed 1/2 of the hardcoded sleeps in Streams -- [...truncated 1668 lines...] kafka.log.OffsetIndexTest > lookupExtremeCases PASSED kafka.log.OffsetIndexTest >

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

2016-05-25 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Removed 1/2 of the hardcoded sleeps in Streams -- [...truncated 5592 lines...] org.apache.kafka.streams.kstream.internals.KStreamFlatMapValuesTest >

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Grant Henke
+1 (non-binding) On Wed, May 25, 2016 at 8:20 AM, Ben Stopford wrote: > +1 (non-binding) > > > On 25 May 2016, at 14:07, Ismael Juma wrote: > > > > +1 (binding) > > > > I also think `log.cleaner.compaction.delay.ms` is clearer. As an aside, > I > > did

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

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300130#comment-15300130 ] Ismael Juma commented on KAFKA-3744: Also, I took a look at the PR and it's not clear to me why

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

2016-05-25 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300119#comment-15300119 ] Ismael Juma commented on KAFKA-3744: It changes the message format so it needs a KIP. :) The KIP page

[GitHub] kafka pull request: MINOR: Removed 1/2 of the hardcoded sleep time...

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

[jira] [Created] (KAFKA-3753) Metrics for StateStores

2016-05-25 Thread Jeff Klukas (JIRA)
Jeff Klukas created KAFKA-3753: -- Summary: Metrics for StateStores Key: KAFKA-3753 URL: https://issues.apache.org/jira/browse/KAFKA-3753 Project: Kafka Issue Type: Improvement

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

2016-05-25 Thread David Kay (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15300075#comment-15300075 ] David Kay commented on KAFKA-3744: -- Hi Ismael, thanks for the reply. I can submit a KIP, but I don't

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Ben Stopford
+1 (non-binding) > On 25 May 2016, at 14:07, Ismael Juma wrote: > > +1 (binding) > > I also think `log.cleaner.compaction.delay.ms` is clearer. As an aside, I > did notice that the topic level config for `log.segment.delete.delay.ms` > (mentioned by Ewen) is

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Ismael Juma
+1 (binding) I also think `log.cleaner.compaction.delay.ms` is clearer. As an aside, I did notice that the topic level config for `log.segment.delete.delay.ms` (mentioned by Ewen) is `file.delete.delay.ms`, which seems a bit inconsistent. Ismael On Wed, May 25, 2016 at 4:43 AM, Ewen

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Manikumar Reddy
+1 (non binding) On Wed, May 25, 2016 at 4:03 PM, Tom Crayford wrote: > +1 (non binding) > > Agree on log.cleaner.compaction.delay.ms being the better name. > > I think this setting is going to be extremely hard to tune for users, and > worry about adding yet more

[VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-25 Thread Tom Crayford
+1 (non binding) Agree on log.cleaner.compaction.delay.ms being the better name. I think this setting is going to be extremely hard to tune for users, and worry about adding yet more configuration - Kafka already has a huge number of tunables though, so we're in well trod ground with "just add

Re: Kafka server contains messages but consumer cannot receive any(Producer used compression.type = snappy)

2016-05-25 Thread Tom Crayford
Which consumer are you using? Can you see it connecting to the broker in the broker logs? I'd recommend putting your configs for producer, consumer and broker in a reply to assist debugging. Also please attach any relevant code or log files. Thanks Tom Crayford Heroku Kafka On Wednesday, 25 May

Re: Kafka server contains messages but consumer cannot receive any(Producer used compression.type = snappy)

2016-05-25 Thread Shaolu Xu
Hi All, Anyone have idea about this, Please help me find the issue. Thanks, Nicole On Wed, May 25, 2016 at 11:24 AM, Shaolu Xu wrote: > Hi dev, > > Kafka version: 0.9.0 language: Java > > When using kafka, I can set a codec by setting the* > compression.type=snappy