Re: [DISCUSS] KIP-68 Add a consumed log retention before log retention

2016-10-10 Thread Pengwei (L)
Hi Dong Thanks for the questions: 1. Now we don't distinguish inactive or active groups. Because in some case maybe inactive group will become active again, and using the previous commit offset. So we will not delete the log segment in the consumer retention if there are some groups consum

[jira] [Updated] (KAFKA-4273) Streams DSL - Add TTL / retention period support for intermediate topics and state stores

2016-10-10 Thread Davor Poldrugo (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Poldrugo updated KAFKA-4273: -- Description: Hi! I'm using Streams DSL (0.10.0.1), which can only use RocksDB for local state as

[jira] [Created] (KAFKA-4282) View / Delete Replication Quotas via Config Command EntityName Wildcards for Topic/Broker

2016-10-10 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-4282: --- Summary: View / Delete Replication Quotas via Config Command EntityName Wildcards for Topic/Broker Key: KAFKA-4282 URL: https://issues.apache.org/jira/browse/KAFKA-4282

Re: [DISCUSS] KIP-68 Add a consumed log retention before log retention

2016-10-10 Thread Dong Lin
Hey David, Thanks for reply. Please see comment inline. On Mon, Oct 10, 2016 at 12:40 AM, Pengwei (L) wrote: > Hi Dong >Thanks for the questions: > > 1. Now we don't distinguish inactive or active groups. Because in some > case maybe inactive group will become active again, and using the p

Re: [DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2016-10-10 Thread Rajini Sivaram
Gwen, Thank you for reviewing the KIP. There has been interest in making the password verification in SASL/PLAIN more pluggable. So I think it makes sense to have a pluggable interface that can be adopted for any SASL mechanism rather than just SCRAM. With the current proposal, you can plugin ano

[jira] [Work started] (KAFKA-4180) Shared authentification with multiple actives Kafka producers/consumers

2016-10-10 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4180 started by Mickael Maison. - > Shared authentification with multiple actives Kafka producers/consumers > ---

[jira] [Work started] (KAFKA-4081) Consumer API consumer new interface commitSyn does not verify the validity of offset

2016-10-10 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4081?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4081 started by Mickael Maison. - > Consumer API consumer new interface commitSyn does not verify the validity of

Re: Store flushing on commit.interval.ms from KIP-63 introduces aggregation latency

2016-10-10 Thread Eno Thereska
Hi Greg, Thanks for trying 0.10.1. The best option you have for your specific app is to simply turn off caching by setting the cache size to 0. That should give you the old behaviour: streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0L); Your PR is an alternative, but it

[GitHub] kafka pull request #1999: 2.11-0.10.0.1 can't see /consumers/group node

2016-10-10 Thread cyfonly
GitHub user cyfonly opened a pull request: https://github.com/apache/kafka/pull/1999 2.11-0.10.0.1 can't see /consumers/group node My producers and consumers all connected to kafka borkers and can exchange messages quite well, I can see "/consumers" node in zookeeper but there's no

[jira] [Work started] (KAFKA-4283) records deleted from CachingKeyValueStore still appear in range and all queries

2016-10-10 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4283 started by Damian Guy. - > records deleted from CachingKeyValueStore still appear in range and all > queries >

[jira] [Created] (KAFKA-4283) records deleted from CachingKeyValueStore still appear in range and all queries

2016-10-10 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4283: - Summary: records deleted from CachingKeyValueStore still appear in range and all queries Key: KAFKA-4283 URL: https://issues.apache.org/jira/browse/KAFKA-4283 Project: Kafk

[jira] [Created] (KAFKA-4284) Partitioner never closed by producer

2016-10-10 Thread Theo Hultberg (JIRA)
Theo Hultberg created KAFKA-4284: Summary: Partitioner never closed by producer Key: KAFKA-4284 URL: https://issues.apache.org/jira/browse/KAFKA-4284 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-2091) Expose a Partitioner interface in the new producer

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

[GitHub] kafka pull request #2000: Make Partitioner a Closeable and close it when clo...

2016-10-10 Thread iconara
GitHub user iconara opened a pull request: https://github.com/apache/kafka/pull/2000 Make Partitioner a Closeable and close it when closing the producer Even though Partitioner has a close method it is not closed when the producer is closed. Serializers, interceptors and metrics are

[jira] [Updated] (KAFKA-4284) Partitioner never closed by producer

2016-10-10 Thread Theo Hultberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Theo Hultberg updated KAFKA-4284: - Description: Partitioners are never closed by the producer, even though the Partitioner interface

[jira] [Updated] (KAFKA-4284) Partitioner never closed by producer

2016-10-10 Thread Theo Hultberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Theo Hultberg updated KAFKA-4284: - Description: Partitioners are never closed by the producer, even though the Partitioner interface

[jira] [Updated] (KAFKA-4284) Partitioner never closed by producer

2016-10-10 Thread Theo Hultberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Theo Hultberg updated KAFKA-4284: - Description: Partitioners are never closed by the producer, even though the Partitioner interface

[jira] [Updated] (KAFKA-4284) Partitioner never closed by producer

2016-10-10 Thread Theo Hultberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Theo Hultberg updated KAFKA-4284: - Status: Patch Available (was: Open) Patch available here: https://github.com/apache/kafka/pull/20

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

2016-10-10 Thread Mickael Maison
Hi all, I would like to discuss the following KIP proposal: https://cwiki.apache.org/confluence/display/KAFKA/KIP-81%3A+Max+in-flight+fetches Feedback and comments are welcome. Thanks ! Mickael

[jira] [Commented] (KAFKA-4283) records deleted from CachingKeyValueStore still appear in range and all queries

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

[GitHub] kafka pull request #2001: KAFKA-4283: records deleted from CachingKeyValueSt...

2016-10-10 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2001 KAFKA-4283: records deleted from CachingKeyValueStore still appear in range and all queries Records that are deleted/removed from the CachingKeyValueStore shouldn't appear in range and all queries.

[jira] [Updated] (KAFKA-4283) records deleted from CachingKeyValueStore still appear in range and all queries

2016-10-10 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4283: -- Status: Patch Available (was: In Progress) > records deleted from CachingKeyValueStore still appear in

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

2016-10-10 Thread Ismael Juma
Hi Mickael, Thanks for the KIP. A quick comment on the rejected alternative of using a bounded memory pool: "While this might be the best way to handle that on the server side it's unclear if this would suit the client well. Usually the client has a rough idea about how many partitions it will be

[GitHub] kafka pull request #1994: MINOR: Introduction Doc: Fixed incomplete sentence

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1994 --- 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 enab

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2016-10-10 Thread David J. Garcia (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15562785#comment-15562785 ] David J. Garcia commented on KAFKA-3514: maybe just use Max(all_paritions_ts) inst

[jira] [Created] (KAFKA-4285) ReplicaVerificationToolTest.test_replica_lags: Timed out waiting to reach non-zero number of replica lags.

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-4285: Summary: ReplicaVerificationToolTest.test_replica_lags: Timed out waiting to reach non-zero number of replica lags. Key: KAFKA-4285 URL: https://issues.apache.org/

Re: [DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2016-10-10 Thread Gwen Shapira
I think it is fine to break the password store to an interface in a separate KIP. I actually love the idea of smaller KIPs dealing with more specific functionality. I just wasn't clear why it was rejected. Thank you for clarifying. I'm happy with current proposal. Gwen On Mon, Oct 10, 2016 at 2:

[jira] [Resolved] (KAFKA-4285) ReplicaVerificationToolTest.test_replica_lags: Timed out waiting to reach non-zero number of replica lags.

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4285?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4285. -- Resolution: Duplicate > ReplicaVerificationToolTest.test_replica_lags: Timed out

[jira] [Commented] (KAFKA-3808) Transient failure in ReplicaVerificationToolTest

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15562803#comment-15562803 ] Ewen Cheslack-Postava commented on KAFKA-3808: -- Seen again running against tr

[GitHub] kafka pull request #1996: MINOR: Fixed introduction doc - wrong streams api ...

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1996 --- 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 enab

Re: Store flushing on commit.interval.ms from KIP-63 introduces aggregation latency

2016-10-10 Thread Greg Fodor
Hey Eno, thanks for the suggestion -- understood that my patch is not something that could be accepted given the API change, I posted it to help make the discussion concrete and because i needed a workaround. (Likely we'll maintain this patch internally so we can move forward with the new version,

[jira] [Created] (KAFKA-4286) metric reporter may hit NullPointerException during shutdown

2016-10-10 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4286: -- Summary: metric reporter may hit NullPointerException during shutdown Key: KAFKA-4286 URL: https://issues.apache.org/jira/browse/KAFKA-4286 Project: Kafka Issue Type: B

[jira] [Commented] (KAFKA-4286) metric reporter may hit NullPointerException during shutdown

2016-10-10 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4286?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15562886#comment-15562886 ] Jun Rao commented on KAFKA-4286: There are a few ways that we could fix this. One way is t

[jira] [Commented] (KAFKA-4280) Add REST resource for showing available connector plugin configs

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15562889#comment-15562889 ] Ewen Cheslack-Postava commented on KAFKA-4280: -- [~gwenshap] You can get the l

Re: [DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2016-10-10 Thread Rajini Sivaram
Gwen, Thank you, will raise a separate KIP for a pluggable interface. On Mon, Oct 10, 2016 at 5:55 PM, Gwen Shapira wrote: > I think it is fine to break the password store to an interface in a > separate KIP. I actually love the idea of smaller KIPs dealing with > more specific functionality. I

[jira] [Comment Edited] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2016-10-10 Thread David J. Garcia (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15562785#comment-15562785 ] David J. Garcia edited comment on KAFKA-3514 at 10/10/16 6:42 PM: --

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

2016-10-10 Thread Apache Jenkins Server
See

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fixed incomplete sentence in introduction docs -- [...truncated 14063 lines...] org.apache.kafka.streams.processor.internals.assignment.TaskAssignorTest > testSticki

[VOTE] 0.10.1.0 RC1

2016-10-10 Thread Jason Gustafson
Hello Kafka users, developers and client-developers, This is the second candidate for release of Apache Kafka 0.10.1.0. This is a minor release that includes great new features including throttled replication, secure quotas, time-based log searching, and queryable state for Kafka Streams. A full l

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fixed incomplete sentence in introduction docs -- [...truncated 13973 lines...] org.apache.kafka.streams.state.internals.NamedCacheTest > shouldKeepTrackOfSize PASSE

[jira] [Created] (KAFKA-4287) auto generate DynamicConfig in docs

2016-10-10 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4287: -- Summary: auto generate DynamicConfig in docs Key: KAFKA-4287 URL: https://issues.apache.org/jira/browse/KAFKA-4287 Project: Kafka Issue Type: Improvement Compo

[GitHub] kafka pull request #1966: KAFKA-4244: fixing formating issues in docs. missi...

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1966 --- 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 enab

[jira] [Commented] (KAFKA-4244) Update our website look & feel

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

[jira] [Resolved] (KAFKA-4244) Update our website look & feel

2016-10-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4244?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4244. Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1966 [https:

Re: [DISCUSS] KIP-68 Add a consumed log retention before log retention

2016-10-10 Thread Renu Tewari
Hi David This is a very timely KIP given the number of use cases in the streams processing pipeline than need consumed log retention management. Some questions that Becket and Dong asked just wanted to make sure are described in the KIP. 1. How is the configuration setup per topic to know what

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

2016-10-10 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #846: KAFKA-3175 : Topic not accessible after deletion ev...

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/846 --- 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 enabl

[jira] [Commented] (KAFKA-3175) topic not accessible after deletion even when delete.topic.enable is disabled

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

[jira] [Updated] (KAFKA-3175) topic not accessible after deletion even when delete.topic.enable is disabled

2016-10-10 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3175?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-3175: -- Resolution: Fixed Reviewer: Joel Koshy Status: Resolved (was: Patch Available) > topic no

[GitHub] kafka pull request #2002: MINOR: Fix typos in documentation

2016-10-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2002 MINOR: Fix typos in documentation And improve readability by adding proper punctuations. You can merge this pull request into a Git repository by running: $ git pull https://github.com/v

[GitHub] kafka pull request #2003: make documentation follow latest template

2016-10-10 Thread derrickdoo
GitHub user derrickdoo opened a pull request: https://github.com/apache/kafka/pull/2003 make documentation follow latest template Make the latest version of our docs follow the latest site template structure. You can merge this pull request into a Git repository by running: $

[GitHub] kafka pull request #2004: MINOR: Fix table of content in and cross reference...

2016-10-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2004 MINOR: Fix table of content in and cross references in the documentation Removed a non-existing reference in table of contents and fixed some section numbers. You can merge this pull request

[GitHub] kafka pull request #2003: make documentation follow latest template

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2003 --- 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 enab

[GitHub] kafka pull request #2002: MINOR: Fix typos in documentation

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2002 --- 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 enab

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fixed introduction doc - wrong streams api link -- [...truncated 6680 lines...] kafka.api.PlaintextConsumerTest > testMultiConsumerDefaultAssignment PASSED kafka.api

[GitHub] kafka pull request #2005: KAFKA-4269 extracted code updating topics when reg...

2016-10-10 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/2005 KAFKA-4269 extracted code updating topics when regex pattern specifie… …d out of topicGroups method. The topicGroups method only called from StreamPartitionAssignor when KafkaStreams object is

[jira] [Commented] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

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

[jira] [Updated] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

2016-10-10 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-4269: --- Fix Version/s: 0.10.1.0 Status: Patch Available (was: In Progress) > Multiple KStream inst

[jira] [Commented] (KAFKA-4114) Allow for different "auto.offset.reset" strategies for different input streams

2016-10-10 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4114?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15563934#comment-15563934 ] Bill Bejeck commented on KAFKA-4114: I have a working solution, just waiting for resol

[jira] [Commented] (KAFKA-4114) Allow for different "auto.offset.reset" strategies for different input streams

2016-10-10 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4114?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15563941#comment-15563941 ] Matthias J. Sax commented on KAFKA-4114: Just open an PR whenever you are ready. :

[GitHub] kafka pull request #2004: MINOR: Fix table of content and section numbers in...

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2004 --- 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 enab

[jira] [Created] (KAFKA-4288) Add an "Important Server Configs" section to documentation

2016-10-10 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-4288: -- Summary: Add an "Important Server Configs" section to documentation Key: KAFKA-4288 URL: https://issues.apache.org/jira/browse/KAFKA-4288 Project: Kafka

[GitHub] kafka pull request #2006: KAFKA-4289 - moved short-lived loggers to companio...

2016-10-10 Thread radai-rosenblatt
GitHub user radai-rosenblatt opened a pull request: https://github.com/apache/kafka/pull/2006 KAFKA-4289 - moved short-lived loggers to companion objects Signed-off-by: radai-rosenblatt You can merge this pull request into a Git repository by running: $ git pull https://github

[jira] [Created] (KAFKA-4289) CPU wasted on reflection calls initializing short-lived loggers

2016-10-10 Thread radai rosenblatt (JIRA)
radai rosenblatt created KAFKA-4289: --- Summary: CPU wasted on reflection calls initializing short-lived loggers Key: KAFKA-4289 URL: https://issues.apache.org/jira/browse/KAFKA-4289 Project: Kafka

[jira] [Commented] (KAFKA-4289) CPU wasted on reflection calls initializing short-lived loggers

2016-10-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15563996#comment-15563996 ] ASF GitHub Bot commented on KAFKA-4289: --- GitHub user radai-rosenblatt opened a pull

[GitHub] kafka-site pull request #24: update htaccess to load images nested inside of...

2016-10-10 Thread derrickdoo
GitHub user derrickdoo opened a pull request: https://github.com/apache/kafka-site/pull/24 update htaccess to load images nested inside of doc folders You can merge this pull request into a Git repository by running: $ git pull https://github.com/derrickdoo/kafka-site imagePat

[GitHub] kafka-site issue #24: update htaccess to load images nested inside of doc fo...

2016-10-10 Thread derrickdoo
Github user derrickdoo commented on the issue: https://github.com/apache/kafka-site/pull/24 - stop displaying code elements as display:block - don't rewrite requests for images nested in the specific doc version directories --- If your project is set up for it, you can reply to t

[GitHub] kafka pull request #2007: Kafka 4114 allow different offset reset strategies

2016-10-10 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/2007 Kafka 4114 allow different offset reset strategies @mjsax Here's my first pass at finer grained auto offset reset strategies. I've left TODO comments about whether we want to consi

[jira] [Commented] (KAFKA-4114) Allow for different "auto.offset.reset" strategies for different input streams

2016-10-10 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4114?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15564050#comment-15564050 ] Bill Bejeck commented on KAFKA-4114: Fair enough doing so now. > Allow for different

[jira] [Updated] (KAFKA-4114) Allow for different "auto.offset.reset" strategies for different input streams

2016-10-10 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-4114: --- Status: Patch Available (was: In Progress) > Allow for different "auto.offset.reset" strategies for d

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

2016-10-10 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (Ubuntu ubuntu ubuntu-eu docker) in workspace

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4244; Fix formatting issues in documentation [jjkoshy] KAFKA-3175; Topic not accessible after deletion even when [jason] MINOR: Make documentation follow latest template [jason] MINOR: Fix typos in document

[GitHub] kafka-site issue #24: update htaccess to load images nested inside of doc fo...

2016-10-10 Thread hachikuji
Github user hachikuji commented on the issue: https://github.com/apache/kafka-site/pull/24 LGTM. Thanks for the fixes! --- 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

[GitHub] kafka pull request #2008: MINOR: Add images missing from documentation

2016-10-10 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2008 MINOR: Add images missing from documentation You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka add-missing-images Alternatively

[GitHub] kafka pull request #2008: MINOR: Add images missing from documentation

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2008 --- 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 enab

[GitHub] kafka pull request #1964: KAFKA-4010: add ConfigDef toEnrichedRst() for addi...

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1964 --- 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 enab

[jira] [Commented] (KAFKA-4010) ConfigDef.toRst() should create sections for each group

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

[jira] [Resolved] (KAFKA-4010) ConfigDef.toRst() should create sections for each group

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4010. -- Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request

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

2016-10-10 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (Ubuntu ubuntu ubuntu-eu docker) in workspace

Re: [VOTE] 0.10.1.0 RC1

2016-10-10 Thread Jason Gustafson
The documentation is mostly fixed now: http://kafka.apache.org/ 0101/documentation.html. Thanks to Derrick Or for all the help. Let me know if anyone notices any additional problems. -Jason On Mon, Oct 10, 2016 at 1:10 PM, Jason Gustafson wrote: > Hello Kafka users, developers and client-develo

[jira] [Created] (KAFKA-4290) High CPU caused by timeout overflow in WorkerCoordinator

2016-10-10 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4290: -- Summary: High CPU caused by timeout overflow in WorkerCoordinator Key: KAFKA-4290 URL: https://issues.apache.org/jira/browse/KAFKA-4290 Project: Kafka Is

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

2016-10-10 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (Ubuntu ubuntu ubuntu-eu docker) in workspace

[jira] [Commented] (KAFKA-4290) High CPU caused by timeout overflow in WorkerCoordinator

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

[GitHub] kafka pull request #2009: KAFKA-4290: Fix timeout overflow in WorkerCoordina...

2016-10-10 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2009 KAFKA-4290: Fix timeout overflow in WorkerCoordinator.poll You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-4290 Alterna

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4244; Fix formatting issues in documentation [jjkoshy] KAFKA-3175; Topic not accessible after deletion even when [jason] MINOR: Make documentation follow latest template [jason] MINOR: Fix typos in documen

[GitHub] kafka pull request #2010: MINOR: Fixed broken links in the documentation

2016-10-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2010 MINOR: Fixed broken links in the documentation You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka doc/fix_hyperlinks A

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4244; Fix formatting issues in documentation [jason] MINOR: Make documentation follow latest template [jason] MINOR: Fix typos in documentation [jason] MINOR: Fix table of contents and section numbers in th

[GitHub] kafka pull request #2009: KAFKA-4290: Fix timeout overflow in WorkerCoordina...

2016-10-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2009 --- 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 enab

[jira] [Resolved] (KAFKA-4290) High CPU caused by timeout overflow in WorkerCoordinator

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4290. -- Resolution: Fixed Reviewer: Ewen Cheslack-Postava > High CPU caused by tim

[jira] [Commented] (KAFKA-4290) High CPU caused by timeout overflow in WorkerCoordinator

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

[jira] [Updated] (KAFKA-4290) High CPU caused by timeout overflow in WorkerCoordinator

2016-10-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4290: - Fix Version/s: 0.10.2.0 > High CPU caused by timeout overflow in WorkerCoordinator

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

2016-10-10 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Add images missing from documentation [me] KAFKA-4010: add ConfigDef toEnrichedRst() for additional fields in -- [...truncated 4982 lines...] kafka.coordinator.Grou

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

2016-10-10 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-eu2 (Ubuntu ubuntu ubuntu-eu docker) in workspace

Jenkins build is back to normal : kafka-trunk-jdk8 #966

2016-10-10 Thread Apache Jenkins Server
See