[jira] [Created] (KAFKA-6348) Kafka consumer can't restore from coordinator failure

2017-12-11 Thread Renjie Liu (JIRA)
Renjie Liu created KAFKA-6348: - Summary: Kafka consumer can't restore from coordinator failure Key: KAFKA-6348 URL: https://issues.apache.org/jira/browse/KAFKA-6348 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Ewen Cheslack-Postava
And to clarify a bit further: the goal is for both standalone and distributed mode to display the same basic information. This hasn't *strictly* been required before because standalone had no worker-level interaction with the cluster (configs stored in memory, offsets on disk, and statuses in

Re: [DISCUSS] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-12-11 Thread Colin McCabe
Sorry... this is probably a silly question, but do Kafka Connect groups share a namespace with consumer groups? If we had a separate API for Kafka Connect groups vs. Consumer groups, would that make sense? Or should we unify them? best, Colin On Mon, Dec 11, 2017, at 16:11, Jason Gustafson

Re: [DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Ewen Cheslack-Postava
On Mon, Dec 11, 2017 at 4:01 PM, Gwen Shapira wrote: > Thanks, Ewen :) > > One thing that wasn't clear to me from the wiki: Will standalone connect > also have a Kafka cluster ID? While it is true that only tasks have > producers and consumers, I think we assumed that all

[jira] [Resolved] (KAFKA-5551) StreamThread should not expose methods for testing

2017-12-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5551?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5551. Resolution: Fixed > StreamThread should not expose methods for testing >

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2017-12-11 Thread Dong Lin
Hey Jun, I have updated the KIP based on our discussion. Thanks! Dong On Sat, Dec 9, 2017 at 10:12 PM, Dong Lin wrote: > Hey Jun, > > Thanks much for your comments. Given that client needs to de-serialize the > metadata anyway, the extra overhead of checking the

Re: [DISCUSS] KIP-234: add support for getting topic defaults from AdminClient

2017-12-11 Thread Colin McCabe
Hi Dan, The KIP looks good overall. On Mon, Dec 11, 2017, at 18:28, Ewen Cheslack-Postava wrote: > I think the key point is when the kafka admin and user creating topics > differ. I think a more realistic example of Dan's point (2) is for > retention. I know that realistically, admins aren't

Re: [VOTE] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-11 Thread Colin McCabe
+1 (non-binding) P.S. Suggest to use whenComplete instead of making addWaiter public. (The differences is very slight : addWaiter returns void, but whenComplete returns a future which gets completed with either an exception if the BiConsumer failed, or the value, otherwise.) Colin On Mon, Dec

[GitHub] kafka-site pull request #111: KAFKA-6334: fix typo in backwards compatibilit...

2017-12-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/111 ---

[jira] [Resolved] (KAFKA-6334) Minor documentation typo

2017-12-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6334. -- Resolution: Fixed > Minor documentation typo > > >

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-11 Thread Colin McCabe
Thanks, Xavier we should definitely think about what happens when exceptions are thrown from these functions. I would suggest maybe we should just implement whenComplete, rather than exposing addWaiter. addWaiter was never intended as a public API, and it's a little weird. whenComplete is

[GitHub] kafka-site issue #111: KAFKA-6334: fix typo in backwards compatibility note

2017-12-11 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/111 LGTM. Merged to asf-site. ---

[GitHub] kafka pull request #4316: kafka-future-whencomplete

2017-12-11 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/4316 kafka-future-whencomplete *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so use a separate comment to ping

Re: [DISCUSS] KIP-234: add support for getting topic defaults from AdminClient

2017-12-11 Thread Ewen Cheslack-Postava
I think the key point is when the kafka admin and user creating topics differ. I think a more realistic example of Dan's point (2) is for retention. I know that realistically, admins aren't just going to randomly drop the broker defaults from 1w to 1d without warning anyone (they'd likely be

Re: [DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Gwen Shapira
Thanks, Ewen :) One thing that wasn't clear to me from the wiki: Will standalone connect also have a Kafka cluster ID? While it is true that only tasks have producers and consumers, I think we assumed that all tasks on one stand-alone will use one Kafka cluster? Another suggestion is not to

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2017-12-11 Thread Jun Rao
Hi, Dong, The case that I am thinking is network partitioning. Suppose one deploys a stretched cluster across multiple AZs in the same region. If the machines in one AZ can't communicate to brokers in other AZs due to a network issue, the brokers in that AZ won't get any new metadata. We can

[GitHub] kafka pull request #4315: KAFKA-6150: KIP-204 part III; Change repartition t...

2017-12-11 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/4315 KAFKA-6150: KIP-204 part III; Change repartition topic segment size and ms 1. Create default internal topic configs in StreamsConfig, especially for repartition topics change the segment size

[jira] [Reopened] (KAFKA-2319) After controlled shutdown: IllegalStateException: Kafka scheduler has not been started

2017-12-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2319?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reopened KAFKA-2319: -- > After controlled shutdown: IllegalStateException: Kafka scheduler has not > been started >

Re: [DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Ted Yu
I agree the formation in the pull request is the cleanest. Thanks On Mon, Dec 11, 2017 at 5:05 PM, Ewen Cheslack-Postava wrote: > I did, but it doesn't seem to gain much. In order to still avoid having > these intermediate states, you'd still need a latch and then to block

Re: [DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Ewen Cheslack-Postava
I did, but it doesn't seem to gain much. In order to still avoid having these intermediate states, you'd still need a latch and then to block any calls to the root resource until you could connect. It would allow starting up the rest of the worker, but if it's just going to fail and put the worker

Re: [DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Ted Yu
Looks good overall. Currently lookupKafkaClusterId() is called synchronously. Have you considered making the call asynchronous (normally the GET / request comes sometime after worker start) ? Thanks On Mon, Dec 11, 2017 at 3:40 PM, Ewen Cheslack-Postava wrote: > I'd like to

[jira] [Resolved] (KAFKA-6341) 'networkThreadTimeNanos' in KafkaChannel is not thread safe

2017-12-11 Thread huxihx (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] huxihx resolved KAFKA-6341. --- Resolution: Not A Bug > 'networkThreadTimeNanos' in KafkaChannel is not thread safe >

[jira] [Created] (KAFKA-6347) Starting offset breach based log segment deletion never considers active segment

2017-12-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6347: Summary: Starting offset breach based log segment deletion never considers active segment Key: KAFKA-6347 URL: https://issues.apache.org/jira/browse/KAFKA-6347

Re: [DISCUSS] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-12-11 Thread Jason Gustafson
Hi Jorge, Kafka group management is actually more general than consumer groups (e.g. there are kafka connect groups). If we are adding these APIs, I would suggest we consider the more general protocol and how to expose group-protocol-specific metadata. For example, it might be reasonable to have

答复: 答复: [VOTE] KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-12-11 Thread Hu Xi
Hi Mickael Maison, Thanks for the comments, but I think it deserves another KIP since this one mainly focuses on metrics things. The way to calculate the lag in ConsumerGroupCommand is seemingly different from what metrics do, so a new KIP might be required. What do you think? huxihx

Re: [VOTE] KIP-225 - Use tags for consumer “records.lag” metrics

2017-12-11 Thread Jason Gustafson
+1. Thanks for the KIP. On Mon, Dec 11, 2017 at 1:54 AM, charly molter wrote: > Hi, > The KIP has been updated. As it has change should I restart the vote? > > In any case I'm still missing one binding vote if anyone wants to help. > Thanks! > > On Wed, Dec 6, 2017 at

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-12-11 Thread Jason Gustafson
Hi Vahid, Seems reasonable to me. A few questions. 1. What error code do we use when the group is not empty? 2. Do we need the capability to delete offsets individually for a partition? I ask because the offset tool lets you set the offset for partitions individually, so it might be natural to

[DISCUSS] KIP-238: Expose Kafka cluster ID in Connect REST API

2017-12-11 Thread Ewen Cheslack-Postava
I'd like to start discussion on a simple KIP to expose Kafka cluster ID info in the Connect REST API: https://cwiki.apache.org/confluence/display/KAFKA/KIP-238%3A+Expose+Kafka+cluster+ID+in+Connect+REST+API Hopefully straightforward, though there are some details on how this affects startup

[GitHub] kafka pull request #4314: KAFKA-6331: Expose Kafka cluster ID in Connect RES...

2017-12-11 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/4314 KAFKA-6331: Expose Kafka cluster ID in Connect REST API (KIP-238) *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so

[GitHub] kafka pull request #4313: MINOR: broker down for significant amt of time sys...

2017-12-11 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/4313 MINOR: broker down for significant amt of time system test System test where a broker is offline more than the configured timeouts. In this case: - Max poll interval set to 45 secs -

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Colin McCabe
On Mon, Dec 11, 2017, at 14:51, Becket Qin wrote: > Hi Jun, > > Yes, I agree avoiding reading the log segment is not the primary goal for > this KIP. I brought this up because recently I saw a significant > throughput > impact when a broker is down for 20 - 30 min and rejoins a cluster. The >

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-12-11 Thread Vahid S Hashemian
If there are no additional feedback on this KIP, I'll start a vote in a couple of days. Thanks. --Vahid From: Vahid S Hashemian/Silicon Valley/IBM To: dev@kafka.apache.org Date: 11/29/2017 11:15 AM Subject:Re: [DISCUSS] KIP-229: DeleteGroups API Hi Dong, Thanks a lot for

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Colin McCabe
On Mon, Dec 11, 2017, at 13:06, Dong Lin wrote: > Hey Colin, > > I went over the latest KIP wiki and have a few comments here. > > 1) The KIP says that client ID is a string if the session belongs to a > Kafka consumer. And it is a numerical follower Id if the session belongs > to a follower.

[jira] [Created] (KAFKA-6346) Consolidate multiple background async log operations

2017-12-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6346: Summary: Consolidate multiple background async log operations Key: KAFKA-6346 URL: https://issues.apache.org/jira/browse/KAFKA-6346 Project: Kafka Issue

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2017-12-11 Thread Dong Lin
Hey Jun, Thanks for the comment. I am open to improve this KIP to address more problems. I probably need more help in understanding what is the current problem with consumer using outdated metadata and whether it is easier to address it together with this KIP. I agree that a consumer can

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Colin McCabe
On Mon, Dec 11, 2017, at 13:17, Dong Lin wrote: > On Thu, Dec 7, 2017 at 1:52 PM, Colin McCabe wrote: > > > On Wed, Dec 6, 2017, at 11:23, Becket Qin wrote: > > > Hi Colin, > > > > > > >A full fetch request will certainly avoid any ambiguity here. But now > > > >we're back

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Becket Qin
Hi Jun, Yes, I agree avoiding reading the log segment is not the primary goal for this KIP. I brought this up because recently I saw a significant throughput impact when a broker is down for 20 - 30 min and rejoins a cluster. The bytes in rate could drop by 50% when that broker is trying to catch

Build failed in Jenkins: kafka-0.11.0-jdk7 #345

2017-12-11 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Increase number of messages in replica verification tool test -- [...truncated 2.61 MB...]

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Dong Lin
Hey Colin, I went over the latest KIP wiki and have a few comments here. 1) The KIP says that client ID is a string if the session belongs to a Kafka consumer. And it is a numerical follower Id if the session belongs to a follower. Can we have a consistent type for the client Id? 2) "The

Re: [DISCUSS] KIP-234: add support for getting topic defaults from AdminClient

2017-12-11 Thread dan
Dong, I agree that it *may* be better for a user to be explicit, however there are a couple reasons they may not. 1) a user doesn't even know what the options are. imagine writing a tool for users to create topics that steps them through things: $ kafka-topics.sh --create Give your topic a name:

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2017-12-11 Thread Jun Rao
Hi, Dong, Thanks for the reply. My suggestion of forcing the metadata refresh from the controller may not work in general since the cached controller could be outdated too. The general problem is that if a consumer's metadata is outdated, it may get stuck with the old leader for a long time. We

Re: [VOTE] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-11 Thread Ewen Cheslack-Postava
+1 (binding) -Ewen On Mon, Dec 11, 2017 at 12:40 PM, Gwen Shapira wrote: > +1 (binding) - nice API improvement, thanks for driving it! > > On Mon, Dec 11, 2017 at 11:52 AM Xavier Léauté > wrote: > > > Thanks Steven, I believe I addressed all the

[GitHub] kafka-site pull request #111: KAFKA-6334: fix typo in backwards compatibilit...

2017-12-11 Thread noslowerdna
GitHub user noslowerdna opened a pull request: https://github.com/apache/kafka-site/pull/111 KAFKA-6334: fix typo in backwards compatibility note You can merge this pull request into a Git repository by running: $ git pull https://github.com/noslowerdna/kafka-site KAFKA-6334

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

2017-12-11 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Increase number of messages in replica verification tool test -- [...truncated 3.39 MB...] kafka.utils.CoreUtilsTest > testCsvList STARTED

Re: [DISCUSS] KIP-236 Interruptible Partition Reassignment

2017-12-11 Thread Jun Rao
Another question is on the compatibility. Since now there are 2 ways of specifying a partition reassignment, one under /admin/reassign_partitions and the other under /admin/reassignments, we probably want to prevent the same topic being reassigned under both paths at the same time? Thanks, Jun

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Jun Rao
Hi, Jiangjie, Thanks for the info. I was thinking of doing the scan of the log segment on every fetch request as we do today. The optimization for this KIP is probably mostly useful for real time consumption, in which case the log segments that need to be accessed are likely still in pagecache.

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-11 Thread Dong Lin
On Thu, Dec 7, 2017 at 1:52 PM, Colin McCabe wrote: > On Wed, Dec 6, 2017, at 11:23, Becket Qin wrote: > > Hi Colin, > > > > >A full fetch request will certainly avoid any ambiguity here. But now > > >we're back to sending full fetch requests whenever there are network > >

Re: [VOTE] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-11 Thread Gwen Shapira
+1 (binding) - nice API improvement, thanks for driving it! On Mon, Dec 11, 2017 at 11:52 AM Xavier Léauté wrote: > Thanks Steven, I believe I addressed all the comments. If the it looks good > to you let's move forward on the vote. > > On Sat, Dec 9, 2017 at 12:50 AM

[jira] [Resolved] (KAFKA-1530) howto update continuously

2017-12-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1530?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-1530. -- Resolution: Fixed > howto update continuously > - > >

Re: [DISCUSS] KIP-228 Negative record timestamp support

2017-12-11 Thread Konstantin Chukhlomin
Hi all, I've updated KIP with few more details: Added (proposed) Changes in binary message format Added Changes from producer

Re: [VOTE] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-11 Thread Xavier Léauté
Thanks Steven, I believe I addressed all the comments. If the it looks good to you let's move forward on the vote. On Sat, Dec 9, 2017 at 12:50 AM Steven Aerts wrote: > Hello Xavier, > > for me it is perfect to take it along. > I made a few small remarks in your PR. > >

[jira] [Created] (KAFKA-6345) NetworkClient.inFlightRequestCount() is not thread safe, causing ConcurrentModificationExceptions when sensors are read

2017-12-11 Thread radai rosenblatt (JIRA)
radai rosenblatt created KAFKA-6345: --- Summary: NetworkClient.inFlightRequestCount() is not thread safe, causing ConcurrentModificationExceptions when sensors are read Key: KAFKA-6345 URL:

[jira] [Resolved] (KAFKA-1561) Data Loss for Incremented Replica Factor and Leader Election

2017-12-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1561?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-1561. -- Resolution: Fixed > Data Loss for Incremented Replica Factor and Leader Election >

[GitHub] kafka pull request #4312: MINOR: Increase number of messages in replica veri...

2017-12-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4312 ---

[GitHub] kafka pull request #4312: MINOR: Increase number of messages in replica veri...

2017-12-11 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/4312 MINOR: Increase number of messages in replica verification tool test Increase the number of messages produced to make the test more reliable. The test failed in a recent build and also fails

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

2017-12-11 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-trunk-jdk9 #248

2017-12-11 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2944: KAFKA-3940: Replaced File.mkdir/mkdirs/delete by t...

2017-12-11 Thread mimaison
Github user mimaison closed the pull request at: https://github.com/apache/kafka/pull/2944 ---

[GitHub] kafka pull request #4311: KAFKA-6298 - Line numbers on log messages are inco...

2017-12-11 Thread mrnakumar
GitHub user mrnakumar opened a pull request: https://github.com/apache/kafka/pull/4311 KAFKA-6298 - Line numbers on log messages are incorrect Modified LogContext.KafkaLogger to add support for location aware logging. If LocationAwareLogger is not available then fallback to the

[jira] [Created] (KAFKA-6344) 0.8.2 clients will store invalid configuration in ZK for Kafka 1.0 brokers

2017-12-11 Thread Vincent Bernat (JIRA)
Vincent Bernat created KAFKA-6344: - Summary: 0.8.2 clients will store invalid configuration in ZK for Kafka 1.0 brokers Key: KAFKA-6344 URL: https://issues.apache.org/jira/browse/KAFKA-6344 Project:

[GitHub] kafka pull request #4215: KAFKA-6121: Restore and global consumer should not...

2017-12-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4215 ---

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2017-12-11 Thread Ted Yu
+1 Original message From: Tom Bentley Date: 12/11/17 6:06 AM (GMT-08:00) To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-86: Configurable SASL callback handlers +1 (non-binding) On 5 May 2017 at 11:57, Mickael Maison wrote:

[jira] [Resolved] (KAFKA-6283) Configuration of custom SCRAM SaslServer implementations

2017-12-11 Thread Tom Bentley (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tom Bentley resolved KAFKA-6283. Resolution: Duplicate > Configuration of custom SCRAM SaslServer implementations >

[GitHub] kafka pull request #4274: KAFKA-6283: Configuration of custom SCRAM SaslServ...

2017-12-11 Thread tombentley
Github user tombentley closed the pull request at: https://github.com/apache/kafka/pull/4274 ---

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2017-12-11 Thread Tom Bentley
+1 (non-binding) On 5 May 2017 at 11:57, Mickael Maison wrote: > Thanks for the KIP Rajini, this will significantly simplify providing > custom credential providers > +1 (non binding) > > On Wed, May 3, 2017 at 8:25 AM, Rajini Sivaram > wrote:

RE: [DISCUSS]KIP-235 DNS alias and secured connections

2017-12-11 Thread Skrzypek, Jonathan
I agree that if there are hostnames in the list which don't correspond to any principal, then the connection will fail, but that's the way the SASL authentication with Kerberos works anyways, so we're not breaking anything here I think. This is the current behaviour, if you put 3 FQDNs in

[jira] [Created] (KAFKA-6343) OOM as the result of creation of 5k topics

2017-12-11 Thread Alex Dunayevsky (JIRA)
Alex Dunayevsky created KAFKA-6343: -- Summary: OOM as the result of creation of 5k topics Key: KAFKA-6343 URL: https://issues.apache.org/jira/browse/KAFKA-6343 Project: Kafka Issue Type: Bug

Re: Requesting permission to publish KIP

2017-12-11 Thread Viktor Somogyi
Thanks Jun, will publish my KIP soon! :) On Wed, Dec 6, 2017 at 6:43 PM, Jun Rao wrote: > Hi, Viktor, > > Just gave you the wiki permission. > > Thanks, > > Jun > > On Wed, Dec 6, 2017 at 8:51 AM, Viktor Somogyi > wrote: > > > Hi Kafka Owners, > > >

Re: 答复: [VOTE] KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-12-11 Thread Mickael Maison
Sorry to only raise this now, but should we also update the kafka-consumer-groups tool to display the start offset (and possibly the lead) ? Apart from that I'm +1 (non binding) Thanks On Mon, Dec 11, 2017 at 4:19 AM, Guozhang Wang wrote: > The by-laws ask for 72 hours only,

[jira] [Created] (KAFKA-6342) Remove workaround for JSON parsing of non-escaped strings

2017-12-11 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-6342: - Summary: Remove workaround for JSON parsing of non-escaped strings Key: KAFKA-6342 URL: https://issues.apache.org/jira/browse/KAFKA-6342 Project: Kafka

Re: [VOTE] KIP-225 - Use tags for consumer “records.lag” metrics

2017-12-11 Thread charly molter
Hi, The KIP has been updated. As it has change should I restart the vote? In any case I'm still missing one binding vote if anyone wants to help. Thanks! On Wed, Dec 6, 2017 at 6:13 PM, charly molter wrote: > Sounds good I'll update the KIP > > On Wed, Dec 6, 2017 at

[jira] [Created] (KAFKA-6341) 'networkThreadTimeNanos' in KafkaChannel is not thread safe

2017-12-11 Thread huxihx (JIRA)
huxihx created KAFKA-6341: - Summary: 'networkThreadTimeNanos' in KafkaChannel is not thread safe Key: KAFKA-6341 URL: https://issues.apache.org/jira/browse/KAFKA-6341 Project: Kafka Issue Type: Bug